From ccfd04a1cea9166fb877882cc5bba7e9151d7fbd Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Fri, 28 Feb 2025 23:20:40 +0000 Subject: [PATCH 1/7] Split datasources --- Cargo.lock | 108 +- Cargo.toml | 12 +- datafusion-examples/Cargo.toml | 2 +- .../examples/advanced_parquet_index.rs | 6 +- .../examples/csv_json_opener.rs | 6 +- datafusion/catalog-listing/src/helpers.rs | 5 + datafusion/catalog/src/session.rs | 11 +- datafusion/core/Cargo.toml | 19 +- .../core/src/datasource/file_format/avro.rs | 201 +-- .../core/src/datasource/file_format/csv.rs | 883 ++-------- .../core/src/datasource/file_format/json.rs | 427 +---- .../core/src/datasource/file_format/mod.rs | 362 +--- .../src/datasource/file_format/options.rs | 15 +- .../src/datasource/file_format/parquet.rs | 1497 ++--------------- .../core/src/datasource/listing/table.rs | 15 +- datafusion/core/src/datasource/mod.rs | 226 ++- .../datasource/physical_plan/arrow_file.rs | 5 +- .../core/src/datasource/physical_plan/avro.rs | 366 +--- .../core/src/datasource/physical_plan/csv.rs | 856 +--------- .../core/src/datasource/physical_plan/json.rs | 471 +----- .../core/src/datasource/physical_plan/mod.rs | 144 +- .../{parquet/mod.rs => parquet.rs} | 613 +------ datafusion/core/src/datasource/statistics.rs | 75 - datafusion/core/src/execution/context/csv.rs | 2 +- datafusion/core/src/execution/context/json.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 4 +- .../core/src/execution/context/parquet.rs | 2 +- .../core/src/execution/session_state.rs | 15 +- .../src/execution/session_state_defaults.rs | 2 + datafusion/core/src/test/mod.rs | 19 +- datafusion/core/src/test_util/mod.rs | 42 +- .../tests/parquet/external_access_plan.rs | 2 +- .../core/tests/parquet/schema_coercion.rs | 15 +- datafusion/datasource-avro/Cargo.toml | 60 + datafusion/datasource-avro/LICENSE.txt | 1 + datafusion/datasource-avro/NOTICE.txt | 1 + datafusion/datasource-avro/README.md | 26 + .../src}/avro_to_arrow/arrow_array_reader.rs | 36 +- .../src}/avro_to_arrow/mod.rs | 22 +- .../src}/avro_to_arrow/reader.rs | 18 +- .../src}/avro_to_arrow/schema.rs | 10 +- datafusion/datasource-avro/src/file_format.rs | 160 ++ datafusion/datasource-avro/src/mod.rs | 30 + datafusion/datasource-avro/src/source.rs | 282 ++++ datafusion/datasource-csv/Cargo.toml | 60 + datafusion/datasource-csv/LICENSE.txt | 1 + datafusion/datasource-csv/NOTICE.txt | 1 + datafusion/datasource-csv/README.md | 26 + datafusion/datasource-csv/src/file_format.rs | 736 ++++++++ datafusion/datasource-csv/src/mod.rs | 38 + datafusion/datasource-csv/src/source.rs | 781 +++++++++ datafusion/datasource-json/Cargo.toml | 56 + datafusion/datasource-json/LICENSE.txt | 1 + datafusion/datasource-json/NOTICE.txt | 1 + datafusion/datasource-json/README.md | 26 + datafusion/datasource-json/src/file_format.rs | 414 +++++ datafusion/datasource-json/src/mod.rs | 21 + datafusion/datasource-json/src/source.rs | 440 +++++ datafusion/datasource-parquet/Cargo.toml | 65 + datafusion/datasource-parquet/LICENSE.txt | 1 + datafusion/datasource-parquet/NOTICE.txt | 1 + datafusion/datasource-parquet/README.md | 26 + .../src}/access_plan.rs | 2 +- .../datasource-parquet/src/file_format.rs | 1416 ++++++++++++++++ .../src}/metrics.rs | 2 +- datafusion/datasource-parquet/src/mod.rs | 547 ++++++ .../src}/opener.rs | 18 +- .../src}/page_filter.rs | 2 +- .../src}/reader.rs | 4 +- .../src}/row_filter.rs | 14 +- .../src}/row_group_filter.rs | 9 +- .../src}/source.rs | 37 +- .../src}/writer.rs | 2 +- datafusion/datasource/Cargo.toml | 5 +- datafusion/datasource/src/decoder.rs | 191 +++ datafusion/datasource/src/file_scan_config.rs | 2 +- datafusion/datasource/src/mod.rs | 113 +- .../src}/schema_adapter.rs | 222 +-- datafusion/datasource/src/test_util.rs | 15 +- 79 files changed, 6565 insertions(+), 5807 deletions(-) rename datafusion/core/src/datasource/physical_plan/{parquet/mod.rs => parquet.rs} (72%) create mode 100644 datafusion/datasource-avro/Cargo.toml create mode 120000 datafusion/datasource-avro/LICENSE.txt create mode 120000 datafusion/datasource-avro/NOTICE.txt create mode 100644 datafusion/datasource-avro/README.md rename datafusion/{core/src/datasource => datasource-avro/src}/avro_to_arrow/arrow_array_reader.rs (99%) rename datafusion/{core/src/datasource => datasource-avro/src}/avro_to_arrow/mod.rs (67%) rename datafusion/{core/src/datasource => datasource-avro/src}/avro_to_arrow/reader.rs (95%) rename datafusion/{core/src/datasource => datasource-avro/src}/avro_to_arrow/schema.rs (98%) create mode 100644 datafusion/datasource-avro/src/file_format.rs create mode 100644 datafusion/datasource-avro/src/mod.rs create mode 100644 datafusion/datasource-avro/src/source.rs create mode 100644 datafusion/datasource-csv/Cargo.toml create mode 120000 datafusion/datasource-csv/LICENSE.txt create mode 120000 datafusion/datasource-csv/NOTICE.txt create mode 100644 datafusion/datasource-csv/README.md create mode 100644 datafusion/datasource-csv/src/file_format.rs create mode 100644 datafusion/datasource-csv/src/mod.rs create mode 100644 datafusion/datasource-csv/src/source.rs create mode 100644 datafusion/datasource-json/Cargo.toml create mode 120000 datafusion/datasource-json/LICENSE.txt create mode 120000 datafusion/datasource-json/NOTICE.txt create mode 100644 datafusion/datasource-json/README.md create mode 100644 datafusion/datasource-json/src/file_format.rs create mode 100644 datafusion/datasource-json/src/mod.rs create mode 100644 datafusion/datasource-json/src/source.rs create mode 100644 datafusion/datasource-parquet/Cargo.toml create mode 120000 datafusion/datasource-parquet/LICENSE.txt create mode 120000 datafusion/datasource-parquet/NOTICE.txt create mode 100644 datafusion/datasource-parquet/README.md rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/access_plan.rs (99%) create mode 100644 datafusion/datasource-parquet/src/file_format.rs rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/metrics.rs (99%) create mode 100644 datafusion/datasource-parquet/src/mod.rs rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/opener.rs (95%) rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/page_filter.rs (99%) rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/reader.rs (98%) rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/row_filter.rs (98%) rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/row_group_filter.rs (99%) rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/source.rs (94%) rename datafusion/{core/src/datasource/physical_plan/parquet => datasource-parquet/src}/writer.rs (98%) create mode 100644 datafusion/datasource/src/decoder.rs rename datafusion/{core/src/datasource => datasource/src}/schema_adapter.rs (69%) diff --git a/Cargo.lock b/Cargo.lock index e005fb08877c..0671b69c9841 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1719,7 +1719,6 @@ dependencies = [ name = "datafusion" version = "45.0.0" dependencies = [ - "apache-avro", "arrow", "arrow-ipc", "arrow-schema", @@ -1735,6 +1734,10 @@ dependencies = [ "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", + "datafusion-datasource-avro", + "datafusion-datasource-csv", + "datafusion-datasource-json", + "datafusion-datasource-parquet", "datafusion-doc", "datafusion-execution", "datafusion-expr", @@ -1759,7 +1762,6 @@ dependencies = [ "itertools 0.14.0", "log", "nix", - "num-traits", "object_store", "parking_lot", "parquet", @@ -1930,6 +1932,7 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", + "parquet", "rand 0.8.5", "tempfile", "tokio", @@ -1939,6 +1942,107 @@ dependencies = [ "zstd", ] +[[package]] +name = "datafusion-datasource-avro" +version = "45.0.0" +dependencies = [ + "apache-avro", + "arrow", + "async-trait", + "bytes", + "chrono", + "datafusion-catalog", + "datafusion-common", + "datafusion-datasource", + "datafusion-execution", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "futures", + "num-traits", + "object_store", + "rstest", + "serde_json", + "tokio", +] + +[[package]] +name = "datafusion-datasource-csv" +version = "45.0.0" +dependencies = [ + "arrow", + "async-trait", + "bytes", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "futures", + "itertools 0.14.0", + "log", + "object_store", + "rand 0.8.5", + "regex", + "tokio", + "url", +] + +[[package]] +name = "datafusion-datasource-json" +version = "45.0.0" +dependencies = [ + "arrow", + "async-trait", + "bytes", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "futures", + "object_store", + "serde_json", + "tokio", +] + +[[package]] +name = "datafusion-datasource-parquet" +version = "45.0.0" +dependencies = [ + "arrow", + "async-trait", + "bytes", + "chrono", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-optimizer", + "datafusion-physical-plan", + "futures", + "itertools 0.14.0", + "log", + "object_store", + "parking_lot", + "parquet", + "rand 0.8.5", + "tokio", +] + [[package]] name = "datafusion-doc" version = "45.0.0" diff --git a/Cargo.toml b/Cargo.toml index efdcb1082b1d..de0f8c415c67 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,12 +16,16 @@ # under the License. [workspace] -exclude = ["dev/depcheck"] members = [ "datafusion/common", "datafusion/common-runtime", "datafusion/catalog", "datafusion/catalog-listing", + "datafusion/datasource", + "datafusion/datasource-avro", + "datafusion/datasource-csv", + "datafusion/datasource-json", + "datafusion/datasource-parquet", "datafusion/core", "datafusion/expr", "datafusion/expr-common", @@ -57,6 +61,7 @@ members = [ "datafusion/macros", "datafusion/doc", ] +exclude = ["dev/depcheck"] resolver = "2" [workspace.package] @@ -80,6 +85,7 @@ version = "45.0.0" ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } +apache-avro = { version = "0.17", default-features = false } arrow = { version = "54.2.1", features = [ "prettyprint", "chrono-tz", @@ -106,6 +112,10 @@ datafusion-catalog-listing = { path = "datafusion/catalog-listing", version = "4 datafusion-common = { path = "datafusion/common", version = "45.0.0", default-features = false } datafusion-common-runtime = { path = "datafusion/common-runtime", version = "45.0.0" } datafusion-datasource = { path = "datafusion/datasource", version = "45.0.0", default-features = false } +datafusion-datasource-avro = { path = "datafusion/datasource-avro", version = "45.0.0", default-features = false } +datafusion-datasource-csv = { path = "datafusion/datasource-csv", version = "45.0.0", default-features = false } +datafusion-datasource-json = { path = "datafusion/datasource-json", version = "45.0.0", default-features = false } +datafusion-datasource-parquet = { path = "datafusion/datasource-parquet", version = "45.0.0", default-features = false } datafusion-doc = { path = "datafusion/doc", version = "45.0.0" } datafusion-execution = { path = "datafusion/execution", version = "45.0.0" } datafusion-expr = { path = "datafusion/expr", version = "45.0.0" } diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index ea3139adac3d..d2bbdd78e3f2 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -61,7 +61,7 @@ async-trait = { workspace = true } bytes = { workspace = true } dashmap = { workspace = true } # note only use main datafusion crate for examples -datafusion = { workspace = true, default-features = true, features = ["avro"] } +datafusion = { workspace = true, default-features = true } datafusion-proto = { workspace = true } env_logger = { workspace = true } futures = { workspace = true } diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index bb1cf3c8f78d..d6cf61c61d73 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -23,8 +23,6 @@ use std::path::{Path, PathBuf}; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; -use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; -use arrow::datatypes::SchemaRef; use datafusion::catalog::Session; use datafusion::common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, @@ -32,7 +30,7 @@ use datafusion::common::{ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; use datafusion::datasource::physical_plan::{ - parquet::ParquetFileReaderFactory, FileMeta, FileScanConfig, ParquetSource, + FileMeta, FileScanConfig, ParquetFileReaderFactory, ParquetSource, }; use datafusion::datasource::TableProvider; use datafusion::execution::object_store::ObjectStoreUrl; @@ -53,6 +51,8 @@ use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; +use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use arrow::datatypes::SchemaRef; use async_trait::async_trait; use bytes::Bytes; use futures::future::BoxFuture; diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index 574137afe5c9..6dc38a436a0c 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -18,15 +18,15 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema}; -use datafusion::datasource::physical_plan::JsonSource; use datafusion::{ assert_batches_eq, - datasource::physical_plan::FileSource, datasource::{ file_format::file_compression_type::FileCompressionType, listing::PartitionedFile, object_store::ObjectStoreUrl, - physical_plan::{CsvSource, FileScanConfig, FileStream, JsonOpener}, + physical_plan::{ + CsvSource, FileScanConfig, FileSource, FileStream, JsonOpener, JsonSource, + }, }, error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, diff --git a/datafusion/catalog-listing/src/helpers.rs b/datafusion/catalog-listing/src/helpers.rs index cf475263535a..f86bc0744816 100644 --- a/datafusion/catalog-listing/src/helpers.rs +++ b/datafusion/catalog-listing/src/helpers.rs @@ -532,6 +532,7 @@ pub fn describe_partition(partition: &Partition) -> (&str, usize, Vec<&str>) { #[cfg(test)] mod tests { use async_trait::async_trait; + use datafusion_common::config::TableOptions; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use futures::FutureExt; @@ -1068,5 +1069,9 @@ mod tests { fn as_any(&self) -> &dyn Any { unimplemented!() } + + fn table_options(&self) -> &TableOptions { + unimplemented!() + } } } diff --git a/datafusion/catalog/src/session.rs b/datafusion/catalog/src/session.rs index db49529ac43f..8915862dd028 100644 --- a/datafusion/catalog/src/session.rs +++ b/datafusion/catalog/src/session.rs @@ -16,7 +16,7 @@ // under the License. use async_trait::async_trait; -use datafusion_common::config::ConfigOptions; +use datafusion_common::config::{ConfigOptions, TableOptions}; use datafusion_common::{DFSchema, Result}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; @@ -120,6 +120,15 @@ pub trait Session: Send + Sync { fn execution_props(&self) -> &ExecutionProps; fn as_any(&self) -> &dyn Any; + + /// Return the table options + fn table_options(&self) -> &TableOptions; + + /// return the TableOptions options with its extensions + fn default_table_options(&self) -> TableOptions { + self.table_options() + .combine_with_session_config(self.config_options()) + } } /// Create a new task context instance from Session diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 438e2600a66d..477ac3e75310 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -40,9 +40,15 @@ nested_expressions = ["datafusion-functions-nested"] # This feature is deprecated. Use the `nested_expressions` feature instead. array_expressions = ["nested_expressions"] # Used to enable the avro format -avro = ["apache-avro", "num-traits", "datafusion-common/avro", "datafusion-datasource/avro"] +avro = ["datafusion-common/avro", "datafusion-datasource-avro"] backtrace = ["datafusion-common/backtrace"] -compression = ["xz2", "bzip2", "flate2", "zstd", "datafusion-datasource/compression"] +compression = [ + "xz2", + "bzip2", + "flate2", + "zstd", + "datafusion-datasource/compression", +] crypto_expressions = ["datafusion-functions/crypto_expressions"] datetime_expressions = ["datafusion-functions/datetime_expressions"] default = [ @@ -54,6 +60,7 @@ default = [ "string_expressions", "unicode_expressions", "compression", + "avro", "parquet", "recursive_protection", ] @@ -61,7 +68,7 @@ encoding_expressions = ["datafusion-functions/encoding_expressions"] # Used for testing ONLY: causes all values to hash to the same value (test for collisions) force_hash_collisions = ["datafusion-physical-plan/force_hash_collisions", "datafusion-common/force_hash_collisions"] math_expressions = ["datafusion-functions/math_expressions"] -parquet = ["datafusion-common/parquet", "dep:parquet"] +parquet = ["datafusion-common/parquet", "dep:parquet", "datafusion-datasource-parquet"] pyarrow = ["datafusion-common/pyarrow", "parquet"] regex_expressions = [ "datafusion-functions/regex_expressions", @@ -82,7 +89,6 @@ unicode_expressions = [ extended_tests = [] [dependencies] -apache-avro = { version = "0.17", optional = true } arrow = { workspace = true } arrow-ipc = { workspace = true } arrow-schema = { workspace = true } @@ -95,6 +101,10 @@ datafusion-catalog-listing = { workspace = true } datafusion-common = { workspace = true, features = ["object_store"] } datafusion-common-runtime = { workspace = true } datafusion-datasource = { workspace = true } +datafusion-datasource-avro = { workspace = true, optional = true } +datafusion-datasource-csv = { workspace = true } +datafusion-datasource-json = { workspace = true } +datafusion-datasource-parquet = { workspace = true, optional = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true } @@ -114,7 +124,6 @@ flate2 = { version = "1.1.0", optional = true } futures = { workspace = true } itertools = { workspace = true } log = { workspace = true } -num-traits = { version = "0.2", optional = true } object_store = { workspace = true } parking_lot = { workspace = true } parquet = { workspace = true, optional = true, default-features = true } diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index e7314e839bf2..3723a728a08f 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -15,163 +15,31 @@ // specific language governing permissions and limitations // under the License. -//! [`AvroFormat`] Apache Avro [`FileFormat`] abstractions - -use std::any::Any; -use std::collections::HashMap; -use std::fmt; -use std::sync::Arc; - -use super::file_compression_type::FileCompressionType; -use super::FileFormat; -use super::FileFormatFactory; -use crate::datasource::avro_to_arrow::read_avro_schema_from_reader; -use crate::datasource::physical_plan::AvroSource; -use crate::error::Result; -use crate::physical_plan::ExecutionPlan; -use crate::physical_plan::Statistics; - -use arrow::datatypes::Schema; -use arrow::datatypes::SchemaRef; -use async_trait::async_trait; -use datafusion_catalog::Session; -use datafusion_common::internal_err; -use datafusion_common::parsers::CompressionTypeVariant; -use datafusion_common::GetExt; -use datafusion_common::DEFAULT_AVRO_EXTENSION; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_physical_expr::PhysicalExpr; -use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; - -#[derive(Default)] -/// Factory struct used to create [AvroFormat] -pub struct AvroFormatFactory; - -impl AvroFormatFactory { - /// Creates an instance of [AvroFormatFactory] - pub fn new() -> Self { - Self {} - } -} - -impl FileFormatFactory for AvroFormatFactory { - fn create( - &self, - _state: &dyn Session, - _format_options: &HashMap, - ) -> Result> { - Ok(Arc::new(AvroFormat)) - } +//! Re-exports the [`datafusion_datasource_avro::file_format`] module, and contains tests for it. - fn default(&self) -> Arc { - Arc::new(AvroFormat) - } - - fn as_any(&self) -> &dyn Any { - self - } -} - -impl fmt::Debug for AvroFormatFactory { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("AvroFormatFactory").finish() - } -} - -impl GetExt for AvroFormatFactory { - fn get_ext(&self) -> String { - // Removes the dot, i.e. ".parquet" -> "parquet" - DEFAULT_AVRO_EXTENSION[1..].to_string() - } -} - -/// Avro `FileFormat` implementation. -#[derive(Default, Debug)] -pub struct AvroFormat; - -#[async_trait] -impl FileFormat for AvroFormat { - fn as_any(&self) -> &dyn Any { - self - } - - fn get_ext(&self) -> String { - AvroFormatFactory::new().get_ext() - } - - fn get_ext_with_compression( - &self, - file_compression_type: &FileCompressionType, - ) -> Result { - let ext = self.get_ext(); - match file_compression_type.get_variant() { - CompressionTypeVariant::UNCOMPRESSED => Ok(ext), - _ => internal_err!("Avro FileFormat does not support compression."), - } - } - - async fn infer_schema( - &self, - _state: &dyn Session, - store: &Arc, - objects: &[ObjectMeta], - ) -> Result { - let mut schemas = vec![]; - for object in objects { - let r = store.as_ref().get(&object.location).await?; - let schema = match r.payload { - GetResultPayload::File(mut file, _) => { - read_avro_schema_from_reader(&mut file)? - } - GetResultPayload::Stream(_) => { - // TODO: Fetching entire file to get schema is potentially wasteful - let data = r.bytes().await?; - read_avro_schema_from_reader(&mut data.as_ref())? - } - }; - schemas.push(schema); - } - let merged_schema = Schema::try_merge(schemas)?; - Ok(Arc::new(merged_schema)) - } - - async fn infer_stats( - &self, - _state: &dyn Session, - _store: &Arc, - table_schema: SchemaRef, - _object: &ObjectMeta, - ) -> Result { - Ok(Statistics::new_unknown(&table_schema)) - } - - async fn create_physical_plan( - &self, - _state: &dyn Session, - conf: FileScanConfig, - _filters: Option<&Arc>, - ) -> Result> { - Ok(conf.with_source(self.file_source()).build()) - } - - fn file_source(&self) -> Arc { - Arc::new(AvroSource::new()) - } -} +pub use datafusion_datasource_avro::file_format::*; #[cfg(test)] -#[cfg(feature = "avro")] mod tests { - use super::*; - use crate::datasource::file_format::test_util::scan_format; - use crate::physical_plan::collect; - use crate::prelude::{SessionConfig, SessionContext}; + use std::sync::Arc; + + use crate::{ + datasource::file_format::test_util::scan_format, prelude::SessionContext, + }; use arrow::array::{as_string_array, Array}; - use datafusion_common::cast::{ - as_binary_array, as_boolean_array, as_float32_array, as_float64_array, - as_int32_array, as_timestamp_microsecond_array, + use datafusion_catalog::Session; + use datafusion_common::{ + assert_batches_eq, + cast::{ + as_binary_array, as_boolean_array, as_float32_array, as_float64_array, + as_int32_array, as_timestamp_microsecond_array, + }, + test_util, Result, }; + + use datafusion_datasource_avro::AvroFormat; + use datafusion_execution::config::SessionConfig; + use datafusion_physical_plan::{collect, ExecutionPlan}; use futures::StreamExt; #[tokio::test] @@ -260,7 +128,7 @@ mod tests { "| 1 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30312f30312f3039 | 31 | 2009-01-01T00:01:00 |", "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+"]; - crate::assert_batches_eq!(expected, &batches); + assert_batches_eq!(expected, &batches); Ok(()) } @@ -510,36 +378,9 @@ mod tests { projection: Option>, limit: Option, ) -> Result> { - let testdata = crate::test_util::arrow_test_data(); + let testdata = test_util::arrow_test_data(); let store_root = format!("{testdata}/avro"); let format = AvroFormat {}; scan_format(state, &format, &store_root, file_name, projection, limit).await } } - -#[cfg(test)] -#[cfg(not(feature = "avro"))] -mod tests { - use super::*; - - use super::super::test_util::scan_format; - use crate::error::DataFusionError; - use crate::prelude::SessionContext; - - #[tokio::test] - async fn test() -> Result<()> { - let session_ctx = SessionContext::new(); - let state = session_ctx.state(); - let format = AvroFormat {}; - let testdata = crate::test_util::arrow_test_data(); - let filename = "avro/alltypes_plain.avro"; - let result = scan_format(&state, &format, &testdata, filename, None, None).await; - assert!(matches!( - result, - Err(DataFusionError::NotImplemented(msg)) - if msg == *"cannot read avro schema without the 'avro' feature enabled" - )); - - Ok(()) - } -} diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 45ad3e8c1c30..9c6d42007d12 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -15,764 +15,183 @@ // specific language governing permissions and limitations // under the License. -//! [`CsvFormat`], Comma Separated Value (CSV) [`FileFormat`] abstractions - -use std::any::Any; -use std::collections::{HashMap, HashSet}; -use std::fmt::{self, Debug}; -use std::sync::Arc; - -use super::write::orchestration::spawn_writer_tasks_and_join; -use super::{ - Decoder, DecoderDeserializer, FileFormat, FileFormatFactory, - DEFAULT_SCHEMA_INFER_MAX_RECORD, -}; -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; -use crate::datasource::file_format::write::BatchSerializer; -use crate::datasource::physical_plan::{CsvSource, FileSink, FileSinkConfig}; -use crate::error::Result; -use crate::execution::context::SessionState; -use crate::physical_plan::insert::{DataSink, DataSinkExec}; -use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics, -}; - -use arrow::array::RecordBatch; -use arrow::csv::WriterBuilder; -use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; -use arrow::error::ArrowError; -use datafusion_catalog::Session; -use datafusion_common::config::{ConfigField, ConfigFileType, CsvOptions}; -use datafusion_common::file_options::csv_writer::CsvWriterOptions; -use datafusion_common::{ - exec_err, not_impl_err, DataFusionError, GetExt, DEFAULT_CSV_EXTENSION, -}; -use datafusion_common_runtime::SpawnedTask; -use datafusion_datasource::display::FileGroupDisplay; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_execution::TaskContext; -use datafusion_expr::dml::InsertOp; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; - -use async_trait::async_trait; -use bytes::{Buf, Bytes}; -use futures::stream::BoxStream; -use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; -use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; -use regex::Regex; - -#[derive(Default)] -/// Factory struct used to create [CsvFormatFactory] -pub struct CsvFormatFactory { - /// the options for csv file read - pub options: Option, -} - -impl CsvFormatFactory { - /// Creates an instance of [CsvFormatFactory] - pub fn new() -> Self { - Self { options: None } - } - - /// Creates an instance of [CsvFormatFactory] with customized default options - pub fn new_with_options(options: CsvOptions) -> Self { - Self { - options: Some(options), - } - } -} +//! Re-exports the [`datafusion_datasource_csv::file_format`] module, and contains tests for it. -impl Debug for CsvFormatFactory { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("CsvFormatFactory") - .field("options", &self.options) - .finish() - } -} - -impl FileFormatFactory for CsvFormatFactory { - fn create( - &self, - state: &dyn Session, - format_options: &HashMap, - ) -> Result> { - let state = state.as_any().downcast_ref::().unwrap(); - let csv_options = match &self.options { - None => { - let mut table_options = state.default_table_options(); - table_options.set_config_format(ConfigFileType::CSV); - table_options.alter_with_string_hash_map(format_options)?; - table_options.csv - } - Some(csv_options) => { - let mut csv_options = csv_options.clone(); - for (k, v) in format_options { - csv_options.set(k, v)?; - } - csv_options - } - }; +pub use datafusion_datasource_csv::file_format::*; - Ok(Arc::new(CsvFormat::default().with_options(csv_options))) - } - - fn default(&self) -> Arc { - Arc::new(CsvFormat::default()) - } - - fn as_any(&self) -> &dyn Any { - self - } -} - -impl GetExt for CsvFormatFactory { - fn get_ext(&self) -> String { - // Removes the dot, i.e. ".parquet" -> "parquet" - DEFAULT_CSV_EXTENSION[1..].to_string() - } -} - -/// Character Separated Value `FileFormat` implementation. -#[derive(Debug, Default)] -pub struct CsvFormat { - options: CsvOptions, -} - -impl CsvFormat { - /// Return a newline delimited stream from the specified file on - /// Stream, decompressing if necessary - /// Each returned `Bytes` has a whole number of newline delimited rows - async fn read_to_delimited_chunks<'a>( - &self, - store: &Arc, - object: &ObjectMeta, - ) -> BoxStream<'a, Result> { - // stream to only read as many rows as needed into memory - let stream = store - .get(&object.location) - .await - .map_err(DataFusionError::ObjectStore); - let stream = match stream { - Ok(stream) => self - .read_to_delimited_chunks_from_stream( - stream - .into_stream() - .map_err(DataFusionError::ObjectStore) - .boxed(), - ) - .await - .map_err(DataFusionError::from) - .left_stream(), - Err(e) => { - futures::stream::once(futures::future::ready(Err(e))).right_stream() - } - }; - stream.boxed() - } - - async fn read_to_delimited_chunks_from_stream<'a>( - &self, - stream: BoxStream<'a, Result>, - ) -> BoxStream<'a, Result> { - let file_compression_type: FileCompressionType = self.options.compression.into(); - let decoder = file_compression_type.convert_stream(stream); - let steam = match decoder { - Ok(decoded_stream) => { - newline_delimited_stream(decoded_stream.map_err(|e| match e { - DataFusionError::ObjectStore(e) => e, - err => object_store::Error::Generic { - store: "read to delimited chunks failed", - source: Box::new(err), - }, - })) - .map_err(DataFusionError::from) - .left_stream() - } - Err(e) => { - futures::stream::once(futures::future::ready(Err(e))).right_stream() - } - }; - steam.boxed() - } - - /// Set the csv options - pub fn with_options(mut self, options: CsvOptions) -> Self { - self.options = options; - self - } - - /// Retrieve the csv options - pub fn options(&self) -> &CsvOptions { - &self.options - } - - /// Set a limit in terms of records to scan to infer the schema - /// - default to `DEFAULT_SCHEMA_INFER_MAX_RECORD` - pub fn with_schema_infer_max_rec(mut self, max_rec: usize) -> Self { - self.options.schema_infer_max_rec = Some(max_rec); - self - } - - /// Set true to indicate that the first line is a header. - /// - default to true - pub fn with_has_header(mut self, has_header: bool) -> Self { - self.options.has_header = Some(has_header); - self - } - - /// Set the regex to use for null values in the CSV reader. - /// - default to treat empty values as null. - pub fn with_null_regex(mut self, null_regex: Option) -> Self { - self.options.null_regex = null_regex; - self - } - - /// Returns `Some(true)` if the first line is a header, `Some(false)` if - /// it is not, and `None` if it is not specified. - pub fn has_header(&self) -> Option { - self.options.has_header - } - - /// Lines beginning with this byte are ignored. - pub fn with_comment(mut self, comment: Option) -> Self { - self.options.comment = comment; - self - } - - /// The character separating values within a row. - /// - default to ',' - pub fn with_delimiter(mut self, delimiter: u8) -> Self { - self.options.delimiter = delimiter; - self - } - - /// The quote character in a row. - /// - default to '"' - pub fn with_quote(mut self, quote: u8) -> Self { - self.options.quote = quote; - self - } - - /// The escape character in a row. - /// - default is None - pub fn with_escape(mut self, escape: Option) -> Self { - self.options.escape = escape; - self - } - - /// The character used to indicate the end of a row. - /// - default to None (CRLF) - pub fn with_terminator(mut self, terminator: Option) -> Self { - self.options.terminator = terminator; - self - } - - /// Specifies whether newlines in (quoted) values are supported. - /// - /// Parsing newlines in quoted values may be affected by execution behaviour such as - /// parallel file scanning. Setting this to `true` ensures that newlines in values are - /// parsed successfully, which may reduce performance. - /// - /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. - pub fn with_newlines_in_values(mut self, newlines_in_values: bool) -> Self { - self.options.newlines_in_values = Some(newlines_in_values); - self - } - - /// Set a `FileCompressionType` of CSV - /// - defaults to `FileCompressionType::UNCOMPRESSED` - pub fn with_file_compression_type( - mut self, - file_compression_type: FileCompressionType, - ) -> Self { - self.options.compression = file_compression_type.into(); - self - } - - /// The delimiter character. - pub fn delimiter(&self) -> u8 { - self.options.delimiter - } - - /// The quote character. - pub fn quote(&self) -> u8 { - self.options.quote - } - - /// The escape character. - pub fn escape(&self) -> Option { - self.options.escape - } -} - -#[derive(Debug)] -pub(crate) struct CsvDecoder { - inner: arrow::csv::reader::Decoder, -} - -impl CsvDecoder { - pub(crate) fn new(decoder: arrow::csv::reader::Decoder) -> Self { - Self { inner: decoder } - } -} - -impl Decoder for CsvDecoder { - fn decode(&mut self, buf: &[u8]) -> Result { - self.inner.decode(buf) - } - - fn flush(&mut self) -> Result, ArrowError> { - self.inner.flush() - } - - fn can_flush_early(&self) -> bool { - self.inner.capacity() == 0 - } -} - -impl Debug for CsvSerializer { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("CsvSerializer") - .field("header", &self.header) - .finish() - } -} +#[cfg(test)] +mod tests { + use std::fmt::{self, Display}; + use std::ops::Range; + use std::sync::{Arc, Mutex}; -impl From for DecoderDeserializer { - fn from(decoder: arrow::csv::reader::Decoder) -> Self { - DecoderDeserializer::new(CsvDecoder::new(decoder)) - } -} + use super::*; -#[async_trait] -impl FileFormat for CsvFormat { - fn as_any(&self) -> &dyn Any { - self - } + use crate::datasource::file_format::test_util::scan_format; + use crate::datasource::listing::ListingOptions; + use crate::execution::session_state::SessionStateBuilder; + use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext}; + use arrow_schema::{DataType, Field, Schema, SchemaRef}; + use datafusion_catalog::Session; + use datafusion_common::cast::as_string_array; + use datafusion_common::internal_err; + use datafusion_common::stats::Precision; + use datafusion_common::test_util::arrow_test_data; + use datafusion_common::{assert_batches_eq, Result}; + use datafusion_datasource::decoder::{ + BatchDeserializer, DecoderDeserializer, DeserializerOutput, + }; + use datafusion_datasource::file_compression_type::FileCompressionType; + use datafusion_datasource::file_format::FileFormat; + use datafusion_datasource::write::BatchSerializer; + use datafusion_expr::{col, lit}; + use datafusion_physical_plan::{collect, ExecutionPlan}; - fn get_ext(&self) -> String { - CsvFormatFactory::new().get_ext() - } + use arrow::array::{ + BooleanArray, Float64Array, Int32Array, RecordBatch, StringArray, + }; + use arrow::compute::concat_batches; + use arrow::csv::ReaderBuilder; + use arrow::util::pretty::pretty_format_batches; + use async_trait::async_trait; + use bytes::Bytes; + use chrono::DateTime; + use futures::stream::BoxStream; + use futures::StreamExt; + use object_store::local::LocalFileSystem; + use object_store::path::Path; + use object_store::{ + Attributes, GetOptions, GetResult, GetResultPayload, ListResult, MultipartUpload, + ObjectMeta, ObjectStore, PutMultipartOpts, PutOptions, PutPayload, PutResult, + }; + use regex::Regex; + use rstest::*; - fn get_ext_with_compression( - &self, - file_compression_type: &FileCompressionType, - ) -> Result { - let ext = self.get_ext(); - Ok(format!("{}{}", ext, file_compression_type.get_ext())) + /// Mock ObjectStore to provide an variable stream of bytes on get + /// Able to keep track of how many iterations of the provided bytes were repeated + #[derive(Debug)] + struct VariableStream { + bytes_to_repeat: Bytes, + max_iterations: usize, + iterations_detected: Arc>, } - async fn infer_schema( - &self, - state: &dyn Session, - store: &Arc, - objects: &[ObjectMeta], - ) -> Result { - let mut schemas = vec![]; - - let mut records_to_read = self - .options - .schema_infer_max_rec - .unwrap_or(DEFAULT_SCHEMA_INFER_MAX_RECORD); - - for object in objects { - let stream = self.read_to_delimited_chunks(store, object).await; - let (schema, records_read) = self - .infer_schema_from_stream(state, records_to_read, stream) - .await - .map_err(|err| { - DataFusionError::Context( - format!("Error when processing CSV file {}", &object.location), - Box::new(err), - ) - })?; - records_to_read -= records_read; - schemas.push(schema); - if records_to_read == 0 { - break; - } + impl Display for VariableStream { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "VariableStream") } - - let merged_schema = Schema::try_merge(schemas)?; - Ok(Arc::new(merged_schema)) - } - - async fn infer_stats( - &self, - _state: &dyn Session, - _store: &Arc, - table_schema: SchemaRef, - _object: &ObjectMeta, - ) -> Result { - Ok(Statistics::new_unknown(&table_schema)) - } - - async fn create_physical_plan( - &self, - state: &dyn Session, - mut conf: FileScanConfig, - _filters: Option<&Arc>, - ) -> Result> { - conf.file_compression_type = self.options.compression.into(); - // Consult configuration options for default values - let has_header = self - .options - .has_header - .unwrap_or(state.config_options().catalog.has_header); - let newlines_in_values = self - .options - .newlines_in_values - .unwrap_or(state.config_options().catalog.newlines_in_values); - conf.new_lines_in_values = newlines_in_values; - - let source = Arc::new( - CsvSource::new(has_header, self.options.delimiter, self.options.quote) - .with_escape(self.options.escape) - .with_terminator(self.options.terminator) - .with_comment(self.options.comment), - ); - Ok(conf.with_source(source).build()) } - async fn create_writer_physical_plan( - &self, - input: Arc, - state: &dyn Session, - conf: FileSinkConfig, - order_requirements: Option, - ) -> Result> { - if conf.insert_op != InsertOp::Append { - return not_impl_err!("Overwrites are not implemented yet for CSV"); + #[async_trait] + impl ObjectStore for VariableStream { + async fn put_opts( + &self, + _location: &Path, + _payload: PutPayload, + _opts: PutOptions, + ) -> object_store::Result { + unimplemented!() } - // `has_header` and `newlines_in_values` fields of CsvOptions may inherit - // their values from session from configuration settings. To support - // this logic, writer options are built from the copy of `self.options` - // with updated values of these special fields. - let has_header = self - .options() - .has_header - .unwrap_or(state.config_options().catalog.has_header); - let newlines_in_values = self - .options() - .newlines_in_values - .unwrap_or(state.config_options().catalog.newlines_in_values); - - let options = self - .options() - .clone() - .with_has_header(has_header) - .with_newlines_in_values(newlines_in_values); - - let writer_options = CsvWriterOptions::try_from(&options)?; - - let sink = Arc::new(CsvSink::new(conf, writer_options)); - - Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) - } - - fn file_source(&self) -> Arc { - Arc::new(CsvSource::default()) - } -} - -impl CsvFormat { - /// Return the inferred schema reading up to records_to_read from a - /// stream of delimited chunks returning the inferred schema and the - /// number of lines that were read - async fn infer_schema_from_stream( - &self, - state: &dyn Session, - mut records_to_read: usize, - stream: impl Stream>, - ) -> Result<(Schema, usize)> { - let mut total_records_read = 0; - let mut column_names = vec![]; - let mut column_type_possibilities = vec![]; - let mut record_number = -1; - - pin_mut!(stream); - - while let Some(chunk) = stream.next().await.transpose()? { - record_number += 1; - let first_chunk = record_number == 0; - let mut format = arrow::csv::reader::Format::default() - .with_header( - first_chunk - && self - .options - .has_header - .unwrap_or(state.config_options().catalog.has_header), - ) - .with_delimiter(self.options.delimiter) - .with_quote(self.options.quote); - - if let Some(null_regex) = &self.options.null_regex { - let regex = Regex::new(null_regex.as_str()) - .expect("Unable to parse CSV null regex."); - format = format.with_null_regex(regex); - } - - if let Some(escape) = self.options.escape { - format = format.with_escape(escape); - } - - if let Some(comment) = self.options.comment { - format = format.with_comment(comment); - } - - let (Schema { fields, .. }, records_read) = - format.infer_schema(chunk.reader(), Some(records_to_read))?; - - records_to_read -= records_read; - total_records_read += records_read; - - if first_chunk { - // set up initial structures for recording inferred schema across chunks - (column_names, column_type_possibilities) = fields - .into_iter() - .map(|field| { - let mut possibilities = HashSet::new(); - if records_read > 0 { - // at least 1 data row read, record the inferred datatype - possibilities.insert(field.data_type().clone()); - } - (field.name().clone(), possibilities) - }) - .unzip(); - } else { - if fields.len() != column_type_possibilities.len() { - return exec_err!( - "Encountered unequal lengths between records on CSV file whilst inferring schema. \ - Expected {} fields, found {} fields at record {}", - column_type_possibilities.len(), - fields.len(), - record_number + 1 - ); - } - - column_type_possibilities.iter_mut().zip(&fields).for_each( - |(possibilities, field)| { - possibilities.insert(field.data_type().clone()); - }, - ); - } - - if records_to_read == 0 { - break; - } + async fn put_multipart_opts( + &self, + _location: &Path, + _opts: PutMultipartOpts, + ) -> object_store::Result> { + unimplemented!() } - let schema = build_schema_helper(column_names, &column_type_possibilities); - Ok((schema, total_records_read)) - } -} - -fn build_schema_helper(names: Vec, types: &[HashSet]) -> Schema { - let fields = names - .into_iter() - .zip(types) - .map(|(field_name, data_type_possibilities)| { - // ripped from arrow::csv::reader::infer_reader_schema_with_csv_options - // determine data type based on possible types - // if there are incompatible types, use DataType::Utf8 - match data_type_possibilities.len() { - 1 => Field::new( - field_name, - data_type_possibilities.iter().next().unwrap().clone(), - true, - ), - 2 => { - if data_type_possibilities.contains(&DataType::Int64) - && data_type_possibilities.contains(&DataType::Float64) - { - // we have an integer and double, fall down to double - Field::new(field_name, DataType::Float64, true) - } else { - // default to Utf8 for conflicting datatypes (e.g bool and int) - Field::new(field_name, DataType::Utf8, true) - } - } - _ => Field::new(field_name, DataType::Utf8, true), - } - }) - .collect::(); - Schema::new(fields) -} + async fn get(&self, location: &Path) -> object_store::Result { + let bytes = self.bytes_to_repeat.clone(); + let range = 0..bytes.len() * self.max_iterations; + let arc = self.iterations_detected.clone(); + let stream = futures::stream::repeat_with(move || { + let arc_inner = arc.clone(); + *arc_inner.lock().unwrap() += 1; + Ok(bytes.clone()) + }) + .take(self.max_iterations) + .boxed(); + + Ok(GetResult { + payload: GetResultPayload::Stream(stream), + meta: ObjectMeta { + location: location.clone(), + last_modified: Default::default(), + size: range.end, + e_tag: None, + version: None, + }, + range: Default::default(), + attributes: Attributes::default(), + }) + } -impl Default for CsvSerializer { - fn default() -> Self { - Self::new() - } -} + async fn get_opts( + &self, + _location: &Path, + _opts: GetOptions, + ) -> object_store::Result { + unimplemented!() + } -/// Define a struct for serializing CSV records to a stream -pub struct CsvSerializer { - // CSV writer builder - builder: WriterBuilder, - // Flag to indicate whether there will be a header - header: bool, -} + async fn get_ranges( + &self, + _location: &Path, + _ranges: &[Range], + ) -> object_store::Result> { + unimplemented!() + } -impl CsvSerializer { - /// Constructor for the CsvSerializer object - pub fn new() -> Self { - Self { - builder: WriterBuilder::new(), - header: true, + async fn head(&self, _location: &Path) -> object_store::Result { + unimplemented!() } - } - /// Method for setting the CSV writer builder - pub fn with_builder(mut self, builder: WriterBuilder) -> Self { - self.builder = builder; - self - } + async fn delete(&self, _location: &Path) -> object_store::Result<()> { + unimplemented!() + } - /// Method for setting the CSV writer header status - pub fn with_header(mut self, header: bool) -> Self { - self.header = header; - self - } -} + fn list( + &self, + _prefix: Option<&Path>, + ) -> BoxStream<'_, object_store::Result> { + unimplemented!() + } -impl BatchSerializer for CsvSerializer { - fn serialize(&self, batch: RecordBatch, initial: bool) -> Result { - let mut buffer = Vec::with_capacity(4096); - let builder = self.builder.clone(); - let header = self.header && initial; - let mut writer = builder.with_header(header).build(&mut buffer); - writer.write(&batch)?; - drop(writer); - Ok(Bytes::from(buffer)) - } -} + async fn list_with_delimiter( + &self, + _prefix: Option<&Path>, + ) -> object_store::Result { + unimplemented!() + } -/// Implements [`DataSink`] for writing to a CSV file. -pub struct CsvSink { - /// Config options for writing data - config: FileSinkConfig, - writer_options: CsvWriterOptions, -} + async fn copy(&self, _from: &Path, _to: &Path) -> object_store::Result<()> { + unimplemented!() + } -impl Debug for CsvSink { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("CsvSink").finish() + async fn copy_if_not_exists( + &self, + _from: &Path, + _to: &Path, + ) -> object_store::Result<()> { + unimplemented!() + } } -} -impl DisplayAs for CsvSink { - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "CsvSink(file_groups=",)?; - FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; - write!(f, ")") + impl VariableStream { + pub fn new(bytes_to_repeat: Bytes, max_iterations: usize) -> Self { + Self { + bytes_to_repeat, + max_iterations, + iterations_detected: Arc::new(Mutex::new(0)), } } - } -} -impl CsvSink { - /// Create from config. - pub fn new(config: FileSinkConfig, writer_options: CsvWriterOptions) -> Self { - Self { - config, - writer_options, + pub fn get_iterations_detected(&self) -> usize { + *self.iterations_detected.lock().unwrap() } } - /// Retrieve the writer options - pub fn writer_options(&self) -> &CsvWriterOptions { - &self.writer_options - } -} - -#[async_trait] -impl FileSink for CsvSink { - fn config(&self) -> &FileSinkConfig { - &self.config - } - - async fn spawn_writer_tasks_and_join( - &self, - context: &Arc, - demux_task: SpawnedTask>, - file_stream_rx: DemuxedStreamReceiver, - object_store: Arc, - ) -> Result { - let builder = self.writer_options.writer_options.clone(); - let header = builder.header(); - let serializer = Arc::new( - CsvSerializer::new() - .with_builder(builder) - .with_header(header), - ) as _; - spawn_writer_tasks_and_join( - context, - serializer, - self.writer_options.compression.into(), - object_store, - demux_task, - file_stream_rx, - ) - .await - } -} - -#[async_trait] -impl DataSink for CsvSink { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> &SchemaRef { - self.config.output_schema() - } - - async fn write_all( - &self, - data: SendableRecordBatchStream, - context: &Arc, - ) -> Result { - FileSink::write_all(self, data, context).await - } -} - -#[cfg(test)] -mod tests { - use super::super::test_util::scan_format; - use super::*; - use crate::assert_batches_eq; - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::file_format::test_util::VariableStream; - use crate::datasource::file_format::{ - BatchDeserializer, DecoderDeserializer, DeserializerOutput, - }; - use crate::datasource::listing::ListingOptions; - use crate::execution::session_state::SessionStateBuilder; - use crate::physical_plan::collect; - use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext}; - use crate::test_util::arrow_test_data; - - use arrow::array::{BooleanArray, Float64Array, Int32Array, StringArray}; - use arrow::compute::concat_batches; - use arrow::csv::ReaderBuilder; - use arrow::util::pretty::pretty_format_batches; - use datafusion_common::cast::as_string_array; - use datafusion_common::internal_err; - use datafusion_common::stats::Precision; - use datafusion_expr::{col, lit}; - - use chrono::DateTime; - use object_store::local::LocalFileSystem; - use object_store::path::Path; - use regex::Regex; - use rstest::*; - #[tokio::test] async fn read_small_batches() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); @@ -1056,6 +475,11 @@ mod tests { async fn query_compress_data( file_compression_type: FileCompressionType, ) -> Result<()> { + use arrow_schema::{DataType, Field, Schema}; + use datafusion_common::DataFusionError; + use datafusion_datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; + use futures::TryStreamExt; + let mut cfg = SessionConfig::new(); cfg.options_mut().catalog.has_header = true; let session_state = SessionStateBuilder::new() @@ -1675,10 +1099,7 @@ mod tests { } } - fn csv_expected_batch( - schema: SchemaRef, - line_count: usize, - ) -> Result { + fn csv_expected_batch(schema: SchemaRef, line_count: usize) -> Result { let mut c1 = Vec::with_capacity(line_count); let mut c2 = Vec::with_capacity(line_count); let mut c3 = Vec::with_capacity(line_count); @@ -1721,7 +1142,7 @@ mod tests { (int_value, float_value, bool_value, char_value) } - fn csv_schema() -> Arc { + fn csv_schema() -> SchemaRef { Arc::new(Schema::new(vec![ Field::new("c1", DataType::Int32, true), Field::new("c2", DataType::Float64, true), diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 1a2aaf3af8be..49b3378e1e68 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -15,424 +15,28 @@ // specific language governing permissions and limitations // under the License. -//! [`JsonFormat`]: Line delimited JSON [`FileFormat`] abstractions - -use std::any::Any; -use std::collections::HashMap; -use std::fmt; -use std::fmt::Debug; -use std::io::BufReader; -use std::sync::Arc; - -use super::write::orchestration::spawn_writer_tasks_and_join; -use super::{ - Decoder, DecoderDeserializer, FileFormat, FileFormatFactory, - DEFAULT_SCHEMA_INFER_MAX_RECORD, -}; -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; -use crate::datasource::file_format::write::BatchSerializer; -use crate::datasource::physical_plan::{FileSink, FileSinkConfig, JsonSource}; -use crate::error::Result; -use crate::execution::SessionState; -use crate::physical_plan::insert::{DataSink, DataSinkExec}; -use crate::physical_plan::{ - DisplayAs, DisplayFormatType, SendableRecordBatchStream, Statistics, -}; - -use arrow::array::RecordBatch; -use arrow::datatypes::{Schema, SchemaRef}; -use arrow::error::ArrowError; -use arrow::json; -use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; -use datafusion_catalog::Session; -use datafusion_common::config::{ConfigField, ConfigFileType, JsonOptions}; -use datafusion_common::file_options::json_writer::JsonWriterOptions; -use datafusion_common::{not_impl_err, GetExt, DEFAULT_JSON_EXTENSION}; -use datafusion_common_runtime::SpawnedTask; -use datafusion_datasource::display::FileGroupDisplay; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_execution::TaskContext; -use datafusion_expr::dml::InsertOp; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::ExecutionPlan; - -use async_trait::async_trait; -use bytes::{Buf, Bytes}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; -use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; - -#[derive(Default)] -/// Factory struct used to create [JsonFormat] -pub struct JsonFormatFactory { - /// the options carried by format factory - pub options: Option, -} - -impl JsonFormatFactory { - /// Creates an instance of [JsonFormatFactory] - pub fn new() -> Self { - Self { options: None } - } - - /// Creates an instance of [JsonFormatFactory] with customized default options - pub fn new_with_options(options: JsonOptions) -> Self { - Self { - options: Some(options), - } - } -} - -impl FileFormatFactory for JsonFormatFactory { - fn create( - &self, - state: &dyn Session, - format_options: &HashMap, - ) -> Result> { - let state = state.as_any().downcast_ref::().unwrap(); - let json_options = match &self.options { - None => { - let mut table_options = state.default_table_options(); - table_options.set_config_format(ConfigFileType::JSON); - table_options.alter_with_string_hash_map(format_options)?; - table_options.json - } - Some(json_options) => { - let mut json_options = json_options.clone(); - for (k, v) in format_options { - json_options.set(k, v)?; - } - json_options - } - }; - - Ok(Arc::new(JsonFormat::default().with_options(json_options))) - } - - fn default(&self) -> Arc { - Arc::new(JsonFormat::default()) - } +//! Re-exports the [`datafusion_datasource_json::file_format`] module, and contains tests for it. - fn as_any(&self) -> &dyn Any { - self - } -} - -impl GetExt for JsonFormatFactory { - fn get_ext(&self) -> String { - // Removes the dot, i.e. ".parquet" -> "parquet" - DEFAULT_JSON_EXTENSION[1..].to_string() - } -} - -impl Debug for JsonFormatFactory { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("JsonFormatFactory") - .field("options", &self.options) - .finish() - } -} - -/// New line delimited JSON `FileFormat` implementation. -#[derive(Debug, Default)] -pub struct JsonFormat { - options: JsonOptions, -} - -impl JsonFormat { - /// Set JSON options - pub fn with_options(mut self, options: JsonOptions) -> Self { - self.options = options; - self - } - - /// Retrieve JSON options - pub fn options(&self) -> &JsonOptions { - &self.options - } - - /// Set a limit in terms of records to scan to infer the schema - /// - defaults to `DEFAULT_SCHEMA_INFER_MAX_RECORD` - pub fn with_schema_infer_max_rec(mut self, max_rec: usize) -> Self { - self.options.schema_infer_max_rec = Some(max_rec); - self - } - - /// Set a `FileCompressionType` of JSON - /// - defaults to `FileCompressionType::UNCOMPRESSED` - pub fn with_file_compression_type( - mut self, - file_compression_type: FileCompressionType, - ) -> Self { - self.options.compression = file_compression_type.into(); - self - } -} - -#[async_trait] -impl FileFormat for JsonFormat { - fn as_any(&self) -> &dyn Any { - self - } - - fn get_ext(&self) -> String { - JsonFormatFactory::new().get_ext() - } - - fn get_ext_with_compression( - &self, - file_compression_type: &FileCompressionType, - ) -> Result { - let ext = self.get_ext(); - Ok(format!("{}{}", ext, file_compression_type.get_ext())) - } - - async fn infer_schema( - &self, - _state: &dyn Session, - store: &Arc, - objects: &[ObjectMeta], - ) -> Result { - let mut schemas = Vec::new(); - let mut records_to_read = self - .options - .schema_infer_max_rec - .unwrap_or(DEFAULT_SCHEMA_INFER_MAX_RECORD); - let file_compression_type = FileCompressionType::from(self.options.compression); - for object in objects { - let mut take_while = || { - let should_take = records_to_read > 0; - if should_take { - records_to_read -= 1; - } - should_take - }; - - let r = store.as_ref().get(&object.location).await?; - let schema = match r.payload { - GetResultPayload::File(file, _) => { - let decoder = file_compression_type.convert_read(file)?; - let mut reader = BufReader::new(decoder); - let iter = ValueIter::new(&mut reader, None); - infer_json_schema_from_iterator(iter.take_while(|_| take_while()))? - } - GetResultPayload::Stream(_) => { - let data = r.bytes().await?; - let decoder = file_compression_type.convert_read(data.reader())?; - let mut reader = BufReader::new(decoder); - let iter = ValueIter::new(&mut reader, None); - infer_json_schema_from_iterator(iter.take_while(|_| take_while()))? - } - }; - - schemas.push(schema); - if records_to_read == 0 { - break; - } - } - - let schema = Schema::try_merge(schemas)?; - Ok(Arc::new(schema)) - } - - async fn infer_stats( - &self, - _state: &dyn Session, - _store: &Arc, - table_schema: SchemaRef, - _object: &ObjectMeta, - ) -> Result { - Ok(Statistics::new_unknown(&table_schema)) - } - - async fn create_physical_plan( - &self, - _state: &dyn Session, - mut conf: FileScanConfig, - _filters: Option<&Arc>, - ) -> Result> { - let source = Arc::new(JsonSource::new()); - conf.file_compression_type = FileCompressionType::from(self.options.compression); - Ok(conf.with_source(source).build()) - } - - async fn create_writer_physical_plan( - &self, - input: Arc, - _state: &dyn Session, - conf: FileSinkConfig, - order_requirements: Option, - ) -> Result> { - if conf.insert_op != InsertOp::Append { - return not_impl_err!("Overwrites are not implemented yet for Json"); - } - - let writer_options = JsonWriterOptions::try_from(&self.options)?; - - let sink = Arc::new(JsonSink::new(conf, writer_options)); - - Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) - } - - fn file_source(&self) -> Arc { - Arc::new(JsonSource::default()) - } -} - -impl Default for JsonSerializer { - fn default() -> Self { - Self::new() - } -} - -/// Define a struct for serializing Json records to a stream -pub struct JsonSerializer {} - -impl JsonSerializer { - /// Constructor for the JsonSerializer object - pub fn new() -> Self { - Self {} - } -} - -impl BatchSerializer for JsonSerializer { - fn serialize(&self, batch: RecordBatch, _initial: bool) -> Result { - let mut buffer = Vec::with_capacity(4096); - let mut writer = json::LineDelimitedWriter::new(&mut buffer); - writer.write(&batch)?; - Ok(Bytes::from(buffer)) - } -} - -/// Implements [`DataSink`] for writing to a Json file. -pub struct JsonSink { - /// Config options for writing data - config: FileSinkConfig, - /// Writer options for underlying Json writer - writer_options: JsonWriterOptions, -} - -impl Debug for JsonSink { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("JsonSink").finish() - } -} - -impl DisplayAs for JsonSink { - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "JsonSink(file_groups=",)?; - FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; - write!(f, ")") - } - } - } -} - -impl JsonSink { - /// Create from config. - pub fn new(config: FileSinkConfig, writer_options: JsonWriterOptions) -> Self { - Self { - config, - writer_options, - } - } - - /// Retrieve the writer options - pub fn writer_options(&self) -> &JsonWriterOptions { - &self.writer_options - } -} - -#[async_trait] -impl FileSink for JsonSink { - fn config(&self) -> &FileSinkConfig { - &self.config - } - - async fn spawn_writer_tasks_and_join( - &self, - context: &Arc, - demux_task: SpawnedTask>, - file_stream_rx: DemuxedStreamReceiver, - object_store: Arc, - ) -> Result { - let serializer = Arc::new(JsonSerializer::new()) as _; - spawn_writer_tasks_and_join( - context, - serializer, - self.writer_options.compression.into(), - object_store, - demux_task, - file_stream_rx, - ) - .await - } -} - -#[async_trait] -impl DataSink for JsonSink { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> &SchemaRef { - self.config.output_schema() - } - - async fn write_all( - &self, - data: SendableRecordBatchStream, - context: &Arc, - ) -> Result { - FileSink::write_all(self, data, context).await - } -} - -#[derive(Debug)] -pub(crate) struct JsonDecoder { - inner: json::reader::Decoder, -} - -impl JsonDecoder { - pub(crate) fn new(decoder: json::reader::Decoder) -> Self { - Self { inner: decoder } - } -} - -impl Decoder for JsonDecoder { - fn decode(&mut self, buf: &[u8]) -> Result { - self.inner.decode(buf) - } - - fn flush(&mut self) -> Result, ArrowError> { - self.inner.flush() - } - - fn can_flush_early(&self) -> bool { - false - } -} - -impl From for DecoderDeserializer { - fn from(decoder: json::reader::Decoder) -> Self { - DecoderDeserializer::new(JsonDecoder::new(decoder)) - } -} +pub use datafusion_datasource_json::file_format::*; #[cfg(test)] mod tests { - use super::super::test_util::scan_format; + use std::sync::Arc; + use super::*; - use crate::datasource::file_format::{ + + use crate::datasource::file_format::test_util::scan_format; + use crate::prelude::{NdJsonReadOptions, SessionConfig, SessionContext}; + use crate::test::object_store::local_unpartitioned_file; + use arrow::array::RecordBatch; + use arrow_schema::Schema; + use bytes::Bytes; + use datafusion_catalog::Session; + use datafusion_datasource::decoder::{ BatchDeserializer, DecoderDeserializer, DeserializerOutput, }; - use crate::execution::options::NdJsonReadOptions; - use crate::physical_plan::collect; - use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::object_store::local_unpartitioned_file; + use datafusion_datasource::file_format::FileFormat; + use datafusion_physical_plan::{collect, ExecutionPlan}; use arrow::compute::concat_batches; use arrow::datatypes::{DataType, Field}; @@ -442,6 +46,7 @@ mod tests { use datafusion_common::stats::Precision; use datafusion_common::{assert_batches_eq, internal_err}; + use datafusion_common::Result; use futures::StreamExt; use object_store::local::LocalFileSystem; use regex::Regex; diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 5dbf4957a4b5..df74e5d060e6 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -19,211 +19,33 @@ //! See write.rs for write related helper methods pub mod arrow; -pub mod avro; pub mod csv; pub mod json; -pub mod options; + +#[cfg(feature = "avro")] +pub mod avro; + #[cfg(feature = "parquet")] pub mod parquet; -use ::arrow::array::RecordBatch; -use arrow_schema::ArrowError; -use bytes::Buf; -use bytes::Bytes; -use datafusion_common::Result; +pub mod options; + pub use datafusion_datasource::file_compression_type; pub use datafusion_datasource::file_format::*; pub use datafusion_datasource::write; -use futures::stream::BoxStream; -use futures::StreamExt as _; -use futures::{ready, Stream}; -use std::collections::VecDeque; -use std::fmt; -use std::task::Poll; - -/// Possible outputs of a [`BatchDeserializer`]. -#[derive(Debug, PartialEq)] -pub enum DeserializerOutput { - /// A successfully deserialized [`RecordBatch`]. - RecordBatch(RecordBatch), - /// The deserializer requires more data to make progress. - RequiresMoreData, - /// The input data has been exhausted. - InputExhausted, -} - -/// Trait defining a scheme for deserializing byte streams into structured data. -/// Implementors of this trait are responsible for converting raw bytes into -/// `RecordBatch` objects. -pub trait BatchDeserializer: Send + fmt::Debug { - /// Feeds a message for deserialization, updating the internal state of - /// this `BatchDeserializer`. Note that one can call this function multiple - /// times before calling `next`, which will queue multiple messages for - /// deserialization. Returns the number of bytes consumed. - fn digest(&mut self, message: T) -> usize; - - /// Attempts to deserialize any pending messages and returns a - /// `DeserializerOutput` to indicate progress. - fn next(&mut self) -> Result; - - /// Informs the deserializer that no more messages will be provided for - /// deserialization. - fn finish(&mut self); -} - -/// A general interface for decoders such as [`arrow::json::reader::Decoder`] and -/// [`arrow::csv::reader::Decoder`]. Defines an interface similar to -/// [`Decoder::decode`] and [`Decoder::flush`] methods, but also includes -/// a method to check if the decoder can flush early. Intended to be used in -/// conjunction with [`DecoderDeserializer`]. -/// -/// [`arrow::json::reader::Decoder`]: ::arrow::json::reader::Decoder -/// [`arrow::csv::reader::Decoder`]: ::arrow::csv::reader::Decoder -/// [`Decoder::decode`]: ::arrow::json::reader::Decoder::decode -/// [`Decoder::flush`]: ::arrow::json::reader::Decoder::flush -pub(crate) trait Decoder: Send + fmt::Debug { - /// See [`arrow::json::reader::Decoder::decode`]. - /// - /// [`arrow::json::reader::Decoder::decode`]: ::arrow::json::reader::Decoder::decode - fn decode(&mut self, buf: &[u8]) -> Result; - - /// See [`arrow::json::reader::Decoder::flush`]. - /// - /// [`arrow::json::reader::Decoder::flush`]: ::arrow::json::reader::Decoder::flush - fn flush(&mut self) -> Result, ArrowError>; - - /// Whether the decoder can flush early in its current state. - fn can_flush_early(&self) -> bool; -} - -impl fmt::Debug for DecoderDeserializer { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Deserializer") - .field("buffered_queue", &self.buffered_queue) - .field("finalized", &self.finalized) - .finish() - } -} - -impl BatchDeserializer for DecoderDeserializer { - fn digest(&mut self, message: Bytes) -> usize { - if message.is_empty() { - return 0; - } - - let consumed = message.len(); - self.buffered_queue.push_back(message); - consumed - } - - fn next(&mut self) -> Result { - while let Some(buffered) = self.buffered_queue.front_mut() { - let decoded = self.decoder.decode(buffered)?; - buffered.advance(decoded); - - if buffered.is_empty() { - self.buffered_queue.pop_front(); - } - - // Flush when the stream ends or batch size is reached - // Certain implementations can flush early - if decoded == 0 || self.decoder.can_flush_early() { - return match self.decoder.flush() { - Ok(Some(batch)) => Ok(DeserializerOutput::RecordBatch(batch)), - Ok(None) => continue, - Err(e) => Err(e), - }; - } - } - if self.finalized { - Ok(DeserializerOutput::InputExhausted) - } else { - Ok(DeserializerOutput::RequiresMoreData) - } - } - - fn finish(&mut self) { - self.finalized = true; - // Ensure the decoder is flushed: - self.buffered_queue.push_back(Bytes::new()); - } -} - -/// A generic, decoder-based deserialization scheme for processing encoded data. -/// -/// This struct is responsible for converting a stream of bytes, which represent -/// encoded data, into a stream of `RecordBatch` objects, following the specified -/// schema and formatting options. It also handles any buffering necessary to satisfy -/// the `Decoder` interface. -pub(crate) struct DecoderDeserializer { - /// The underlying decoder used for deserialization - pub(crate) decoder: T, - /// The buffer used to store the remaining bytes to be decoded - pub(crate) buffered_queue: VecDeque, - /// Whether the input stream has been fully consumed - pub(crate) finalized: bool, -} - -impl DecoderDeserializer { - /// Creates a new `DecoderDeserializer` with the provided decoder. - pub fn new(decoder: T) -> Self { - DecoderDeserializer { - decoder, - buffered_queue: VecDeque::new(), - finalized: false, - } - } -} - -/// Deserializes a stream of bytes into a stream of [`RecordBatch`] objects using the -/// provided deserializer. -/// -/// Returns a boxed stream of `Result`. The stream yields [`RecordBatch`] -/// objects as they are produced by the deserializer, or an [`ArrowError`] if an error -/// occurs while polling the input or deserializing. -pub(crate) fn deserialize_stream<'a>( - mut input: impl Stream> + Unpin + Send + 'a, - mut deserializer: impl BatchDeserializer + 'a, -) -> BoxStream<'a, Result> { - futures::stream::poll_fn(move |cx| loop { - match ready!(input.poll_next_unpin(cx)).transpose()? { - Some(b) => _ = deserializer.digest(b), - None => deserializer.finish(), - }; - - return match deserializer.next()? { - DeserializerOutput::RecordBatch(rb) => Poll::Ready(Some(Ok(rb))), - DeserializerOutput::InputExhausted => Poll::Ready(None), - DeserializerOutput::RequiresMoreData => continue, - }; - }) - .boxed() -} #[cfg(test)] pub(crate) mod test_util { - use std::fmt::{self, Display}; - use std::ops::Range; - use std::sync::{Arc, Mutex}; + use std::sync::Arc; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::test::object_store::local_unpartitioned_file; - use async_trait::async_trait; - use bytes::Bytes; use datafusion_catalog::Session; use datafusion_common::Result; - use datafusion_datasource::file_format::FileFormat; - use datafusion_datasource::file_scan_config::FileScanConfig; - use datafusion_physical_plan::ExecutionPlan; - use futures::stream::BoxStream; - use futures::StreamExt; - use object_store::local::LocalFileSystem; - use object_store::path::Path; - use object_store::{ - Attributes, GetOptions, GetResult, GetResultPayload, ListResult, MultipartUpload, - ObjectMeta, ObjectStore, PutMultipartOpts, PutOptions, PutPayload, PutResult, + use datafusion_datasource::{ + file_format::FileFormat, file_scan_config::FileScanConfig, PartitionedFile, }; + use datafusion_execution::object_store::ObjectStoreUrl; + + use crate::test::object_store::local_unpartitioned_file; pub async fn scan_format( state: &dyn Session, @@ -232,8 +54,8 @@ pub(crate) mod test_util { file_name: &str, projection: Option>, limit: Option, - ) -> Result> { - let store = Arc::new(LocalFileSystem::new()) as _; + ) -> Result> { + let store = Arc::new(object_store::local::LocalFileSystem::new()) as _; let meta = local_unpartitioned_file(format!("{store_root}/{file_name}")); let file_schema = format @@ -270,129 +92,41 @@ pub(crate) mod test_util { .await?; Ok(exec) } +} - /// Mock ObjectStore to provide an variable stream of bytes on get - /// Able to keep track of how many iterations of the provided bytes were repeated - #[derive(Debug)] - pub struct VariableStream { - bytes_to_repeat: Bytes, - max_iterations: usize, - iterations_detected: Arc>, - } - - impl Display for VariableStream { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "VariableStream") - } - } - - #[async_trait] - impl ObjectStore for VariableStream { - async fn put_opts( - &self, - _location: &Path, - _payload: PutPayload, - _opts: PutOptions, - ) -> object_store::Result { - unimplemented!() - } - - async fn put_multipart_opts( - &self, - _location: &Path, - _opts: PutMultipartOpts, - ) -> object_store::Result> { - unimplemented!() - } - - async fn get(&self, location: &Path) -> object_store::Result { - let bytes = self.bytes_to_repeat.clone(); - let range = 0..bytes.len() * self.max_iterations; - let arc = self.iterations_detected.clone(); - let stream = futures::stream::repeat_with(move || { - let arc_inner = arc.clone(); - *arc_inner.lock().unwrap() += 1; - Ok(bytes.clone()) - }) - .take(self.max_iterations) - .boxed(); - - Ok(GetResult { - payload: GetResultPayload::Stream(stream), - meta: ObjectMeta { - location: location.clone(), - last_modified: Default::default(), - size: range.end, - e_tag: None, - version: None, - }, - range: Default::default(), - attributes: Attributes::default(), - }) - } - - async fn get_opts( - &self, - _location: &Path, - _opts: GetOptions, - ) -> object_store::Result { - unimplemented!() - } - - async fn get_ranges( - &self, - _location: &Path, - _ranges: &[Range], - ) -> object_store::Result> { - unimplemented!() - } - - async fn head(&self, _location: &Path) -> object_store::Result { - unimplemented!() - } - - async fn delete(&self, _location: &Path) -> object_store::Result<()> { - unimplemented!() - } - - fn list( - &self, - _prefix: Option<&Path>, - ) -> BoxStream<'_, object_store::Result> { - unimplemented!() - } - - async fn list_with_delimiter( - &self, - _prefix: Option<&Path>, - ) -> object_store::Result { - unimplemented!() - } - - async fn copy(&self, _from: &Path, _to: &Path) -> object_store::Result<()> { - unimplemented!() - } - - async fn copy_if_not_exists( - &self, - _from: &Path, - _to: &Path, - ) -> object_store::Result<()> { - unimplemented!() - } - } - - impl VariableStream { - pub fn new(bytes_to_repeat: Bytes, max_iterations: usize) -> Self { - Self { - bytes_to_repeat, - max_iterations, - iterations_detected: Arc::new(Mutex::new(0)), - } - } +#[cfg(test)] +mod tests { + #[cfg(feature = "parquet")] + #[tokio::test] + async fn write_parquet_results_error_handling() -> datafusion_common::Result<()> { + use std::sync::Arc; + + use object_store::local::LocalFileSystem; + use tempfile::TempDir; + use url::Url; + + use crate::{ + dataframe::DataFrameWriteOptions, + prelude::{CsvReadOptions, SessionContext}, + }; - pub fn get_iterations_detected(&self) -> usize { - *self.iterations_detected.lock().unwrap() - } + let ctx = SessionContext::new(); + // register a local file system object store for /tmp directory + let tmp_dir = TempDir::new()?; + let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); + let local_url = Url::parse("file://local").unwrap(); + ctx.register_object_store(&local_url, local); + + let options = CsvReadOptions::default() + .schema_infer_max_records(2) + .has_header(true); + let df = ctx.read_csv("tests/data/corrupt.csv", options).await?; + let out_dir_url = "file://local/out"; + let e = df + .write_parquet(out_dir_url, DataFrameWriteOptions::new(), None) + .await + .expect_err("should fail because input file does not match inferred schema"); + assert_eq!(e.strip_backtrace(), "Arrow error: Parser error: Error while parsing value d for column 0 at line 4"); + Ok(()) } } diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 2f32479ed2b0..08e9a628dd61 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -19,16 +19,17 @@ use std::sync::Arc; -use crate::datasource::file_format::arrow::ArrowFormat; -use crate::datasource::file_format::file_compression_type::FileCompressionType; +#[cfg(feature = "avro")] +use crate::datasource::file_format::avro::AvroFormat; + #[cfg(feature = "parquet")] use crate::datasource::file_format::parquet::ParquetFormat; + +use crate::datasource::file_format::arrow::ArrowFormat; +use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; use crate::datasource::listing::ListingTableUrl; -use crate::datasource::{ - file_format::{avro::AvroFormat, csv::CsvFormat, json::JsonFormat}, - listing::ListingOptions, -}; +use crate::datasource::{file_format::csv::CsvFormat, listing::ListingOptions}; use crate::error::Result; use crate::execution::context::{SessionConfig, SessionState}; @@ -40,6 +41,7 @@ use datafusion_common::{ }; use async_trait::async_trait; +use datafusion_datasource_json::file_format::JsonFormat; use datafusion_expr::SortExpr; /// Options that control the reading of CSV files. @@ -629,6 +631,7 @@ impl ReadOptions<'_> for NdJsonReadOptions<'_> { } } +#[cfg(feature = "avro")] #[async_trait] impl ReadOptions<'_> for AvroReadOptions<'_> { fn to_listing_options( diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 4a24871aeef7..e2c7d1ecafa3 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -15,1360 +15,17 @@ // specific language governing permissions and limitations // under the License. -//! [`ParquetFormat`]: Parquet [`FileFormat`] abstractions - -use std::any::Any; -use std::fmt; -use std::fmt::Debug; -use std::ops::Range; -use std::sync::Arc; - -use super::write::demux::DemuxedStreamReceiver; -use super::write::{create_writer, SharedBuffer}; -use super::{FileFormat, FileFormatFactory, FilePushdownSupport}; -use crate::arrow::array::RecordBatch; -use crate::arrow::datatypes::{Fields, Schema, SchemaRef}; -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::write::get_writer_schema; -use crate::datasource::physical_plan::parquet::can_expr_be_pushed_down_with_schemas; -use crate::datasource::physical_plan::parquet::source::ParquetSource; -use crate::datasource::physical_plan::{FileSink, FileSinkConfig}; -use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; -use crate::error::Result; -use crate::execution::SessionState; -use crate::physical_plan::insert::{DataSink, DataSinkExec}; -use crate::physical_plan::{ - Accumulator, DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, - Statistics, -}; - -use arrow::compute::sum; -use arrow_schema::{DataType, Field, FieldRef}; -use datafusion_catalog::Session; -use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; -use datafusion_common::parsers::CompressionTypeVariant; -use datafusion_common::stats::Precision; -use datafusion_common::HashMap; -use datafusion_common::{ - internal_datafusion_err, internal_err, not_impl_err, DataFusionError, GetExt, - DEFAULT_PARQUET_EXTENSION, -}; -use datafusion_common_runtime::SpawnedTask; -use datafusion_datasource::display::FileGroupDisplay; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; -use datafusion_execution::TaskContext; -use datafusion_expr::dml::InsertOp; -use datafusion_expr::Expr; -use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; - -use async_trait::async_trait; -use bytes::Bytes; -use futures::future::BoxFuture; -use futures::{FutureExt, StreamExt, TryStreamExt}; -use log::debug; -use object_store::buffered::BufWriter; -use object_store::path::Path; -use object_store::{ObjectMeta, ObjectStore}; -use parquet::arrow::arrow_reader::statistics::StatisticsConverter; -use parquet::arrow::arrow_writer::{ - compute_leaves, get_column_writers, ArrowColumnChunk, ArrowColumnWriter, - ArrowLeafColumn, ArrowWriterOptions, -}; -use parquet::arrow::async_reader::MetadataFetch; -use parquet::arrow::{parquet_to_arrow_schema, ArrowSchemaConverter, AsyncArrowWriter}; -use parquet::errors::ParquetError; -use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; -use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder}; -use parquet::file::writer::SerializedFileWriter; -use parquet::format::FileMetaData; -use tokio::io::{AsyncWrite, AsyncWriteExt}; -use tokio::sync::mpsc::{self, Receiver, Sender}; -use tokio::task::JoinSet; - -/// Initial writing buffer size. Note this is just a size hint for efficiency. It -/// will grow beyond the set value if needed. -const INITIAL_BUFFER_BYTES: usize = 1048576; - -/// When writing parquet files in parallel, if the buffered Parquet data exceeds -/// this size, it is flushed to object store -const BUFFER_FLUSH_BYTES: usize = 1024000; - -#[derive(Default)] -/// Factory struct used to create [ParquetFormat] -pub struct ParquetFormatFactory { - /// inner options for parquet - pub options: Option, -} - -impl ParquetFormatFactory { - /// Creates an instance of [ParquetFormatFactory] - pub fn new() -> Self { - Self { options: None } - } - - /// Creates an instance of [ParquetFormatFactory] with customized default options - pub fn new_with_options(options: TableParquetOptions) -> Self { - Self { - options: Some(options), - } - } -} - -impl FileFormatFactory for ParquetFormatFactory { - fn create( - &self, - state: &dyn Session, - format_options: &std::collections::HashMap, - ) -> Result> { - let state = state.as_any().downcast_ref::().unwrap(); - let parquet_options = match &self.options { - None => { - let mut table_options = state.default_table_options(); - table_options.set_config_format(ConfigFileType::PARQUET); - table_options.alter_with_string_hash_map(format_options)?; - table_options.parquet - } - Some(parquet_options) => { - let mut parquet_options = parquet_options.clone(); - for (k, v) in format_options { - parquet_options.set(k, v)?; - } - parquet_options - } - }; - - Ok(Arc::new( - ParquetFormat::default().with_options(parquet_options), - )) - } - - fn default(&self) -> Arc { - Arc::new(ParquetFormat::default()) - } - - fn as_any(&self) -> &dyn Any { - self - } -} - -impl GetExt for ParquetFormatFactory { - fn get_ext(&self) -> String { - // Removes the dot, i.e. ".parquet" -> "parquet" - DEFAULT_PARQUET_EXTENSION[1..].to_string() - } -} - -impl Debug for ParquetFormatFactory { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("ParquetFormatFactory") - .field("ParquetFormatFactory", &self.options) - .finish() - } -} -/// The Apache Parquet `FileFormat` implementation -#[derive(Debug, Default)] -pub struct ParquetFormat { - options: TableParquetOptions, -} - -impl ParquetFormat { - /// Construct a new Format with no local overrides - pub fn new() -> Self { - Self::default() - } - - /// Activate statistics based row group level pruning - /// - If `None`, defaults to value on `config_options` - pub fn with_enable_pruning(mut self, enable: bool) -> Self { - self.options.global.pruning = enable; - self - } - - /// Return `true` if pruning is enabled - pub fn enable_pruning(&self) -> bool { - self.options.global.pruning - } - - /// Provide a hint to the size of the file metadata. If a hint is provided - /// the reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. - /// Without a hint, two read are required. One read to fetch the 8-byte parquet footer and then - /// another read to fetch the metadata length encoded in the footer. - /// - /// - If `None`, defaults to value on `config_options` - pub fn with_metadata_size_hint(mut self, size_hint: Option) -> Self { - self.options.global.metadata_size_hint = size_hint; - self - } - - /// Return the metadata size hint if set - pub fn metadata_size_hint(&self) -> Option { - self.options.global.metadata_size_hint - } - - /// Tell the parquet reader to skip any metadata that may be in - /// the file Schema. This can help avoid schema conflicts due to - /// metadata. - /// - /// - If `None`, defaults to value on `config_options` - pub fn with_skip_metadata(mut self, skip_metadata: bool) -> Self { - self.options.global.skip_metadata = skip_metadata; - self - } - - /// Returns `true` if schema metadata will be cleared prior to - /// schema merging. - pub fn skip_metadata(&self) -> bool { - self.options.global.skip_metadata - } - - /// Set Parquet options for the ParquetFormat - pub fn with_options(mut self, options: TableParquetOptions) -> Self { - self.options = options; - self - } - - /// Parquet options - pub fn options(&self) -> &TableParquetOptions { - &self.options - } - - /// Return `true` if should use view types. - /// - /// If this returns true, DataFusion will instruct the parquet reader - /// to read string / binary columns using view `StringView` or `BinaryView` - /// if the table schema specifies those types, regardless of any embedded metadata - /// that may specify an alternate Arrow type. The parquet reader is optimized - /// for reading `StringView` and `BinaryView` and such queries are significantly faster. - /// - /// If this returns false, the parquet reader will read the columns according to the - /// defaults or any embedded Arrow type information. This may result in reading - /// `StringArrays` and then casting to `StringViewArray` which is less efficient. - pub fn force_view_types(&self) -> bool { - self.options.global.schema_force_view_types - } - - /// If true, will use view types. See [`Self::force_view_types`] for details - pub fn with_force_view_types(mut self, use_views: bool) -> Self { - self.options.global.schema_force_view_types = use_views; - self - } - - /// Return `true` if binary types will be read as strings. - /// - /// If this returns true, DataFusion will instruct the parquet reader - /// to read binary columns such as `Binary` or `BinaryView` as the - /// corresponding string type such as `Utf8` or `LargeUtf8`. - /// The parquet reader has special optimizations for `Utf8` and `LargeUtf8` - /// validation, and such queries are significantly faster than reading - /// binary columns and then casting to string columns. - pub fn binary_as_string(&self) -> bool { - self.options.global.binary_as_string - } - - /// If true, will read binary types as strings. See [`Self::binary_as_string`] for details - pub fn with_binary_as_string(mut self, binary_as_string: bool) -> Self { - self.options.global.binary_as_string = binary_as_string; - self - } -} - -/// Clears all metadata (Schema level and field level) on an iterator -/// of Schemas -fn clear_metadata( - schemas: impl IntoIterator, -) -> impl Iterator { - schemas.into_iter().map(|schema| { - let fields = schema - .fields() - .iter() - .map(|field| { - field.as_ref().clone().with_metadata(Default::default()) // clear meta - }) - .collect::(); - Schema::new(fields) - }) -} - -async fn fetch_schema_with_location( - store: &dyn ObjectStore, - file: &ObjectMeta, - metadata_size_hint: Option, -) -> Result<(Path, Schema)> { - let loc_path = file.location.clone(); - let schema = fetch_schema(store, file, metadata_size_hint).await?; - Ok((loc_path, schema)) -} - -#[async_trait] -impl FileFormat for ParquetFormat { - fn as_any(&self) -> &dyn Any { - self - } - - fn get_ext(&self) -> String { - ParquetFormatFactory::new().get_ext() - } - - fn get_ext_with_compression( - &self, - file_compression_type: &FileCompressionType, - ) -> Result { - let ext = self.get_ext(); - match file_compression_type.get_variant() { - CompressionTypeVariant::UNCOMPRESSED => Ok(ext), - _ => internal_err!("Parquet FileFormat does not support compression."), - } - } - - async fn infer_schema( - &self, - state: &dyn Session, - store: &Arc, - objects: &[ObjectMeta], - ) -> Result { - let mut schemas: Vec<_> = futures::stream::iter(objects) - .map(|object| { - fetch_schema_with_location( - store.as_ref(), - object, - self.metadata_size_hint(), - ) - }) - .boxed() // Workaround https://github.com/rust-lang/rust/issues/64552 - .buffered(state.config_options().execution.meta_fetch_concurrency) - .try_collect() - .await?; - - // Schema inference adds fields based the order they are seen - // which depends on the order the files are processed. For some - // object stores (like local file systems) the order returned from list - // is not deterministic. Thus, to ensure deterministic schema inference - // sort the files first. - // https://github.com/apache/datafusion/pull/6629 - schemas.sort_by(|(location1, _), (location2, _)| location1.cmp(location2)); - - let schemas = schemas - .into_iter() - .map(|(_, schema)| schema) - .collect::>(); - - let schema = if self.skip_metadata() { - Schema::try_merge(clear_metadata(schemas)) - } else { - Schema::try_merge(schemas) - }?; - - let schema = if self.binary_as_string() { - transform_binary_to_string(&schema) - } else { - schema - }; - - let schema = if self.force_view_types() { - transform_schema_to_view(&schema) - } else { - schema - }; - - Ok(Arc::new(schema)) - } - - async fn infer_stats( - &self, - _state: &dyn Session, - store: &Arc, - table_schema: SchemaRef, - object: &ObjectMeta, - ) -> Result { - let stats = fetch_statistics( - store.as_ref(), - table_schema, - object, - self.metadata_size_hint(), - ) - .await?; - Ok(stats) - } - - async fn create_physical_plan( - &self, - _state: &dyn Session, - conf: FileScanConfig, - filters: Option<&Arc>, - ) -> Result> { - let mut predicate = None; - let mut metadata_size_hint = None; - - // If enable pruning then combine the filters to build the predicate. - // If disable pruning then set the predicate to None, thus readers - // will not prune data based on the statistics. - if self.enable_pruning() { - if let Some(pred) = filters.cloned() { - predicate = Some(pred); - } - } - if let Some(metadata) = self.metadata_size_hint() { - metadata_size_hint = Some(metadata); - } - - let mut source = ParquetSource::new(self.options.clone()); - - if let Some(predicate) = predicate { - source = source.with_predicate(Arc::clone(&conf.file_schema), predicate); - } - if let Some(metadata_size_hint) = metadata_size_hint { - source = source.with_metadata_size_hint(metadata_size_hint) - } - Ok(conf.with_source(Arc::new(source)).build()) - } - - async fn create_writer_physical_plan( - &self, - input: Arc, - _state: &dyn Session, - conf: FileSinkConfig, - order_requirements: Option, - ) -> Result> { - if conf.insert_op != InsertOp::Append { - return not_impl_err!("Overwrites are not implemented yet for Parquet"); - } - - let sink = Arc::new(ParquetSink::new(conf, self.options.clone())); - - Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) - } - - fn supports_filters_pushdown( - &self, - file_schema: &Schema, - table_schema: &Schema, - filters: &[&Expr], - ) -> Result { - if !self.options().global.pushdown_filters { - return Ok(FilePushdownSupport::NoSupport); - } - - let all_supported = filters.iter().all(|filter| { - can_expr_be_pushed_down_with_schemas(filter, file_schema, table_schema) - }); - - Ok(if all_supported { - FilePushdownSupport::Supported - } else { - FilePushdownSupport::NotSupportedForFilter - }) - } - - fn file_source(&self) -> Arc { - Arc::new(ParquetSource::default()) - } -} - -/// Coerces the file schema if the table schema uses a view type. -#[cfg(not(target_arch = "wasm32"))] -pub fn coerce_file_schema_to_view_type( - table_schema: &Schema, - file_schema: &Schema, -) -> Option { - let mut transform = false; - let table_fields: HashMap<_, _> = table_schema - .fields - .iter() - .map(|f| { - let dt = f.data_type(); - if dt.equals_datatype(&DataType::Utf8View) - || dt.equals_datatype(&DataType::BinaryView) - { - transform = true; - } - (f.name(), dt) - }) - .collect(); - - if !transform { - return None; - } - - let transformed_fields: Vec> = file_schema - .fields - .iter() - .map( - |field| match (table_fields.get(field.name()), field.data_type()) { - (Some(DataType::Utf8View), DataType::Utf8 | DataType::LargeUtf8) => { - field_with_new_type(field, DataType::Utf8View) - } - ( - Some(DataType::BinaryView), - DataType::Binary | DataType::LargeBinary, - ) => field_with_new_type(field, DataType::BinaryView), - _ => Arc::clone(field), - }, - ) - .collect(); - - Some(Schema::new_with_metadata( - transformed_fields, - file_schema.metadata.clone(), - )) -} - -/// If the table schema uses a string type, coerce the file schema to use a string type. -/// -/// See [ParquetFormat::binary_as_string] for details -#[cfg(not(target_arch = "wasm32"))] -pub fn coerce_file_schema_to_string_type( - table_schema: &Schema, - file_schema: &Schema, -) -> Option { - let mut transform = false; - let table_fields: HashMap<_, _> = table_schema - .fields - .iter() - .map(|f| (f.name(), f.data_type())) - .collect(); - let transformed_fields: Vec> = file_schema - .fields - .iter() - .map( - |field| match (table_fields.get(field.name()), field.data_type()) { - // table schema uses string type, coerce the file schema to use string type - ( - Some(DataType::Utf8), - DataType::Binary | DataType::LargeBinary | DataType::BinaryView, - ) => { - transform = true; - field_with_new_type(field, DataType::Utf8) - } - // table schema uses large string type, coerce the file schema to use large string type - ( - Some(DataType::LargeUtf8), - DataType::Binary | DataType::LargeBinary | DataType::BinaryView, - ) => { - transform = true; - field_with_new_type(field, DataType::LargeUtf8) - } - // table schema uses string view type, coerce the file schema to use view type - ( - Some(DataType::Utf8View), - DataType::Binary | DataType::LargeBinary | DataType::BinaryView, - ) => { - transform = true; - field_with_new_type(field, DataType::Utf8View) - } - _ => Arc::clone(field), - }, - ) - .collect(); - - if !transform { - None - } else { - Some(Schema::new_with_metadata( - transformed_fields, - file_schema.metadata.clone(), - )) - } -} - -/// Create a new field with the specified data type, copying the other -/// properties from the input field -fn field_with_new_type(field: &FieldRef, new_type: DataType) -> FieldRef { - Arc::new(field.as_ref().clone().with_data_type(new_type)) -} - -/// Transform a schema to use view types for Utf8 and Binary -/// -/// See [ParquetFormat::force_view_types] for details -pub fn transform_schema_to_view(schema: &Schema) -> Schema { - let transformed_fields: Vec> = schema - .fields - .iter() - .map(|field| match field.data_type() { - DataType::Utf8 | DataType::LargeUtf8 => { - field_with_new_type(field, DataType::Utf8View) - } - DataType::Binary | DataType::LargeBinary => { - field_with_new_type(field, DataType::BinaryView) - } - _ => Arc::clone(field), - }) - .collect(); - Schema::new_with_metadata(transformed_fields, schema.metadata.clone()) -} - -/// Transform a schema so that any binary types are strings -pub fn transform_binary_to_string(schema: &Schema) -> Schema { - let transformed_fields: Vec> = schema - .fields - .iter() - .map(|field| match field.data_type() { - DataType::Binary => field_with_new_type(field, DataType::Utf8), - DataType::LargeBinary => field_with_new_type(field, DataType::LargeUtf8), - DataType::BinaryView => field_with_new_type(field, DataType::Utf8View), - _ => Arc::clone(field), - }) - .collect(); - Schema::new_with_metadata(transformed_fields, schema.metadata.clone()) -} - -/// [`MetadataFetch`] adapter for reading bytes from an [`ObjectStore`] -struct ObjectStoreFetch<'a> { - store: &'a dyn ObjectStore, - meta: &'a ObjectMeta, -} - -impl<'a> ObjectStoreFetch<'a> { - fn new(store: &'a dyn ObjectStore, meta: &'a ObjectMeta) -> Self { - Self { store, meta } - } -} - -impl MetadataFetch for ObjectStoreFetch<'_> { - fn fetch( - &mut self, - range: Range, - ) -> BoxFuture<'_, Result> { - async { - self.store - .get_range(&self.meta.location, range) - .await - .map_err(ParquetError::from) - } - .boxed() - } -} - -/// Fetches parquet metadata from ObjectStore for given object -/// -/// This component is a subject to **change** in near future and is exposed for low level integrations -/// through [`ParquetFileReaderFactory`]. -/// -/// [`ParquetFileReaderFactory`]: crate::datasource::physical_plan::ParquetFileReaderFactory -pub async fn fetch_parquet_metadata( - store: &dyn ObjectStore, - meta: &ObjectMeta, - size_hint: Option, -) -> Result { - let file_size = meta.size; - let fetch = ObjectStoreFetch::new(store, meta); - - ParquetMetaDataReader::new() - .with_prefetch_hint(size_hint) - .load_and_finish(fetch, file_size) - .await - .map_err(DataFusionError::from) -} - -/// Read and parse the schema of the Parquet file at location `path` -async fn fetch_schema( - store: &dyn ObjectStore, - file: &ObjectMeta, - metadata_size_hint: Option, -) -> Result { - let metadata = fetch_parquet_metadata(store, file, metadata_size_hint).await?; - let file_metadata = metadata.file_metadata(); - let schema = parquet_to_arrow_schema( - file_metadata.schema_descr(), - file_metadata.key_value_metadata(), - )?; - Ok(schema) -} - -/// Read and parse the statistics of the Parquet file at location `path` -/// -/// See [`statistics_from_parquet_meta_calc`] for more details -async fn fetch_statistics( - store: &dyn ObjectStore, - table_schema: SchemaRef, - file: &ObjectMeta, - metadata_size_hint: Option, -) -> Result { - let metadata = fetch_parquet_metadata(store, file, metadata_size_hint).await?; - statistics_from_parquet_meta_calc(&metadata, table_schema) -} - -/// Convert statistics in [`ParquetMetaData`] into [`Statistics`] using ['StatisticsConverter`] -/// -/// The statistics are calculated for each column in the table schema -/// using the row group statistics in the parquet metadata. -pub fn statistics_from_parquet_meta_calc( - metadata: &ParquetMetaData, - table_schema: SchemaRef, -) -> Result { - let row_groups_metadata = metadata.row_groups(); - - let mut statistics = Statistics::new_unknown(&table_schema); - let mut has_statistics = false; - let mut num_rows = 0_usize; - let mut total_byte_size = 0_usize; - for row_group_meta in row_groups_metadata { - num_rows += row_group_meta.num_rows() as usize; - total_byte_size += row_group_meta.total_byte_size() as usize; - - if !has_statistics { - row_group_meta.columns().iter().for_each(|column| { - has_statistics = column.statistics().is_some(); - }); - } - } - statistics.num_rows = Precision::Exact(num_rows); - statistics.total_byte_size = Precision::Exact(total_byte_size); - - let file_metadata = metadata.file_metadata(); - let mut file_schema = parquet_to_arrow_schema( - file_metadata.schema_descr(), - file_metadata.key_value_metadata(), - )?; - if let Some(merged) = coerce_file_schema_to_string_type(&table_schema, &file_schema) { - file_schema = merged; - } - - if let Some(merged) = coerce_file_schema_to_view_type(&table_schema, &file_schema) { - file_schema = merged; - } - - statistics.column_statistics = if has_statistics { - let (mut max_accs, mut min_accs) = create_max_min_accs(&table_schema); - let mut null_counts_array = - vec![Precision::Exact(0); table_schema.fields().len()]; - - table_schema - .fields() - .iter() - .enumerate() - .for_each(|(idx, field)| { - match StatisticsConverter::try_new( - field.name(), - &file_schema, - file_metadata.schema_descr(), - ) { - Ok(stats_converter) => { - summarize_min_max_null_counts( - &mut min_accs, - &mut max_accs, - &mut null_counts_array, - idx, - num_rows, - &stats_converter, - row_groups_metadata, - ) - .ok(); - } - Err(e) => { - debug!("Failed to create statistics converter: {}", e); - null_counts_array[idx] = Precision::Exact(num_rows); - } - } - }); - - get_col_stats( - &table_schema, - null_counts_array, - &mut max_accs, - &mut min_accs, - ) - } else { - Statistics::unknown_column(&table_schema) - }; - - Ok(statistics) -} - -/// Deprecated -/// Use [`statistics_from_parquet_meta_calc`] instead. -/// This method was deprecated because it didn't need to be async so a new method was created -/// that exposes a synchronous API. -#[deprecated( - since = "40.0.0", - note = "please use `statistics_from_parquet_meta_calc` instead" -)] -pub async fn statistics_from_parquet_meta( - metadata: &ParquetMetaData, - table_schema: SchemaRef, -) -> Result { - statistics_from_parquet_meta_calc(metadata, table_schema) -} - -fn summarize_min_max_null_counts( - min_accs: &mut [Option], - max_accs: &mut [Option], - null_counts_array: &mut [Precision], - arrow_schema_index: usize, - num_rows: usize, - stats_converter: &StatisticsConverter, - row_groups_metadata: &[RowGroupMetaData], -) -> Result<()> { - let max_values = stats_converter.row_group_maxes(row_groups_metadata)?; - let min_values = stats_converter.row_group_mins(row_groups_metadata)?; - let null_counts = stats_converter.row_group_null_counts(row_groups_metadata)?; - - if let Some(max_acc) = &mut max_accs[arrow_schema_index] { - max_acc.update_batch(&[max_values])?; - } - - if let Some(min_acc) = &mut min_accs[arrow_schema_index] { - min_acc.update_batch(&[min_values])?; - } - - null_counts_array[arrow_schema_index] = Precision::Exact(match sum(&null_counts) { - Some(null_count) => null_count as usize, - None => num_rows, - }); - - Ok(()) -} - -/// Implements [`DataSink`] for writing to a parquet file. -pub struct ParquetSink { - /// Config options for writing data - config: FileSinkConfig, - /// Underlying parquet options - parquet_options: TableParquetOptions, - /// File metadata from successfully produced parquet files. The Mutex is only used - /// to allow inserting to HashMap from behind borrowed reference in DataSink::write_all. - written: Arc>>, -} - -impl Debug for ParquetSink { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("ParquetSink").finish() - } -} - -impl DisplayAs for ParquetSink { - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "ParquetSink(file_groups=",)?; - FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; - write!(f, ")") - } - } - } -} - -impl ParquetSink { - /// Create from config. - pub fn new(config: FileSinkConfig, parquet_options: TableParquetOptions) -> Self { - Self { - config, - parquet_options, - written: Default::default(), - } - } - - /// Retrieve the file metadata for the written files, keyed to the path - /// which may be partitioned (in the case of hive style partitioning). - pub fn written(&self) -> HashMap { - self.written.lock().clone() - } - - /// Create writer properties based upon configuration settings, - /// including partitioning and the inclusion of arrow schema metadata. - fn create_writer_props(&self) -> Result { - let schema = if self.parquet_options.global.allow_single_file_parallelism { - // If parallelizing writes, we may be also be doing hive style partitioning - // into multiple files which impacts the schema per file. - // Refer to `get_writer_schema()` - &get_writer_schema(&self.config) - } else { - self.config.output_schema() - }; - - // TODO: avoid this clone in follow up PR, where the writer properties & schema - // are calculated once on `ParquetSink::new` - let mut parquet_opts = self.parquet_options.clone(); - if !self.parquet_options.global.skip_arrow_metadata { - parquet_opts.arrow_schema(schema); - } - - Ok(WriterPropertiesBuilder::try_from(&parquet_opts)?.build()) - } - - /// Creates an AsyncArrowWriter which serializes a parquet file to an ObjectStore - /// AsyncArrowWriters are used when individual parquet file serialization is not parallelized - async fn create_async_arrow_writer( - &self, - location: &Path, - object_store: Arc, - parquet_props: WriterProperties, - ) -> Result> { - let buf_writer = BufWriter::new(object_store, location.clone()); - let options = ArrowWriterOptions::new() - .with_properties(parquet_props) - .with_skip_arrow_metadata(self.parquet_options.global.skip_arrow_metadata); - - let writer = AsyncArrowWriter::try_new_with_options( - buf_writer, - get_writer_schema(&self.config), - options, - )?; - Ok(writer) - } - - /// Parquet options - pub fn parquet_options(&self) -> &TableParquetOptions { - &self.parquet_options - } -} - -#[async_trait] -impl FileSink for ParquetSink { - fn config(&self) -> &FileSinkConfig { - &self.config - } - - async fn spawn_writer_tasks_and_join( - &self, - context: &Arc, - demux_task: SpawnedTask>, - mut file_stream_rx: DemuxedStreamReceiver, - object_store: Arc, - ) -> Result { - let parquet_opts = &self.parquet_options; - let allow_single_file_parallelism = - parquet_opts.global.allow_single_file_parallelism; - - let mut file_write_tasks: JoinSet< - std::result::Result<(Path, FileMetaData), DataFusionError>, - > = JoinSet::new(); - - let parquet_props = self.create_writer_props()?; - let parallel_options = ParallelParquetWriterOptions { - max_parallel_row_groups: parquet_opts - .global - .maximum_parallel_row_group_writers, - max_buffered_record_batches_per_stream: parquet_opts - .global - .maximum_buffered_record_batches_per_stream, - }; - - while let Some((path, mut rx)) = file_stream_rx.recv().await { - if !allow_single_file_parallelism { - let mut writer = self - .create_async_arrow_writer( - &path, - Arc::clone(&object_store), - parquet_props.clone(), - ) - .await?; - let mut reservation = - MemoryConsumer::new(format!("ParquetSink[{}]", path)) - .register(context.memory_pool()); - file_write_tasks.spawn(async move { - while let Some(batch) = rx.recv().await { - writer.write(&batch).await?; - reservation.try_resize(writer.memory_size())?; - } - let file_metadata = writer - .close() - .await - .map_err(DataFusionError::ParquetError)?; - Ok((path, file_metadata)) - }); - } else { - let writer = create_writer( - // Parquet files as a whole are never compressed, since they - // manage compressed blocks themselves. - FileCompressionType::UNCOMPRESSED, - &path, - Arc::clone(&object_store), - ) - .await?; - let schema = get_writer_schema(&self.config); - let props = parquet_props.clone(); - let parallel_options_clone = parallel_options.clone(); - let pool = Arc::clone(context.memory_pool()); - file_write_tasks.spawn(async move { - let file_metadata = output_single_parquet_file_parallelized( - writer, - rx, - schema, - &props, - parallel_options_clone, - pool, - ) - .await?; - Ok((path, file_metadata)) - }); - } - } - - let mut row_count = 0; - while let Some(result) = file_write_tasks.join_next().await { - match result { - Ok(r) => { - let (path, file_metadata) = r?; - row_count += file_metadata.num_rows; - let mut written_files = self.written.lock(); - written_files - .try_insert(path.clone(), file_metadata) - .map_err(|e| internal_datafusion_err!("duplicate entry detected for partitioned file {path}: {e}"))?; - drop(written_files); - } - Err(e) => { - if e.is_panic() { - std::panic::resume_unwind(e.into_panic()); - } else { - unreachable!(); - } - } - } - } - - demux_task - .join_unwind() - .await - .map_err(DataFusionError::ExecutionJoin)??; - - Ok(row_count as u64) - } -} - -#[async_trait] -impl DataSink for ParquetSink { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> &SchemaRef { - self.config.output_schema() - } - - async fn write_all( - &self, - data: SendableRecordBatchStream, - context: &Arc, - ) -> Result { - FileSink::write_all(self, data, context).await - } -} - -/// Consumes a stream of [ArrowLeafColumn] via a channel and serializes them using an [ArrowColumnWriter] -/// Once the channel is exhausted, returns the ArrowColumnWriter. -async fn column_serializer_task( - mut rx: Receiver, - mut writer: ArrowColumnWriter, - mut reservation: MemoryReservation, -) -> Result<(ArrowColumnWriter, MemoryReservation)> { - while let Some(col) = rx.recv().await { - writer.write(&col)?; - reservation.try_resize(writer.memory_size())?; - } - Ok((writer, reservation)) -} - -type ColumnWriterTask = SpawnedTask>; -type ColSender = Sender; - -/// Spawns a parallel serialization task for each column -/// Returns join handles for each columns serialization task along with a send channel -/// to send arrow arrays to each serialization task. -fn spawn_column_parallel_row_group_writer( - schema: Arc, - parquet_props: Arc, - max_buffer_size: usize, - pool: &Arc, -) -> Result<(Vec, Vec)> { - let schema_desc = ArrowSchemaConverter::new().convert(&schema)?; - let col_writers = get_column_writers(&schema_desc, &parquet_props, &schema)?; - let num_columns = col_writers.len(); - - let mut col_writer_tasks = Vec::with_capacity(num_columns); - let mut col_array_channels = Vec::with_capacity(num_columns); - for writer in col_writers.into_iter() { - // Buffer size of this channel limits the number of arrays queued up for column level serialization - let (send_array, receive_array) = - mpsc::channel::(max_buffer_size); - col_array_channels.push(send_array); - - let reservation = - MemoryConsumer::new("ParquetSink(ArrowColumnWriter)").register(pool); - let task = SpawnedTask::spawn(column_serializer_task( - receive_array, - writer, - reservation, - )); - col_writer_tasks.push(task); - } - - Ok((col_writer_tasks, col_array_channels)) -} - -/// Settings related to writing parquet files in parallel -#[derive(Clone)] -struct ParallelParquetWriterOptions { - max_parallel_row_groups: usize, - max_buffered_record_batches_per_stream: usize, -} - -/// This is the return type of calling [ArrowColumnWriter].close() on each column -/// i.e. the Vec of encoded columns which can be appended to a row group -type RBStreamSerializeResult = Result<(Vec, MemoryReservation, usize)>; - -/// Sends the ArrowArrays in passed [RecordBatch] through the channels to their respective -/// parallel column serializers. -async fn send_arrays_to_col_writers( - col_array_channels: &[ColSender], - rb: &RecordBatch, - schema: Arc, -) -> Result<()> { - // Each leaf column has its own channel, increment next_channel for each leaf column sent. - let mut next_channel = 0; - for (array, field) in rb.columns().iter().zip(schema.fields()) { - for c in compute_leaves(field, array)? { - // Do not surface error from closed channel (means something - // else hit an error, and the plan is shutting down). - if col_array_channels[next_channel].send(c).await.is_err() { - return Ok(()); - } - - next_channel += 1; - } - } - - Ok(()) -} - -/// Spawns a tokio task which joins the parallel column writer tasks, -/// and finalizes the row group -fn spawn_rg_join_and_finalize_task( - column_writer_tasks: Vec, - rg_rows: usize, - pool: &Arc, -) -> SpawnedTask { - let mut rg_reservation = - MemoryConsumer::new("ParquetSink(SerializedRowGroupWriter)").register(pool); - - SpawnedTask::spawn(async move { - let num_cols = column_writer_tasks.len(); - let mut finalized_rg = Vec::with_capacity(num_cols); - for task in column_writer_tasks.into_iter() { - let (writer, _col_reservation) = task - .join_unwind() - .await - .map_err(DataFusionError::ExecutionJoin)??; - let encoded_size = writer.get_estimated_total_bytes(); - rg_reservation.grow(encoded_size); - finalized_rg.push(writer.close()?); - } - - Ok((finalized_rg, rg_reservation, rg_rows)) - }) -} - -/// This task coordinates the serialization of a parquet file in parallel. -/// As the query produces RecordBatches, these are written to a RowGroup -/// via parallel [ArrowColumnWriter] tasks. Once the desired max rows per -/// row group is reached, the parallel tasks are joined on another separate task -/// and sent to a concatenation task. This task immediately continues to work -/// on the next row group in parallel. So, parquet serialization is parallelized -/// across both columns and row_groups, with a theoretical max number of parallel tasks -/// given by n_columns * num_row_groups. -fn spawn_parquet_parallel_serialization_task( - mut data: Receiver, - serialize_tx: Sender>, - schema: Arc, - writer_props: Arc, - parallel_options: ParallelParquetWriterOptions, - pool: Arc, -) -> SpawnedTask> { - SpawnedTask::spawn(async move { - let max_buffer_rb = parallel_options.max_buffered_record_batches_per_stream; - let max_row_group_rows = writer_props.max_row_group_size(); - let (mut column_writer_handles, mut col_array_channels) = - spawn_column_parallel_row_group_writer( - Arc::clone(&schema), - Arc::clone(&writer_props), - max_buffer_rb, - &pool, - )?; - let mut current_rg_rows = 0; - - while let Some(mut rb) = data.recv().await { - // This loop allows the "else" block to repeatedly split the RecordBatch to handle the case - // when max_row_group_rows < execution.batch_size as an alternative to a recursive async - // function. - loop { - if current_rg_rows + rb.num_rows() < max_row_group_rows { - send_arrays_to_col_writers( - &col_array_channels, - &rb, - Arc::clone(&schema), - ) - .await?; - current_rg_rows += rb.num_rows(); - break; - } else { - let rows_left = max_row_group_rows - current_rg_rows; - let a = rb.slice(0, rows_left); - send_arrays_to_col_writers( - &col_array_channels, - &a, - Arc::clone(&schema), - ) - .await?; - - // Signal the parallel column writers that the RowGroup is done, join and finalize RowGroup - // on a separate task, so that we can immediately start on the next RG before waiting - // for the current one to finish. - drop(col_array_channels); - let finalize_rg_task = spawn_rg_join_and_finalize_task( - column_writer_handles, - max_row_group_rows, - &pool, - ); - - // Do not surface error from closed channel (means something - // else hit an error, and the plan is shutting down). - if serialize_tx.send(finalize_rg_task).await.is_err() { - return Ok(()); - } - - current_rg_rows = 0; - rb = rb.slice(rows_left, rb.num_rows() - rows_left); - - (column_writer_handles, col_array_channels) = - spawn_column_parallel_row_group_writer( - Arc::clone(&schema), - Arc::clone(&writer_props), - max_buffer_rb, - &pool, - )?; - } - } - } - - drop(col_array_channels); - // Handle leftover rows as final rowgroup, which may be smaller than max_row_group_rows - if current_rg_rows > 0 { - let finalize_rg_task = spawn_rg_join_and_finalize_task( - column_writer_handles, - current_rg_rows, - &pool, - ); +//! Re-exports the [`datafusion_datasource_parquet::file_format`] module, and contains tests for it. - // Do not surface error from closed channel (means something - // else hit an error, and the plan is shutting down). - if serialize_tx.send(finalize_rg_task).await.is_err() { - return Ok(()); - } - } - - Ok(()) - }) -} - -/// Consume RowGroups serialized by other parallel tasks and concatenate them in -/// to the final parquet file, while flushing finalized bytes to an [ObjectStore] -async fn concatenate_parallel_row_groups( - mut serialize_rx: Receiver>, - schema: Arc, - writer_props: Arc, - mut object_store_writer: Box, - pool: Arc, -) -> Result { - let merged_buff = SharedBuffer::new(INITIAL_BUFFER_BYTES); - - let mut file_reservation = - MemoryConsumer::new("ParquetSink(SerializedFileWriter)").register(&pool); - - let schema_desc = ArrowSchemaConverter::new().convert(schema.as_ref())?; - let mut parquet_writer = SerializedFileWriter::new( - merged_buff.clone(), - schema_desc.root_schema_ptr(), - writer_props, - )?; - - while let Some(task) = serialize_rx.recv().await { - let result = task.join_unwind().await; - let mut rg_out = parquet_writer.next_row_group()?; - let (serialized_columns, mut rg_reservation, _cnt) = - result.map_err(DataFusionError::ExecutionJoin)??; - for chunk in serialized_columns { - chunk.append_to_row_group(&mut rg_out)?; - rg_reservation.free(); - - let mut buff_to_flush = merged_buff.buffer.try_lock().unwrap(); - file_reservation.try_resize(buff_to_flush.len())?; - - if buff_to_flush.len() > BUFFER_FLUSH_BYTES { - object_store_writer - .write_all(buff_to_flush.as_slice()) - .await?; - buff_to_flush.clear(); - file_reservation.try_resize(buff_to_flush.len())?; // will set to zero - } - } - rg_out.close()?; - } - - let file_metadata = parquet_writer.close()?; - let final_buff = merged_buff.buffer.try_lock().unwrap(); - - object_store_writer.write_all(final_buff.as_slice()).await?; - object_store_writer.shutdown().await?; - file_reservation.free(); - - Ok(file_metadata) -} - -/// Parallelizes the serialization of a single parquet file, by first serializing N -/// independent RecordBatch streams in parallel to RowGroups in memory. Another -/// task then stitches these independent RowGroups together and streams this large -/// single parquet file to an ObjectStore in multiple parts. -async fn output_single_parquet_file_parallelized( - object_store_writer: Box, - data: Receiver, - output_schema: Arc, - parquet_props: &WriterProperties, - parallel_options: ParallelParquetWriterOptions, - pool: Arc, -) -> Result { - let max_rowgroups = parallel_options.max_parallel_row_groups; - // Buffer size of this channel limits maximum number of RowGroups being worked on in parallel - let (serialize_tx, serialize_rx) = - mpsc::channel::>(max_rowgroups); - - let arc_props = Arc::new(parquet_props.clone()); - let launch_serialization_task = spawn_parquet_parallel_serialization_task( - data, - serialize_tx, - Arc::clone(&output_schema), - Arc::clone(&arc_props), - parallel_options, - Arc::clone(&pool), - ); - let file_metadata = concatenate_parallel_row_groups( - serialize_rx, - Arc::clone(&output_schema), - Arc::clone(&arc_props), - object_store_writer, - pool, - ) - .await?; - - launch_serialization_task - .join_unwind() - .await - .map_err(DataFusionError::ExecutionJoin)??; - Ok(file_metadata) -} +pub use datafusion_datasource_parquet::file_format::*; #[cfg(test)] pub(crate) mod test_util { - use super::*; - use crate::test::object_store::local_unpartitioned_file; - - use parquet::arrow::ArrowWriter; - use tempfile::NamedTempFile; + use arrow::array::RecordBatch; + use datafusion_common::Result; + use object_store::ObjectMeta; - /// How many rows per page should be written - const ROWS_PER_PAGE: usize = 2; + use crate::test::object_store::local_unpartitioned_file; /// Writes `batches` to a temporary parquet file /// @@ -1378,12 +35,28 @@ pub(crate) mod test_util { pub async fn store_parquet( batches: Vec, multi_page: bool, - ) -> Result<(Vec, Vec)> { + ) -> Result<(Vec, Vec)> { + /// How many rows per page should be written + const ROWS_PER_PAGE: usize = 2; + /// write batches chunk_size rows at a time + fn write_in_chunks( + writer: &mut parquet::arrow::ArrowWriter, + batch: &RecordBatch, + chunk_size: usize, + ) { + let mut i = 0; + while i < batch.num_rows() { + let num = chunk_size.min(batch.num_rows() - i); + writer.write(&batch.slice(i, num)).unwrap(); + i += num; + } + } + // we need the tmp files to be sorted as some tests rely on the how the returning files are ordered // https://github.com/apache/datafusion/pull/6629 let tmp_files = { let mut tmp_files: Vec<_> = (0..batches.len()) - .map(|_| NamedTempFile::new().expect("creating temp file")) + .map(|_| tempfile::NamedTempFile::new().expect("creating temp file")) .collect(); tmp_files.sort_by(|a, b| a.path().cmp(b.path())); tmp_files @@ -1394,7 +67,7 @@ pub(crate) mod test_util { .into_iter() .zip(tmp_files.into_iter()) .map(|(batch, mut output)| { - let builder = WriterProperties::builder(); + let builder = parquet::file::properties::WriterProperties::builder(); let props = if multi_page { builder.set_data_page_row_count_limit(ROWS_PER_PAGE) } else { @@ -1402,9 +75,12 @@ pub(crate) mod test_util { } .build(); - let mut writer = - ArrowWriter::try_new(&mut output, batch.schema(), Some(props)) - .expect("creating writer"); + let mut writer = parquet::arrow::ArrowWriter::try_new( + &mut output, + batch.schema(), + Some(props), + ) + .expect("creating writer"); if multi_page { // write in smaller batches as the parquet writer @@ -1422,65 +98,69 @@ pub(crate) mod test_util { Ok((meta, files)) } - - /// write batches chunk_size rows at a time - fn write_in_chunks( - writer: &mut ArrowWriter, - batch: &RecordBatch, - chunk_size: usize, - ) { - let mut i = 0; - while i < batch.num_rows() { - let num = chunk_size.min(batch.num_rows() - i); - writer.write(&batch.slice(i, num)).unwrap(); - i += num; - } - } } #[cfg(test)] mod tests { - use super::super::test_util::scan_format; - use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; - use crate::execution::SessionState; - use crate::physical_plan::collect; - use crate::test_util::bounded_stream; - use std::fmt::{Display, Formatter}; + + use std::fmt::{self, Display, Formatter}; + use std::pin::Pin; use std::sync::atomic::{AtomicUsize, Ordering}; + use std::sync::Arc; + use std::task::{Context, Poll}; use std::time::Duration; - use super::*; - use crate::datasource::file_format::parquet::test_util::store_parquet; + use crate::datasource::file_format::test_util::scan_format; + use crate::execution::SessionState; use crate::physical_plan::metrics::MetricValue; use crate::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; - use arrow::array::{ - types::Int32Type, Array, ArrayRef, DictionaryArray, Int32Array, Int64Array, - StringArray, - }; - use arrow::datatypes::{DataType, Field}; - use async_trait::async_trait; + + use arrow::array::RecordBatch; + use arrow_schema::{Schema, SchemaRef}; + use datafusion_catalog::Session; use datafusion_common::cast::{ as_binary_array, as_binary_view_array, as_boolean_array, as_float32_array, as_float64_array, as_int32_array, as_timestamp_nanosecond_array, }; - use datafusion_common::config::ParquetOptions; + use datafusion_common::config::{ParquetOptions, TableParquetOptions}; + use datafusion_common::stats::Precision; use datafusion_common::ScalarValue::Utf8; - use datafusion_common::{assert_batches_eq, ScalarValue}; + use datafusion_common::{assert_batches_eq, Result, ScalarValue}; + use datafusion_datasource::file_format::FileFormat; + use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig}; + use datafusion_datasource::{ListingTableUrl, PartitionedFile}; + use datafusion_datasource_parquet::{ + fetch_parquet_metadata, fetch_statistics, statistics_from_parquet_meta_calc, + ParquetFormat, ParquetFormatFactory, ParquetSink, + }; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::runtime_env::RuntimeEnv; + use datafusion_execution::{RecordBatchStream, TaskContext}; + use datafusion_expr::dml::InsertOp; use datafusion_physical_plan::stream::RecordBatchStreamAdapter; + use datafusion_physical_plan::{collect, ExecutionPlan}; + + use arrow::array::{ + types::Int32Type, Array, ArrayRef, DictionaryArray, Int32Array, Int64Array, + StringArray, + }; + use arrow::datatypes::{DataType, Field}; + use async_trait::async_trait; use futures::stream::BoxStream; + use futures::{Stream, StreamExt}; use log::error; use object_store::local::LocalFileSystem; + use object_store::ObjectMeta; use object_store::{ - GetOptions, GetResult, ListResult, MultipartUpload, PutMultipartOpts, PutOptions, - PutPayload, PutResult, + path::Path, GetOptions, GetResult, ListResult, MultipartUpload, ObjectStore, + PutMultipartOpts, PutOptions, PutPayload, PutResult, }; use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::arrow::ParquetRecordBatchStreamBuilder; use parquet::file::metadata::{KeyValue, ParquetColumnIndex, ParquetOffsetIndex}; use parquet::file::page_index::index::Index; + use parquet::format::FileMetaData; use tokio::fs::File; enum ForceViews { @@ -2302,7 +982,7 @@ mod tests { } #[tokio::test] async fn test_read_parquet_page_index() -> Result<()> { - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let path = format!("{testdata}/alltypes_tiny_pages.parquet"); let file = File::open(path).await.unwrap(); let options = ArrowReaderOptions::new().with_page_index(true); @@ -2385,7 +1065,7 @@ mod tests { projection: Option>, limit: Option, ) -> Result> { - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let state = state.as_any().downcast_ref::().unwrap(); let format = state .get_file_format_factory("parquet") @@ -2954,4 +1634,43 @@ mod tests { Ok(()) } + + /// Creates an bounded stream for testing purposes. + fn bounded_stream( + batch: RecordBatch, + limit: usize, + ) -> datafusion_execution::SendableRecordBatchStream { + Box::pin(BoundedStream { + count: 0, + limit, + batch, + }) + } + + struct BoundedStream { + limit: usize, + count: usize, + batch: RecordBatch, + } + + impl Stream for BoundedStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + if self.count >= self.limit { + return Poll::Ready(None); + } + self.count += 1; + Poll::Ready(Some(Ok(self.batch.clone()))) + } + } + + impl RecordBatchStream for BoundedStream { + fn schema(&self) -> SchemaRef { + self.batch.schema() + } + } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 41e939d60b08..6fe3d08723bc 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1173,7 +1173,6 @@ impl ListingTable { #[cfg(test)] mod tests { use super::*; - use crate::datasource::file_format::avro::AvroFormat; use crate::datasource::file_format::csv::CsvFormat; use crate::datasource::file_format::json::JsonFormat; #[cfg(feature = "parquet")] @@ -1185,6 +1184,7 @@ mod tests { assert_batches_eq, test::{columns, object_store::register_test_store}, }; + use datafusion_datasource_avro::AvroFormat; use datafusion_physical_plan::collect; use arrow::compute::SortOptions; @@ -1368,7 +1368,7 @@ mod tests { let path = String::from("table/p1=v1/file.avro"); register_test_store(&ctx, &[(&path, 100)]); - let opt = ListingOptions::new(Arc::new(AvroFormat {})) + let opt = ListingOptions::new(Arc::new(AvroFormat)) .with_file_extension(AvroFormat.get_ext()) .with_table_partition_cols(vec![(String::from("p1"), DataType::Utf8)]) .with_target_partitions(4); @@ -1403,6 +1403,7 @@ mod tests { Ok(()) } + #[cfg(feature = "avro")] #[tokio::test] async fn test_assert_list_files_for_scan_grouping() -> Result<()> { // more expected partitions than files @@ -1492,6 +1493,7 @@ mod tests { Ok(()) } + #[cfg(feature = "avro")] #[tokio::test] async fn test_assert_list_files_for_multi_path() -> Result<()> { // more expected partitions than files @@ -1612,9 +1614,7 @@ mod tests { let ctx = SessionContext::new(); register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::>()); - let format = AvroFormat {}; - - let opt = ListingOptions::new(Arc::new(format)) + let opt = ListingOptions::new(Arc::new(AvroFormat)) .with_file_extension_opt(file_ext) .with_target_partitions(target_partitions); @@ -1636,6 +1636,7 @@ mod tests { /// Check that the files listed by the table match the specified `output_partitioning` /// when the object store contains `files`. + #[cfg(feature = "avro")] async fn assert_list_files_for_multi_paths( files: &[&str], table_prefix: &[&str], @@ -1643,10 +1644,12 @@ mod tests { output_partitioning: usize, file_ext: Option<&str>, ) -> Result<()> { + use datafusion_datasource_avro::AvroFormat; + let ctx = SessionContext::new(); register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::>()); - let format = AvroFormat {}; + let format = AvroFormat; let opt = ListingOptions::new(Arc::new(format)) .with_file_extension_opt(file_ext) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 2b7bb14b6f6c..8661def83843 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -19,7 +19,6 @@ //! //! [`ListingTable`]: crate::datasource::listing::ListingTable -pub mod avro_to_arrow; pub mod cte_worktable; pub mod default_table_source; pub mod dynamic_file; @@ -30,11 +29,11 @@ pub mod listing_table_factory; pub mod memory; pub mod physical_plan; pub mod provider; -pub mod schema_adapter; mod statistics; pub mod stream; pub mod view; +pub use datafusion_datasource::schema_adapter; pub use datafusion_datasource::source; // backwards compatibility @@ -92,3 +91,226 @@ fn create_ordering( } Ok(all_sort_orders) } + +#[cfg(all(test, feature = "parquet"))] +mod tests { + + use crate::prelude::SessionContext; + + use std::fs; + use std::sync::Arc; + + use arrow::array::{Int32Array, StringArray}; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use arrow::record_batch::RecordBatch; + use datafusion_common::assert_batches_sorted_eq; + use datafusion_datasource::file_scan_config::FileScanConfig; + use datafusion_datasource::schema_adapter::{ + DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + }; + use datafusion_datasource::PartitionedFile; + use datafusion_datasource_parquet::source::ParquetSource; + + use datafusion_common::record_batch; + + use ::object_store::path::Path; + use ::object_store::ObjectMeta; + use datafusion_physical_plan::collect; + use tempfile::TempDir; + + #[tokio::test] + async fn can_override_schema_adapter() { + // Test shows that SchemaAdapter can add a column that doesn't existing in the + // record batches returned from parquet. This can be useful for schema evolution + // where older files may not have all columns. + + use datafusion_execution::object_store::ObjectStoreUrl; + let tmp_dir = TempDir::new().unwrap(); + let table_dir = tmp_dir.path().join("parquet_test"); + fs::DirBuilder::new().create(table_dir.as_path()).unwrap(); + let f1 = Field::new("id", DataType::Int32, true); + + let file_schema = Arc::new(Schema::new(vec![f1.clone()])); + let filename = "part.parquet".to_string(); + let path = table_dir.as_path().join(filename.clone()); + let file = fs::File::create(path.clone()).unwrap(); + let mut writer = + parquet::arrow::ArrowWriter::try_new(file, file_schema.clone(), None) + .unwrap(); + + let ids = Arc::new(Int32Array::from(vec![1i32])); + let rec_batch = RecordBatch::try_new(file_schema.clone(), vec![ids]).unwrap(); + + writer.write(&rec_batch).unwrap(); + writer.close().unwrap(); + + let location = Path::parse(path.to_str().unwrap()).unwrap(); + let metadata = fs::metadata(path.as_path()).expect("Local file metadata"); + let meta = ObjectMeta { + location, + last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), + size: metadata.len() as usize, + e_tag: None, + version: None, + }; + + let partitioned_file = PartitionedFile { + object_meta: meta, + partition_values: vec![], + range: None, + statistics: None, + extensions: None, + metadata_size_hint: None, + }; + + let f1 = Field::new("id", DataType::Int32, true); + let f2 = Field::new("extra_column", DataType::Utf8, true); + + let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); + let source = Arc::new( + ParquetSource::default() + .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})), + ); + let base_conf = + FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema, source) + .with_file(partitioned_file); + + let parquet_exec = base_conf.build(); + + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + let read = collect(parquet_exec, task_ctx).await.unwrap(); + + let expected = [ + "+----+--------------+", + "| id | extra_column |", + "+----+--------------+", + "| 1 | foo |", + "+----+--------------+", + ]; + + assert_batches_sorted_eq!(expected, &read); + } + + #[test] + fn default_schema_adapter() { + let table_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Utf8, true), + ]); + + // file has a subset of the table schema fields and different type + let file_schema = Schema::new(vec![ + Field::new("c", DataType::Float64, true), // not in table schema + Field::new("b", DataType::Float64, true), + ]); + + let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); + let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(indices, vec![1]); + + let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); + + let mapped_batch = mapper.map_batch(file_batch).unwrap(); + + // the mapped batch has the correct schema and the "b" column has been cast to Utf8 + let expected_batch = record_batch!( + ("a", Int32, vec![None, None]), // missing column filled with nulls + ("b", Utf8, vec!["1.0", "2.0"]) // b was cast to string and order was changed + ) + .unwrap(); + assert_eq!(mapped_batch, expected_batch); + } + + #[test] + fn default_schema_adapter_non_nullable_columns() { + let table_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), // "a"" is declared non nullable + Field::new("b", DataType::Utf8, true), + ]); + let file_schema = Schema::new(vec![ + // since file doesn't have "a" it will be filled with nulls + Field::new("b", DataType::Float64, true), + ]); + + let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); + let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(indices, vec![0]); + + let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); + + // Mapping fails because it tries to fill in a non-nullable column with nulls + let err = mapper.map_batch(file_batch).unwrap_err().to_string(); + assert!(err.contains("Invalid argument error: Column 'a' is declared as non-nullable but contains null values"), "{err}"); + } + + #[derive(Debug)] + struct TestSchemaAdapterFactory; + + impl SchemaAdapterFactory for TestSchemaAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(TestSchemaAdapter { + table_schema: projected_table_schema, + }) + } + } + + struct TestSchemaAdapter { + /// Schema for the table + table_schema: SchemaRef, + } + + impl SchemaAdapter for TestSchemaAdapter { + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.table_schema.field(index); + Some(file_schema.fields.find(field.name())?.0) + } + + fn map_schema( + &self, + file_schema: &Schema, + ) -> datafusion_common::Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + + for (file_idx, file_field) in file_schema.fields.iter().enumerate() { + if self.table_schema.fields().find(file_field.name()).is_some() { + projection.push(file_idx); + } + } + + Ok((Arc::new(TestSchemaMapping {}), projection)) + } + } + + #[derive(Debug)] + struct TestSchemaMapping {} + + impl SchemaMapper for TestSchemaMapping { + fn map_batch( + &self, + batch: RecordBatch, + ) -> datafusion_common::Result { + let f1 = Field::new("id", DataType::Int32, true); + let f2 = Field::new("extra_column", DataType::Utf8, true); + + let schema = Arc::new(Schema::new(vec![f1, f2])); + + let extra_column = Arc::new(StringArray::from(vec!["foo"])); + let mut new_columns = batch.columns().to_vec(); + new_columns.push(extra_column); + + Ok(RecordBatch::try_new(schema, new_columns).unwrap()) + } + + fn map_partial_batch( + &self, + batch: RecordBatch, + ) -> datafusion_common::Result { + self.map_batch(batch) + } + } +} diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index a0e1135e2cac..59860e3594f5 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -21,9 +21,7 @@ use std::any::Any; use std::sync::Arc; use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::{ - FileMeta, FileOpenFuture, FileOpener, JsonSource, -}; +use crate::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener}; use crate::error::Result; use arrow::buffer::Buffer; @@ -34,6 +32,7 @@ use datafusion_common::{Constraints, Statistics}; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; +use datafusion_datasource_json::source::JsonSource; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 08c22183302b..9fa2b3bc1482 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -15,350 +15,28 @@ // specific language governing permissions and limitations // under the License. -//! Execution plan for reading line-delimited Avro files - -use std::any::Any; -use std::fmt::Formatter; -use std::sync::Arc; - -use super::FileOpener; -#[cfg(feature = "avro")] -use crate::datasource::avro_to_arrow::Reader as AvroReader; - -use crate::error::Result; - -use arrow::datatypes::SchemaRef; -use datafusion_common::{Constraints, Statistics}; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_datasource::source::DataSourceExec; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use datafusion_physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, -}; - -use object_store::ObjectStore; - -/// Execution plan for scanning Avro data source -#[derive(Debug, Clone)] -#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] -pub struct AvroExec { - inner: DataSourceExec, - base_config: FileScanConfig, -} - -#[allow(unused, deprecated)] -impl AvroExec { - /// Create a new Avro reader execution plan provided base configurations - pub fn new(base_config: FileScanConfig) -> Self { - let ( - projected_schema, - projected_constraints, - projected_statistics, - projected_output_ordering, - ) = base_config.project(); - let cache = Self::compute_properties( - Arc::clone(&projected_schema), - &projected_output_ordering, - projected_constraints, - &base_config, - ); - let base_config = base_config.with_source(Arc::new(AvroSource::default())); - Self { - inner: DataSourceExec::new(Arc::new(base_config.clone())), - base_config, - } - } - - /// Ref to the base configs - pub fn base_config(&self) -> &FileScanConfig { - &self.base_config - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - orderings: &[LexOrdering], - constraints: Constraints, - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) - .with_constraints(constraints); - let n_partitions = file_scan_config.file_groups.len(); - - PlanProperties::new( - eq_properties, - Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } -} - -#[allow(unused, deprecated)] -impl DisplayAs for AvroExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - self.inner.fmt_as(t, f) - } -} - -#[allow(unused, deprecated)] -impl ExecutionPlan for AvroExec { - fn name(&self) -> &'static str { - "AvroExec" - } - - fn as_any(&self) -> &dyn Any { - self - } +//! Reexports the [`datafusion_datasource_json::source`] module, containing [Avro] based [`FileSource`]. +//! +//! [Avro]: https://avro.apache.org/ +//! [`FileSource`]: datafusion_datasource::file::FileSource - fn properties(&self) -> &PlanProperties { - self.inner.properties() - } - fn children(&self) -> Vec<&Arc> { - Vec::new() - } - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - #[cfg(not(feature = "avro"))] - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - Err(crate::error::DataFusionError::NotImplemented( - "Cannot execute avro plan without avro feature enabled".to_string(), - )) - } - #[cfg(feature = "avro")] - fn execute( - &self, - partition: usize, - context: Arc, - ) -> Result { - self.inner.execute(partition, context) - } - - fn statistics(&self) -> Result { - self.inner.statistics() - } - - fn metrics(&self) -> Option { - self.inner.metrics() - } - - fn fetch(&self) -> Option { - self.inner.fetch() - } - - fn with_fetch(&self, limit: Option) -> Option> { - self.inner.with_fetch(limit) - } -} - -/// AvroSource holds the extra configuration that is necessary for opening avro files -#[derive(Clone, Default)] -pub struct AvroSource { - schema: Option, - batch_size: Option, - projection: Option>, - metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, -} - -impl AvroSource { - /// Initialize an AvroSource with default values - pub fn new() -> Self { - Self::default() - } - - #[cfg(feature = "avro")] - fn open(&self, reader: R) -> Result> { - AvroReader::try_new( - reader, - Arc::clone(self.schema.as_ref().expect("Schema must set before open")), - self.batch_size.expect("Batch size must set before open"), - self.projection.clone(), - ) - } -} - -impl FileSource for AvroSource { - #[cfg(feature = "avro")] - fn create_file_opener( - &self, - object_store: Arc, - _base_config: &FileScanConfig, - _partition: usize, - ) -> Arc { - Arc::new(private::AvroOpener { - config: Arc::new(self.clone()), - object_store, - }) - } - - #[cfg(not(feature = "avro"))] - fn create_file_opener( - &self, - _object_store: Arc, - _base_config: &FileScanConfig, - _partition: usize, - ) -> Arc { - panic!("Avro feature is not enabled in this build") - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn with_batch_size(&self, batch_size: usize) -> Arc { - let mut conf = self.clone(); - conf.batch_size = Some(batch_size); - Arc::new(conf) - } - - fn with_schema(&self, schema: SchemaRef) -> Arc { - let mut conf = self.clone(); - conf.schema = Some(schema); - Arc::new(conf) - } - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, config: &FileScanConfig) -> Arc { - let mut conf = self.clone(); - conf.projection = config.projected_file_column_names(); - Arc::new(conf) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { - &self.metrics - } - - fn statistics(&self) -> Result { - let statistics = &self.projected_statistics; - Ok(statistics - .clone() - .expect("projected_statistics must be set")) - } - - fn file_type(&self) -> &str { - "avro" - } - - fn repartitioned( - &self, - _target_partitions: usize, - _repartition_file_min_size: usize, - _output_ordering: Option, - _config: &FileScanConfig, - ) -> Result> { - Ok(None) - } -} - -#[cfg(feature = "avro")] -mod private { - use super::*; - use crate::datasource::physical_plan::FileMeta; - use crate::datasource::physical_plan::{FileOpenFuture, FileOpener}; - - use bytes::Buf; - use futures::StreamExt; - use object_store::{GetResultPayload, ObjectStore}; - - pub struct DeprecatedAvroConfig { - pub schema: SchemaRef, - pub batch_size: usize, - pub projection: Option>, - pub object_store: Arc, - } - - impl DeprecatedAvroConfig { - fn open(&self, reader: R) -> Result> { - AvroReader::try_new( - reader, - Arc::clone(&self.schema), - self.batch_size, - self.projection.clone(), - ) - } - } - - pub struct DeprecatedAvroOpener { - pub config: Arc, - } - impl FileOpener for DeprecatedAvroOpener { - fn open(&self, file_meta: FileMeta) -> Result { - let config = Arc::clone(&self.config); - Ok(Box::pin(async move { - let r = config.object_store.get(file_meta.location()).await?; - match r.payload { - GetResultPayload::File(file, _) => { - let reader = config.open(file)?; - Ok(futures::stream::iter(reader).boxed()) - } - GetResultPayload::Stream(_) => { - let bytes = r.bytes().await?; - let reader = config.open(bytes.reader())?; - Ok(futures::stream::iter(reader).boxed()) - } - } - })) - } - } - - pub struct AvroOpener { - pub config: Arc, - pub object_store: Arc, - } - - impl FileOpener for AvroOpener { - fn open(&self, file_meta: FileMeta) -> Result { - let config = Arc::clone(&self.config); - let object_store = Arc::clone(&self.object_store); - Ok(Box::pin(async move { - let r = object_store.get(file_meta.location()).await?; - match r.payload { - GetResultPayload::File(file, _) => { - let reader = config.open(file)?; - Ok(futures::stream::iter(reader).boxed()) - } - GetResultPayload::Stream(_) => { - let bytes = r.bytes().await?; - let reader = config.open(bytes.reader())?; - Ok(futures::stream::iter(reader).boxed()) - } - } - })) - } - } -} +pub use datafusion_datasource_avro::source::*; #[cfg(test)] -#[cfg(feature = "avro")] mod tests { - use super::*; - use crate::arrow::datatypes::{DataType, Field, SchemaBuilder}; - use crate::datasource::file_format::{avro::AvroFormat, FileFormat}; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; + + use std::sync::Arc; + use crate::prelude::SessionContext; - use crate::scalar::ScalarValue; use crate::test::object_store::local_unpartitioned_file; - + use arrow::datatypes::{DataType, Field, SchemaBuilder}; + use datafusion_common::{assert_batches_eq, test_util, Result, ScalarValue}; + use datafusion_datasource::file_format::FileFormat; + use datafusion_datasource::file_scan_config::FileScanConfig; + use datafusion_datasource::PartitionedFile; + use datafusion_datasource_avro::source::AvroSource; + use datafusion_datasource_avro::AvroFormat; + use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_plan::ExecutionPlan; use futures::StreamExt; @@ -392,7 +70,7 @@ mod tests { let url = Url::parse("file://").unwrap(); session_ctx.register_object_store(&url, store.clone()); - let testdata = crate::test_util::arrow_test_data(); + let testdata = test_util::arrow_test_data(); let filename = format!("{testdata}/avro/alltypes_plain.avro"); let meta = local_unpartitioned_file(filename); @@ -439,7 +117,7 @@ mod tests { "+----+----------+-------------+", ]; - crate::assert_batches_eq!(expected, &[batch]); + assert_batches_eq!(expected, &[batch]); let batch = results.next().await; assert!(batch.is_none()); @@ -458,7 +136,7 @@ mod tests { let session_ctx = SessionContext::new(); let state = session_ctx.state(); - let testdata = crate::test_util::arrow_test_data(); + let testdata = test_util::arrow_test_data(); let filename = format!("{testdata}/avro/alltypes_plain.avro"); let object_store = Arc::new(LocalFileSystem::new()) as _; let object_store_url = ObjectStoreUrl::local_filesystem(); @@ -513,7 +191,7 @@ mod tests { "+----+----------+-------------+-------------+", ]; - crate::assert_batches_eq!(expected, &[batch]); + assert_batches_eq!(expected, &[batch]); let batch = results.next().await; assert!(batch.is_none()); @@ -532,7 +210,7 @@ mod tests { let session_ctx = SessionContext::new(); let state = session_ctx.state(); - let testdata = crate::test_util::arrow_test_data(); + let testdata = test_util::arrow_test_data(); let filename = format!("{testdata}/avro/alltypes_plain.avro"); let object_store = Arc::new(LocalFileSystem::new()) as _; let object_store_url = ObjectStoreUrl::local_filesystem(); @@ -587,7 +265,7 @@ mod tests { "| 1 | false | 2021-10-26 | 1 |", "+----+----------+------------+-------------+", ]; - crate::assert_batches_eq!(expected, &[batch]); + assert_batches_eq!(expected, &[batch]); let batch = results.next().await; assert!(batch.is_none()); diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index bc7d6779bbfd..e80d04fe4b2f 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -15,794 +15,68 @@ // specific language governing permissions and limitations // under the License. -//! Execution plan for reading CSV files - -use std::any::Any; -use std::fmt; -use std::io::{Read, Seek, SeekFrom}; -use std::sync::Arc; -use std::task::Poll; - -use super::{calculate_range, RangeCalculation}; - -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; -use crate::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; -use crate::datasource::physical_plan::FileMeta; -use crate::datasource::physical_plan::{FileOpenFuture, FileOpener}; -use crate::error::{DataFusionError, Result}; -use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; - -use arrow::csv; -use arrow::datatypes::SchemaRef; -use datafusion_common::config::ConfigOptions; -use datafusion_common::{Constraints, Statistics}; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_datasource::source::DataSourceExec; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; - -use futures::{StreamExt, TryStreamExt}; -use object_store::buffered::BufWriter; -use object_store::{GetOptions, GetResultPayload, ObjectStore}; -use tokio::io::AsyncWriteExt; -use tokio::task::JoinSet; - -/// Old Csv source, deprecated with DataSourceExec implementation and CsvSource -/// -/// See examples on `CsvSource` -#[derive(Debug, Clone)] -#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] -pub struct CsvExec { - base_config: FileScanConfig, - inner: DataSourceExec, -} - -/// Builder for [`CsvExec`]. -/// -/// See example on [`CsvExec`]. -#[derive(Debug, Clone)] -#[deprecated(since = "46.0.0", note = "use FileScanConfig instead")] -pub struct CsvExecBuilder { - file_scan_config: FileScanConfig, - file_compression_type: FileCompressionType, - // TODO: it seems like these format options could be reused across all the various CSV config - has_header: bool, - delimiter: u8, - quote: u8, - terminator: Option, - escape: Option, - comment: Option, - newlines_in_values: bool, -} - -#[allow(unused, deprecated)] -impl CsvExecBuilder { - /// Create a new builder to read the provided file scan configuration. - pub fn new(file_scan_config: FileScanConfig) -> Self { - Self { - file_scan_config, - // TODO: these defaults are duplicated from `CsvOptions` - should they be computed? - has_header: false, - delimiter: b',', - quote: b'"', - terminator: None, - escape: None, - comment: None, - newlines_in_values: false, - file_compression_type: FileCompressionType::UNCOMPRESSED, - } - } - - /// Set whether the first row defines the column names. - /// - /// The default value is `false`. - pub fn with_has_header(mut self, has_header: bool) -> Self { - self.has_header = has_header; - self - } - - /// Set the column delimeter. - /// - /// The default is `,`. - pub fn with_delimeter(mut self, delimiter: u8) -> Self { - self.delimiter = delimiter; - self - } - - /// Set the quote character. - /// - /// The default is `"`. - pub fn with_quote(mut self, quote: u8) -> Self { - self.quote = quote; - self - } - - /// Set the line terminator. If not set, the default is CRLF. - /// - /// The default is None. - pub fn with_terminator(mut self, terminator: Option) -> Self { - self.terminator = terminator; - self - } - - /// Set the escape character. - /// - /// The default is `None` (i.e. quotes cannot be escaped). - pub fn with_escape(mut self, escape: Option) -> Self { - self.escape = escape; - self - } - - /// Set the comment character. - /// - /// The default is `None` (i.e. comments are not supported). - pub fn with_comment(mut self, comment: Option) -> Self { - self.comment = comment; - self - } - - /// Set whether newlines in (quoted) values are supported. - /// - /// Parsing newlines in quoted values may be affected by execution behaviour such as - /// parallel file scanning. Setting this to `true` ensures that newlines in values are - /// parsed successfully, which may reduce performance. - /// - /// The default value is `false`. - pub fn with_newlines_in_values(mut self, newlines_in_values: bool) -> Self { - self.newlines_in_values = newlines_in_values; - self - } - - /// Set the file compression type. - /// - /// The default is [`FileCompressionType::UNCOMPRESSED`]. - pub fn with_file_compression_type( - mut self, - file_compression_type: FileCompressionType, - ) -> Self { - self.file_compression_type = file_compression_type; - self - } +//! Reexports the [`datafusion_datasource_json::source`] module, containing CSV based [`FileSource`]. +//! +//! [`FileSource`]: datafusion_datasource::file::FileSource - /// Build a [`CsvExec`]. - #[must_use] - pub fn build(self) -> CsvExec { - let Self { - file_scan_config: base_config, - file_compression_type, - has_header, - delimiter, - quote, - terminator, - escape, - comment, - newlines_in_values, - } = self; - - let ( - projected_schema, - projected_constraints, - projected_statistics, - projected_output_ordering, - ) = base_config.project(); - let cache = CsvExec::compute_properties( - projected_schema, - &projected_output_ordering, - projected_constraints, - &base_config, - ); - let csv = CsvSource::new(has_header, delimiter, quote) - .with_comment(comment) - .with_escape(escape) - .with_terminator(terminator); - let base_config = base_config - .with_newlines_in_values(newlines_in_values) - .with_file_compression_type(file_compression_type) - .with_source(Arc::new(csv)); - - CsvExec { - inner: DataSourceExec::new(Arc::new(base_config.clone())), - base_config, - } - } -} - -#[allow(unused, deprecated)] -impl CsvExec { - /// Create a new CSV reader execution plan provided base and specific configurations - #[allow(clippy::too_many_arguments)] - pub fn new( - base_config: FileScanConfig, - has_header: bool, - delimiter: u8, - quote: u8, - terminator: Option, - escape: Option, - comment: Option, - newlines_in_values: bool, - file_compression_type: FileCompressionType, - ) -> Self { - CsvExecBuilder::new(base_config) - .with_has_header(has_header) - .with_delimeter(delimiter) - .with_quote(quote) - .with_terminator(terminator) - .with_escape(escape) - .with_comment(comment) - .with_newlines_in_values(newlines_in_values) - .with_file_compression_type(file_compression_type) - .build() - } - - /// Return a [`CsvExecBuilder`]. - /// - /// See example on [`CsvExec`] and [`CsvExecBuilder`] for specifying CSV table options. - pub fn builder(file_scan_config: FileScanConfig) -> CsvExecBuilder { - CsvExecBuilder::new(file_scan_config) - } - - /// Ref to the base configs - pub fn base_config(&self) -> &FileScanConfig { - &self.base_config - } - - fn file_scan_config(&self) -> FileScanConfig { - self.inner - .data_source() - .as_any() - .downcast_ref::() - .unwrap() - .clone() - } - - fn csv_source(&self) -> CsvSource { - let source = self.file_scan_config(); - source - .file_source() - .as_any() - .downcast_ref::() - .unwrap() - .clone() - } - - /// true if the first line of each file is a header - pub fn has_header(&self) -> bool { - self.csv_source().has_header() - } - - /// Specifies whether newlines in (quoted) values are supported. - /// - /// Parsing newlines in quoted values may be affected by execution behaviour such as - /// parallel file scanning. Setting this to `true` ensures that newlines in values are - /// parsed successfully, which may reduce performance. - /// - /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. - pub fn newlines_in_values(&self) -> bool { - let source = self.file_scan_config(); - source.newlines_in_values() - } - - fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - orderings: &[LexOrdering], - constraints: Constraints, - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) - .with_constraints(constraints); - - PlanProperties::new( - eq_properties, - Self::output_partitioning_helper(file_scan_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - fn with_file_groups(mut self, file_groups: Vec>) -> Self { - self.base_config.file_groups = file_groups.clone(); - let mut file_source = self.file_scan_config(); - file_source = file_source.with_file_groups(file_groups); - self.inner = self.inner.with_data_source(Arc::new(file_source)); - self - } -} - -#[allow(unused, deprecated)] -impl DisplayAs for CsvExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - self.inner.fmt_as(t, f) - } -} - -#[allow(unused, deprecated)] -impl ExecutionPlan for CsvExec { - fn name(&self) -> &'static str { - "CsvExec" - } - - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - self.inner.properties() - } - - fn children(&self) -> Vec<&Arc> { - // this is a leaf node and has no children - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - /// Redistribute files across partitions according to their size - /// See comments on `FileGroupPartitioner` for more detail. - /// - /// Return `None` if can't get repartitioned (empty, compressed file, or `newlines_in_values` set). - fn repartitioned( - &self, - target_partitions: usize, - config: &ConfigOptions, - ) -> Result>> { - self.inner.repartitioned(target_partitions, config) - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> Result { - self.inner.execute(partition, context) - } - - fn statistics(&self) -> Result { - self.inner.statistics() - } - - fn metrics(&self) -> Option { - self.inner.metrics() - } - - fn fetch(&self) -> Option { - self.inner.fetch() - } - - fn with_fetch(&self, limit: Option) -> Option> { - self.inner.with_fetch(limit) - } - - fn try_swapping_with_projection( - &self, - projection: &ProjectionExec, - ) -> Result>> { - self.inner.try_swapping_with_projection(projection) - } -} - -/// A Config for [`CsvOpener`] -/// -/// # Example: create a `DataSourceExec` for CSV -/// ``` -/// # use std::sync::Arc; -/// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::{ -/// # physical_plan::FileScanConfig, -/// # listing::PartitionedFile, -/// # }; -/// # use datafusion::datasource::physical_plan::CsvSource; -/// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion::datasource::source::DataSourceExec; -/// -/// # let object_store_url = ObjectStoreUrl::local_filesystem(); -/// # let file_schema = Arc::new(Schema::empty()); -/// -/// let source = Arc::new(CsvSource::new( -/// true, -/// b',', -/// b'"', -/// ) -/// .with_terminator(Some(b'#') -/// )); -/// // Create a DataSourceExec for reading the first 100MB of `file1.csv` -/// let file_scan_config = FileScanConfig::new(object_store_url, file_schema, source) -/// .with_file(PartitionedFile::new("file1.csv", 100*1024*1024)) -/// .with_newlines_in_values(true); // The file contains newlines in values; -/// let exec = file_scan_config.build(); -/// ``` -#[derive(Debug, Clone, Default)] -pub struct CsvSource { - batch_size: Option, - file_schema: Option, - file_projection: Option>, - pub(crate) has_header: bool, - delimiter: u8, - quote: u8, - terminator: Option, - escape: Option, - comment: Option, - metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, -} - -impl CsvSource { - /// Returns a [`CsvSource`] - pub fn new(has_header: bool, delimiter: u8, quote: u8) -> Self { - Self { - has_header, - delimiter, - quote, - ..Self::default() - } - } - - /// true if the first line of each file is a header - pub fn has_header(&self) -> bool { - self.has_header - } - /// A column delimiter - pub fn delimiter(&self) -> u8 { - self.delimiter - } - - /// The quote character - pub fn quote(&self) -> u8 { - self.quote - } - - /// The line terminator - pub fn terminator(&self) -> Option { - self.terminator - } - - /// Lines beginning with this byte are ignored. - pub fn comment(&self) -> Option { - self.comment - } - - /// The escape character - pub fn escape(&self) -> Option { - self.escape - } - - /// Initialize a CsvSource with escape - pub fn with_escape(&self, escape: Option) -> Self { - let mut conf = self.clone(); - conf.escape = escape; - conf - } - - /// Initialize a CsvSource with terminator - pub fn with_terminator(&self, terminator: Option) -> Self { - let mut conf = self.clone(); - conf.terminator = terminator; - conf - } - - /// Initialize a CsvSource with comment - pub fn with_comment(&self, comment: Option) -> Self { - let mut conf = self.clone(); - conf.comment = comment; - conf - } -} - -impl CsvSource { - fn open(&self, reader: R) -> Result> { - Ok(self.builder().build(reader)?) - } - - fn builder(&self) -> csv::ReaderBuilder { - let mut builder = csv::ReaderBuilder::new(Arc::clone( - self.file_schema - .as_ref() - .expect("Schema must be set before initializing builder"), - )) - .with_delimiter(self.delimiter) - .with_batch_size( - self.batch_size - .expect("Batch size must be set before initializing builder"), - ) - .with_header(self.has_header) - .with_quote(self.quote); - if let Some(terminator) = self.terminator { - builder = builder.with_terminator(terminator); - } - if let Some(proj) = &self.file_projection { - builder = builder.with_projection(proj.clone()); - } - if let Some(escape) = self.escape { - builder = builder.with_escape(escape) - } - if let Some(comment) = self.comment { - builder = builder.with_comment(comment); - } - - builder - } -} - -/// A [`FileOpener`] that opens a CSV file and yields a [`FileOpenFuture`] -pub struct CsvOpener { - config: Arc, - file_compression_type: FileCompressionType, - object_store: Arc, -} - -impl CsvOpener { - /// Returns a [`CsvOpener`] - pub fn new( - config: Arc, - file_compression_type: FileCompressionType, - object_store: Arc, - ) -> Self { - Self { - config, - file_compression_type, - object_store, - } - } -} - -impl FileSource for CsvSource { - fn create_file_opener( - &self, - object_store: Arc, - base_config: &FileScanConfig, - _partition: usize, - ) -> Arc { - Arc::new(CsvOpener { - config: Arc::new(self.clone()), - file_compression_type: base_config.file_compression_type, - object_store, - }) - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn with_batch_size(&self, batch_size: usize) -> Arc { - let mut conf = self.clone(); - conf.batch_size = Some(batch_size); - Arc::new(conf) - } - - fn with_schema(&self, schema: SchemaRef) -> Arc { - let mut conf = self.clone(); - conf.file_schema = Some(schema); - Arc::new(conf) - } - - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, config: &FileScanConfig) -> Arc { - let mut conf = self.clone(); - conf.file_projection = config.file_column_projection_indices(); - Arc::new(conf) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { - &self.metrics - } - fn statistics(&self) -> Result { - let statistics = &self.projected_statistics; - Ok(statistics - .clone() - .expect("projected_statistics must be set")) - } - fn file_type(&self) -> &str { - "csv" - } - fn fmt_extra(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, ", has_header={}", self.has_header) - } -} - -impl FileOpener for CsvOpener { - /// Open a partitioned CSV file. - /// - /// If `file_meta.range` is `None`, the entire file is opened. - /// If `file_meta.range` is `Some(FileRange {start, end})`, this signifies that the partition - /// corresponds to the byte range [start, end) within the file. - /// - /// Note: `start` or `end` might be in the middle of some lines. In such cases, the following rules - /// are applied to determine which lines to read: - /// 1. The first line of the partition is the line in which the index of the first character >= `start`. - /// 2. The last line of the partition is the line in which the byte at position `end - 1` resides. - /// - /// Examples: - /// Consider the following partitions enclosed by braces `{}`: - /// - /// {A,1,2,3,4,5,6,7,8,9\n - /// A,1,2,3,4,5,6,7,8,9\n} - /// A,1,2,3,4,5,6,7,8,9\n - /// The lines read would be: [0, 1] - /// - /// A,{1,2,3,4,5,6,7,8,9\n - /// A,1,2,3,4,5,6,7,8,9\n - /// A},1,2,3,4,5,6,7,8,9\n - /// The lines read would be: [1, 2] - fn open(&self, file_meta: FileMeta) -> Result { - // `self.config.has_header` controls whether to skip reading the 1st line header - // If the .csv file is read in parallel and this `CsvOpener` is only reading some middle - // partition, then don't skip first line - let mut csv_has_header = self.config.has_header; - if let Some(FileRange { start, .. }) = file_meta.range { - if start != 0 { - csv_has_header = false; - } - } - - let config = CsvSource { - has_header: csv_has_header, - ..(*self.config).clone() - }; - - let file_compression_type = self.file_compression_type.to_owned(); - - if file_meta.range.is_some() { - assert!( - !file_compression_type.is_compressed(), - "Reading compressed .csv in parallel is not supported" - ); - } - - let store = Arc::clone(&self.object_store); - let terminator = self.config.terminator; - - Ok(Box::pin(async move { - // Current partition contains bytes [start_byte, end_byte) (might contain incomplete lines at boundaries) - - let calculated_range = - calculate_range(&file_meta, &store, terminator).await?; - - let range = match calculated_range { - RangeCalculation::Range(None) => None, - RangeCalculation::Range(Some(range)) => Some(range.into()), - RangeCalculation::TerminateEarly => { - return Ok( - futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() - ) - } - }; - - let options = GetOptions { - range, - ..Default::default() - }; - - let result = store.get_opts(file_meta.location(), options).await?; - - match result.payload { - GetResultPayload::File(mut file, _) => { - let is_whole_file_scanned = file_meta.range.is_none(); - let decoder = if is_whole_file_scanned { - // Don't seek if no range as breaks FIFO files - file_compression_type.convert_read(file)? - } else { - file.seek(SeekFrom::Start(result.range.start as _))?; - file_compression_type.convert_read( - file.take((result.range.end - result.range.start) as u64), - )? - }; - - Ok(futures::stream::iter(config.open(decoder)?).boxed()) - } - GetResultPayload::Stream(s) => { - let decoder = config.builder().build_decoder(); - let s = s.map_err(DataFusionError::from); - let input = file_compression_type.convert_stream(s.boxed())?.fuse(); - - Ok(deserialize_stream( - input, - DecoderDeserializer::from(decoder), - )) - } - } - })) - } -} - -pub async fn plan_to_csv( - task_ctx: Arc, - plan: Arc, - path: impl AsRef, -) -> Result<()> { - let path = path.as_ref(); - let parsed = ListingTableUrl::parse(path)?; - let object_store_url = parsed.object_store(); - let store = task_ctx.runtime_env().object_store(&object_store_url)?; - let mut join_set = JoinSet::new(); - for i in 0..plan.output_partitioning().partition_count() { - let storeref = Arc::clone(&store); - let plan: Arc = Arc::clone(&plan); - let filename = format!("{}/part-{i}.csv", parsed.prefix()); - let file = object_store::path::Path::parse(filename)?; - - let mut stream = plan.execute(i, Arc::clone(&task_ctx))?; - join_set.spawn(async move { - let mut buf_writer = BufWriter::new(storeref, file.clone()); - let mut buffer = Vec::with_capacity(1024); - //only write headers on first iteration - let mut write_headers = true; - while let Some(batch) = stream.next().await.transpose()? { - let mut writer = csv::WriterBuilder::new() - .with_header(write_headers) - .build(buffer); - writer.write(&batch)?; - buffer = writer.into_inner(); - buf_writer.write_all(&buffer).await?; - buffer.clear(); - //prevent writing headers more than once - write_headers = false; - } - buf_writer.shutdown().await.map_err(DataFusionError::from) - }); - } - - while let Some(result) = join_set.join_next().await { - match result { - Ok(res) => res?, // propagate DataFusion error - Err(e) => { - if e.is_panic() { - std::panic::resume_unwind(e.into_panic()); - } else { - unreachable!(); - } - } - } - } - - Ok(()) -} +pub use datafusion_datasource_csv::source::*; #[cfg(test)] mod tests { + + use std::collections::HashMap; use std::fs::{self, File}; use std::io::Write; + use std::sync::Arc; - use super::*; - use crate::dataframe::DataFrameWriteOptions; - use crate::datasource::file_format::csv::CsvFormat; - use crate::prelude::*; - use crate::test::{partitioned_csv_config, partitioned_file_groups}; - use crate::{scalar::ScalarValue, test_util::aggr_test_schema}; + use datafusion_datasource_csv::CsvFormat; + use object_store::ObjectStore; - use arrow::datatypes::*; - use bytes::Bytes; + use crate::prelude::CsvReadOptions; + use crate::prelude::SessionContext; + use crate::test::partitioned_file_groups; use datafusion_common::test_util::arrow_test_data; + use datafusion_common::{assert_batches_eq, Result}; + use datafusion_execution::config::SessionConfig; use datafusion_physical_plan::metrics::MetricsSet; + use datafusion_physical_plan::ExecutionPlan; + #[cfg(feature = "compression")] + use datafusion_datasource::file_compression_type::FileCompressionType; + use datafusion_datasource_csv::partitioned_csv_config; + use datafusion_datasource_csv::source::CsvSource; + use futures::{StreamExt, TryStreamExt}; + + use arrow::datatypes::*; + use bytes::Bytes; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; use tempfile::TempDir; use url::Url; + fn aggr_test_schema() -> SchemaRef { + let mut f1 = Field::new("c1", DataType::Utf8, false); + f1.set_metadata(HashMap::from_iter(vec![("testing".into(), "test".into())])); + let schema = Schema::new(vec![ + f1, + Field::new("c2", DataType::UInt32, false), + Field::new("c3", DataType::Int8, false), + Field::new("c4", DataType::Int16, false), + Field::new("c5", DataType::Int32, false), + Field::new("c6", DataType::Int64, false), + Field::new("c7", DataType::UInt8, false), + Field::new("c8", DataType::UInt16, false), + Field::new("c9", DataType::UInt32, false), + Field::new("c10", DataType::UInt64, false), + Field::new("c11", DataType::Float32, false), + Field::new("c12", DataType::Float64, false), + Field::new("c13", DataType::Utf8, false), + ]); + + Arc::new(schema) + } + #[rstest( file_compression_type, case(FileCompressionType::UNCOMPRESSED), @@ -816,8 +90,6 @@ mod tests { async fn csv_exec_with_projection( file_compression_type: FileCompressionType, ) -> Result<()> { - use crate::datasource::file_format::csv::CsvFormat; - let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); let file_schema = aggr_test_schema(); @@ -863,7 +135,7 @@ mod tests { "+----+-----+------------+", ]; - crate::assert_batches_eq!(expected, &[batch.slice(0, 5)]); + assert_batches_eq!(expected, &[batch.slice(0, 5)]); Ok(()) } @@ -880,8 +152,6 @@ mod tests { async fn csv_exec_with_mixed_order_projection( file_compression_type: FileCompressionType, ) -> Result<()> { - use crate::datasource::file_format::csv::CsvFormat; - let cfg = SessionConfig::new().set_str("datafusion.catalog.has_header", "true"); let session_ctx = SessionContext::new_with_config(cfg); let task_ctx = session_ctx.task_ctx(); @@ -926,7 +196,7 @@ mod tests { "+------------+----+-----+", ]; - crate::assert_batches_eq!(expected, &[batch.slice(0, 5)]); + assert_batches_eq!(expected, &[batch.slice(0, 5)]); Ok(()) } @@ -943,7 +213,7 @@ mod tests { async fn csv_exec_with_limit( file_compression_type: FileCompressionType, ) -> Result<()> { - use crate::datasource::file_format::csv::CsvFormat; + use futures::StreamExt; let cfg = SessionConfig::new().set_str("datafusion.catalog.has_header", "true"); let session_ctx = SessionContext::new_with_config(cfg); @@ -986,7 +256,7 @@ mod tests { "| b | 5 | -82 | 22080 | 1824882165 | 7373730676428214987 | 208 | 34331 | 3342719438 | 3330177516592499461 | 0.82634634 | 0.40975383525297016 | Ig1QcuKsjHXkproePdERo2w0mYzIqd |", "+----+----+-----+--------+------------+----------------------+-----+-------+------------+----------------------+-------------+---------------------+--------------------------------+"]; - crate::assert_batches_eq!(expected, &[batch]); + assert_batches_eq!(expected, &[batch]); Ok(()) } @@ -1004,8 +274,6 @@ mod tests { async fn csv_exec_with_missing_column( file_compression_type: FileCompressionType, ) -> Result<()> { - use crate::datasource::file_format::csv::CsvFormat; - let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); let file_schema = aggr_test_schema_with_missing_col(); @@ -1054,7 +322,7 @@ mod tests { async fn csv_exec_with_partition( file_compression_type: FileCompressionType, ) -> Result<()> { - use crate::datasource::file_format::csv::CsvFormat; + use datafusion_common::ScalarValue; let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); @@ -1109,7 +377,7 @@ mod tests { "| b | 2021-10-26 |", "+----+------------+", ]; - crate::assert_batches_eq!(expected, &[batch.slice(0, 5)]); + assert_batches_eq!(expected, &[batch.slice(0, 5)]); let metrics = csv.metrics().expect("doesn't found metrics"); let time_elapsed_processing = get_value(&metrics, "time_elapsed_processing"); @@ -1242,7 +510,7 @@ mod tests { "+---+---+", ]; - crate::assert_batches_eq!(expected, &result); + assert_batches_eq!(expected, &result); } #[tokio::test] @@ -1273,7 +541,7 @@ mod tests { "+---+---+", ]; - crate::assert_batches_eq!(expected, &result); + assert_batches_eq!(expected, &result); let e = session_ctx .read_csv("memory:///", CsvReadOptions::new().terminator(Some(b'\n'))) @@ -1313,7 +581,7 @@ mod tests { "| id3 | value3 |", "+------+--------+", ]; - crate::assert_batches_eq!(expected, &df); + assert_batches_eq!(expected, &df); Ok(()) } @@ -1342,7 +610,7 @@ mod tests { "| value | end |", "+-------+-----------------------------+", ]; - crate::assert_batches_eq!(expected, &df); + assert_batches_eq!(expected, &df); Ok(()) } @@ -1362,7 +630,11 @@ mod tests { let out_dir_url = "file://local/out"; let e = df - .write_csv(out_dir_url, DataFrameWriteOptions::new(), None) + .write_csv( + out_dir_url, + crate::dataframe::DataFrameWriteOptions::new(), + None, + ) .await .expect_err("should fail because input file does not match inferred schema"); assert_eq!(e.strip_backtrace(), "Arrow error: Parser error: Error while parsing value d for column 0 at line 4"); @@ -1400,8 +672,12 @@ mod tests { let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out/"; let out_dir_url = "file://local/out/"; let df = ctx.sql("SELECT c1, c2 FROM test").await?; - df.write_csv(out_dir_url, DataFrameWriteOptions::new(), None) - .await?; + df.write_csv( + out_dir_url, + crate::dataframe::DataFrameWriteOptions::new(), + None, + ) + .await?; // create a new context and verify that the results were saved to a partitioned csv file let ctx = SessionContext::new_with_config( diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index c9a22add2afc..9bab75fc88c3 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -15,453 +15,43 @@ // specific language governing permissions and limitations // under the License. -//! Execution plan for reading line-delimited JSON files - -use std::any::Any; -use std::io::{BufReader, Read, Seek, SeekFrom}; -use std::sync::Arc; -use std::task::Poll; - -use super::{calculate_range, RangeCalculation}; - -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; -use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; -use crate::datasource::physical_plan::FileMeta; -use crate::datasource::physical_plan::{FileOpenFuture, FileOpener}; -use crate::error::{DataFusionError, Result}; -use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; - -use arrow::json::ReaderBuilder; -use arrow::{datatypes::SchemaRef, json}; -use datafusion_common::{Constraints, Statistics}; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_datasource::source::DataSourceExec; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; - -use futures::{StreamExt, TryStreamExt}; -use object_store::buffered::BufWriter; -use object_store::{GetOptions, GetResultPayload, ObjectStore}; -use tokio::io::AsyncWriteExt; -use tokio::task::JoinSet; - -/// Execution plan for scanning NdJson data source -#[derive(Debug, Clone)] -#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] -pub struct NdJsonExec { - inner: DataSourceExec, - base_config: FileScanConfig, - file_compression_type: FileCompressionType, -} - -#[allow(unused, deprecated)] -impl NdJsonExec { - /// Create a new JSON reader execution plan provided base configurations - pub fn new( - base_config: FileScanConfig, - file_compression_type: FileCompressionType, - ) -> Self { - let ( - projected_schema, - projected_constraints, - projected_statistics, - projected_output_ordering, - ) = base_config.project(); - let cache = Self::compute_properties( - projected_schema, - &projected_output_ordering, - projected_constraints, - &base_config, - ); - - let json = JsonSource::default(); - let base_config = base_config - .with_file_compression_type(file_compression_type) - .with_source(Arc::new(json)); - - Self { - inner: DataSourceExec::new(Arc::new(base_config.clone())), - file_compression_type: base_config.file_compression_type, - base_config, - } - } - - /// Ref to the base configs - pub fn base_config(&self) -> &FileScanConfig { - &self.base_config - } - - /// Ref to file compression type - pub fn file_compression_type(&self) -> &FileCompressionType { - &self.file_compression_type - } - - fn file_scan_config(&self) -> FileScanConfig { - self.inner - .data_source() - .as_any() - .downcast_ref::() - .unwrap() - .clone() - } - - fn json_source(&self) -> JsonSource { - let source = self.file_scan_config(); - source - .file_source() - .as_any() - .downcast_ref::() - .unwrap() - .clone() - } - - fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - orderings: &[LexOrdering], - constraints: Constraints, - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) - .with_constraints(constraints); - - PlanProperties::new( - eq_properties, - Self::output_partitioning_helper(file_scan_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - fn with_file_groups(mut self, file_groups: Vec>) -> Self { - self.base_config.file_groups = file_groups.clone(); - let mut file_source = self.file_scan_config(); - file_source = file_source.with_file_groups(file_groups); - self.inner = self.inner.with_data_source(Arc::new(file_source)); - self - } -} - -#[allow(unused, deprecated)] -impl DisplayAs for NdJsonExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - self.inner.fmt_as(t, f) - } -} - -#[allow(unused, deprecated)] -impl ExecutionPlan for NdJsonExec { - fn name(&self) -> &'static str { - "NdJsonExec" - } - - fn as_any(&self) -> &dyn Any { - self - } - fn properties(&self) -> &PlanProperties { - self.inner.properties() - } - - fn children(&self) -> Vec<&Arc> { - Vec::new() - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - fn repartitioned( - &self, - target_partitions: usize, - config: &datafusion_common::config::ConfigOptions, - ) -> Result>> { - self.inner.repartitioned(target_partitions, config) - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> Result { - self.inner.execute(partition, context) - } - - fn statistics(&self) -> Result { - self.inner.statistics() - } - - fn metrics(&self) -> Option { - self.inner.metrics() - } - - fn fetch(&self) -> Option { - self.inner.fetch() - } - - fn with_fetch(&self, limit: Option) -> Option> { - self.inner.with_fetch(limit) - } -} - -/// A [`FileOpener`] that opens a JSON file and yields a [`FileOpenFuture`] -pub struct JsonOpener { - batch_size: usize, - projected_schema: SchemaRef, - file_compression_type: FileCompressionType, - object_store: Arc, -} - -impl JsonOpener { - /// Returns a [`JsonOpener`] - pub fn new( - batch_size: usize, - projected_schema: SchemaRef, - file_compression_type: FileCompressionType, - object_store: Arc, - ) -> Self { - Self { - batch_size, - projected_schema, - file_compression_type, - object_store, - } - } -} - -/// JsonSource holds the extra configuration that is necessary for [`JsonOpener`] -#[derive(Clone, Default)] -pub struct JsonSource { - batch_size: Option, - metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, -} - -impl JsonSource { - /// Initialize a JsonSource with default values - pub fn new() -> Self { - Self::default() - } -} - -impl FileSource for JsonSource { - fn create_file_opener( - &self, - object_store: Arc, - base_config: &FileScanConfig, - _partition: usize, - ) -> Arc { - Arc::new(JsonOpener { - batch_size: self - .batch_size - .expect("Batch size must set before creating opener"), - projected_schema: base_config.projected_file_schema(), - file_compression_type: base_config.file_compression_type, - object_store, - }) - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn with_batch_size(&self, batch_size: usize) -> Arc { - let mut conf = self.clone(); - conf.batch_size = Some(batch_size); - Arc::new(conf) - } - - fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(Self { ..self.clone() }) - } - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, _config: &FileScanConfig) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { - &self.metrics - } - - fn statistics(&self) -> Result { - let statistics = &self.projected_statistics; - Ok(statistics - .clone() - .expect("projected_statistics must be set to call")) - } - - fn file_type(&self) -> &str { - "json" - } -} - -impl FileOpener for JsonOpener { - /// Open a partitioned NDJSON file. - /// - /// If `file_meta.range` is `None`, the entire file is opened. - /// Else `file_meta.range` is `Some(FileRange{start, end})`, which corresponds to the byte range [start, end) within the file. - /// - /// Note: `start` or `end` might be in the middle of some lines. In such cases, the following rules - /// are applied to determine which lines to read: - /// 1. The first line of the partition is the line in which the index of the first character >= `start`. - /// 2. The last line of the partition is the line in which the byte at position `end - 1` resides. - /// - /// See [`CsvOpener`](super::CsvOpener) for an example. - fn open(&self, file_meta: FileMeta) -> Result { - let store = Arc::clone(&self.object_store); - let schema = Arc::clone(&self.projected_schema); - let batch_size = self.batch_size; - let file_compression_type = self.file_compression_type.to_owned(); - - Ok(Box::pin(async move { - let calculated_range = calculate_range(&file_meta, &store, None).await?; - - let range = match calculated_range { - RangeCalculation::Range(None) => None, - RangeCalculation::Range(Some(range)) => Some(range.into()), - RangeCalculation::TerminateEarly => { - return Ok( - futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() - ) - } - }; +//! Reexports the [`datafusion_datasource_json::source`] module, containing JSON based [`FileSource`]. +//! +//! [`FileSource`]: datafusion_datasource::file::FileSource - let options = GetOptions { - range, - ..Default::default() - }; - - let result = store.get_opts(file_meta.location(), options).await?; - - match result.payload { - GetResultPayload::File(mut file, _) => { - let bytes = match file_meta.range { - None => file_compression_type.convert_read(file)?, - Some(_) => { - file.seek(SeekFrom::Start(result.range.start as _))?; - let limit = result.range.end - result.range.start; - file_compression_type.convert_read(file.take(limit as u64))? - } - }; - - let reader = ReaderBuilder::new(schema) - .with_batch_size(batch_size) - .build(BufReader::new(bytes))?; - - Ok(futures::stream::iter(reader).boxed()) - } - GetResultPayload::Stream(s) => { - let s = s.map_err(DataFusionError::from); - - let decoder = ReaderBuilder::new(schema) - .with_batch_size(batch_size) - .build_decoder()?; - let input = file_compression_type.convert_stream(s.boxed())?.fuse(); - - Ok(deserialize_stream( - input, - DecoderDeserializer::from(decoder), - )) - } - } - })) - } -} - -pub async fn plan_to_json( - task_ctx: Arc, - plan: Arc, - path: impl AsRef, -) -> Result<()> { - let path = path.as_ref(); - let parsed = ListingTableUrl::parse(path)?; - let object_store_url = parsed.object_store(); - let store = task_ctx.runtime_env().object_store(&object_store_url)?; - let mut join_set = JoinSet::new(); - for i in 0..plan.output_partitioning().partition_count() { - let storeref = Arc::clone(&store); - let plan: Arc = Arc::clone(&plan); - let filename = format!("{}/part-{i}.json", parsed.prefix()); - let file = object_store::path::Path::parse(filename)?; - - let mut stream = plan.execute(i, Arc::clone(&task_ctx))?; - join_set.spawn(async move { - let mut buf_writer = BufWriter::new(storeref, file.clone()); - - let mut buffer = Vec::with_capacity(1024); - while let Some(batch) = stream.next().await.transpose()? { - let mut writer = json::LineDelimitedWriter::new(buffer); - writer.write(&batch)?; - buffer = writer.into_inner(); - buf_writer.write_all(&buffer).await?; - buffer.clear(); - } - - buf_writer.shutdown().await.map_err(DataFusionError::from) - }); - } - - while let Some(result) = join_set.join_next().await { - match result { - Ok(res) => res?, // propagate DataFusion error - Err(e) => { - if e.is_panic() { - std::panic::resume_unwind(e.into_panic()); - } else { - unreachable!(); - } - } - } - } - - Ok(()) -} +#[allow(deprecated)] +pub use datafusion_datasource_json::source::*; #[cfg(test)] mod tests { + + use super::*; + use std::fs; use std::path::Path; + use std::sync::Arc; - use super::*; use crate::dataframe::DataFrameWriteOptions; - use crate::datasource::file_format::{json::JsonFormat, FileFormat}; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::execution::context::SessionState; - use crate::prelude::{ - CsvReadOptions, NdJsonReadOptions, SessionConfig, SessionContext, - }; + use crate::execution::SessionState; + use crate::prelude::{CsvReadOptions, NdJsonReadOptions, SessionContext}; use crate::test::partitioned_file_groups; - use crate::{assert_batches_eq, assert_batches_sorted_eq}; + use datafusion_common::cast::{as_int32_array, as_int64_array, as_string_array}; + use datafusion_common::Result; + use datafusion_common::{assert_batches_eq, assert_batches_sorted_eq}; + use datafusion_datasource::file_compression_type::FileCompressionType; + use datafusion_datasource::file_format::FileFormat; + use datafusion_datasource::PartitionedFile; + use datafusion_datasource_json::JsonFormat; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_physical_plan::ExecutionPlan; use arrow::array::Array; + use arrow::datatypes::SchemaRef; use arrow::datatypes::{Field, SchemaBuilder}; - use datafusion_common::cast::{as_int32_array, as_int64_array, as_string_array}; - use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; + use object_store::ObjectStore; use rstest::*; use tempfile::TempDir; use url::Url; @@ -577,6 +167,8 @@ mod tests { let state = session_ctx.state(); let task_ctx = session_ctx.task_ctx(); use arrow::datatypes::DataType; + use datafusion_datasource::file_scan_config::FileScanConfig; + use futures::StreamExt; let tmp_dir = TempDir::new()?; let (object_store_url, file_groups, file_schema) = @@ -638,10 +230,13 @@ mod tests { async fn nd_json_exec_file_with_missing_column( file_compression_type: FileCompressionType, ) -> Result<()> { + use arrow::datatypes::DataType; + use datafusion_datasource::file_scan_config::FileScanConfig; + use futures::StreamExt; + let session_ctx = SessionContext::new(); let state = session_ctx.state(); let task_ctx = session_ctx.task_ctx(); - use arrow::datatypes::DataType; let tmp_dir = TempDir::new()?; let (object_store_url, file_groups, actual_schema) = @@ -686,6 +281,9 @@ mod tests { async fn nd_json_exec_file_projection( file_compression_type: FileCompressionType, ) -> Result<()> { + use datafusion_datasource::file_scan_config::FileScanConfig; + use futures::StreamExt; + let session_ctx = SessionContext::new(); let state = session_ctx.state(); let task_ctx = session_ctx.task_ctx(); @@ -731,6 +329,9 @@ mod tests { async fn nd_json_exec_file_mixed_order_projection( file_compression_type: FileCompressionType, ) -> Result<()> { + use datafusion_datasource::file_scan_config::FileScanConfig; + use futures::StreamExt; + let session_ctx = SessionContext::new(); let state = session_ctx.state(); let task_ctx = session_ctx.task_ctx(); @@ -936,6 +537,8 @@ mod tests { async fn test_json_with_repartitioning( file_compression_type: FileCompressionType, ) -> Result<()> { + use datafusion_execution::config::SessionConfig; + let config = SessionConfig::new() .with_repartition_file_scans(true) .with_repartition_file_min_size(0) diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 42f6912afec0..cae04e5ee6b8 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -18,32 +18,39 @@ //! Execution plans that read file formats mod arrow_file; -mod avro; -mod csv; -mod json; +pub mod csv; +pub mod json; + #[cfg(feature = "parquet")] pub mod parquet; -pub(crate) use self::csv::plan_to_csv; -pub(crate) use self::json::plan_to_json; +#[cfg(feature = "avro")] +pub mod avro; + +#[allow(deprecated)] +#[cfg(feature = "avro")] +pub use avro::{AvroExec, AvroSource}; + #[cfg(feature = "parquet")] -pub use self::parquet::source::ParquetSource; +pub use datafusion_datasource_parquet::source::ParquetSource; #[cfg(feature = "parquet")] #[allow(deprecated)] -pub use self::parquet::{ +pub use datafusion_datasource_parquet::{ ParquetExec, ParquetExecBuilder, ParquetFileMetrics, ParquetFileReaderFactory, }; -use crate::datasource::listing::FileRange; -use crate::error::Result; -use crate::physical_plan::DisplayAs; + #[allow(deprecated)] pub use arrow_file::ArrowExec; pub use arrow_file::ArrowSource; + #[allow(deprecated)] -pub use avro::AvroExec; -pub use avro::AvroSource; +pub use json::NdJsonExec; + +pub use json::{JsonOpener, JsonSource}; + #[allow(deprecated)] pub use csv::{CsvExec, CsvExecBuilder}; + pub use csv::{CsvOpener, CsvSource}; pub use datafusion_datasource::file::FileSource; pub use datafusion_datasource::file_groups::FileGroupPartitioner; @@ -56,121 +63,10 @@ pub use datafusion_datasource::file_sink_config::*; pub use datafusion_datasource::file_stream::{ FileOpenFuture, FileOpener, FileStream, OnError, }; -use futures::StreamExt; -#[allow(deprecated)] -pub use json::NdJsonExec; -pub use json::{JsonOpener, JsonSource}; - -use object_store::{path::Path, GetOptions, GetRange, ObjectStore}; -use std::{ops::Range, sync::Arc}; - -/// Represents the possible outcomes of a range calculation. -/// -/// This enum is used to encapsulate the result of calculating the range of -/// bytes to read from an object (like a file) in an object store. -/// -/// Variants: -/// - `Range(Option>)`: -/// Represents a range of bytes to be read. It contains an `Option` wrapping a -/// `Range`. `None` signifies that the entire object should be read, -/// while `Some(range)` specifies the exact byte range to read. -/// - `TerminateEarly`: -/// Indicates that the range calculation determined no further action is -/// necessary, possibly because the calculated range is empty or invalid. -enum RangeCalculation { - Range(Option>), - TerminateEarly, -} - -/// Calculates an appropriate byte range for reading from an object based on the -/// provided metadata. -/// -/// This asynchronous function examines the `FileMeta` of an object in an object store -/// and determines the range of bytes to be read. The range calculation may adjust -/// the start and end points to align with meaningful data boundaries (like newlines). -/// -/// Returns a `Result` wrapping a `RangeCalculation`, which is either a calculated byte range or an indication to terminate early. -/// -/// Returns an `Error` if any part of the range calculation fails, such as issues in reading from the object store or invalid range boundaries. -async fn calculate_range( - file_meta: &FileMeta, - store: &Arc, - terminator: Option, -) -> Result { - let location = file_meta.location(); - let file_size = file_meta.object_meta.size; - let newline = terminator.unwrap_or(b'\n'); - - match file_meta.range { - None => Ok(RangeCalculation::Range(None)), - Some(FileRange { start, end }) => { - let (start, end) = (start as usize, end as usize); - - let start_delta = if start != 0 { - find_first_newline(store, location, start - 1, file_size, newline).await? - } else { - 0 - }; - - let end_delta = if end != file_size { - find_first_newline(store, location, end - 1, file_size, newline).await? - } else { - 0 - }; - - let range = start + start_delta..end + end_delta; - - if range.start == range.end { - return Ok(RangeCalculation::TerminateEarly); - } - - Ok(RangeCalculation::Range(Some(range))) - } - } -} - -/// Asynchronously finds the position of the first newline character in a specified byte range -/// within an object, such as a file, in an object store. -/// -/// This function scans the contents of the object starting from the specified `start` position -/// up to the `end` position, looking for the first occurrence of a newline character. -/// It returns the position of the first newline relative to the start of the range. -/// -/// Returns a `Result` wrapping a `usize` that represents the position of the first newline character found within the specified range. If no newline is found, it returns the length of the scanned data, effectively indicating the end of the range. -/// -/// The function returns an `Error` if any issues arise while reading from the object store or processing the data stream. -/// -async fn find_first_newline( - object_store: &Arc, - location: &Path, - start: usize, - end: usize, - newline: u8, -) -> Result { - let options = GetOptions { - range: Some(GetRange::Bounded(start..end)), - ..Default::default() - }; - - let result = object_store.get_opts(location, options).await?; - let mut result_stream = result.into_stream(); - - let mut index = 0; - - while let Some(chunk) = result_stream.next().await.transpose()? { - if let Some(position) = chunk.iter().position(|&byte| byte == newline) { - return Ok(index + position); - } - - index += chunk.len(); - } - - Ok(index) -} #[cfg(test)] mod tests { - use super::*; + use std::sync::Arc; use arrow::array::{ cast::AsArray, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs similarity index 72% rename from datafusion/core/src/datasource/physical_plan/parquet/mod.rs rename to datafusion/core/src/datasource/physical_plan/parquet.rs index f677c73cc881..888f3ad9e3b9 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -15,576 +15,57 @@ // specific language governing permissions and limitations // under the License. -//! [`ParquetExec`] FileSource for reading Parquet files - -mod access_plan; -mod metrics; -mod opener; -mod page_filter; -mod reader; -mod row_filter; -mod row_group_filter; -pub mod source; -mod writer; - -use std::any::Any; -use std::fmt::Formatter; -use std::sync::Arc; - -use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::{parquet::source::ParquetSource, DisplayAs}; -use crate::datasource::schema_adapter::SchemaAdapterFactory; -use crate::{ - config::TableParquetOptions, - error::Result, - execution::context::TaskContext, - physical_plan::{ - metrics::MetricsSet, DisplayFormatType, ExecutionPlan, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, - }, -}; - -pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; -use arrow::datatypes::SchemaRef; -use datafusion_common::config::ConfigOptions; -use datafusion_common::Constraints; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_datasource::source::DataSourceExec; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; -use datafusion_physical_optimizer::pruning::PruningPredicate; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -pub use metrics::ParquetFileMetrics; -pub use page_filter::PagePruningAccessPlanFilter; -pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; -pub use row_filter::build_row_filter; -pub use row_filter::can_expr_be_pushed_down_with_schemas; -pub use row_group_filter::RowGroupAccessPlanFilter; -pub use writer::plan_to_parquet; - -use log::debug; - -#[derive(Debug, Clone)] -#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] -/// Deprecated Execution plan replaced with DataSourceExec -pub struct ParquetExec { - inner: DataSourceExec, - base_config: FileScanConfig, - table_parquet_options: TableParquetOptions, - /// Optional predicate for row filtering during parquet scan - predicate: Option>, - /// Optional predicate for pruning row groups (derived from `predicate`) - pruning_predicate: Option>, - /// Optional user defined parquet file reader factory - parquet_file_reader_factory: Option>, - /// Optional user defined schema adapter - schema_adapter_factory: Option>, -} - -#[allow(unused, deprecated)] -impl From for ParquetExecBuilder { - fn from(exec: ParquetExec) -> Self { - exec.into_builder() - } -} - -/// [`ParquetExecBuilder`], deprecated builder for [`ParquetExec`]. -/// -/// ParquetExec is replaced with `DataSourceExec` and it includes `ParquetSource` -/// -/// See example on [`ParquetSource`]. -#[deprecated( - since = "46.0.0", - note = "use DataSourceExec with ParquetSource instead" -)] -#[allow(unused, deprecated)] -pub struct ParquetExecBuilder { - file_scan_config: FileScanConfig, - predicate: Option>, - metadata_size_hint: Option, - table_parquet_options: TableParquetOptions, - parquet_file_reader_factory: Option>, - schema_adapter_factory: Option>, -} - -#[allow(unused, deprecated)] -impl ParquetExecBuilder { - /// Create a new builder to read the provided file scan configuration - pub fn new(file_scan_config: FileScanConfig) -> Self { - Self::new_with_options(file_scan_config, TableParquetOptions::default()) - } - - /// Create a new builder to read the data specified in the file scan - /// configuration with the provided `TableParquetOptions`. - pub fn new_with_options( - file_scan_config: FileScanConfig, - table_parquet_options: TableParquetOptions, - ) -> Self { - Self { - file_scan_config, - predicate: None, - metadata_size_hint: None, - table_parquet_options, - parquet_file_reader_factory: None, - schema_adapter_factory: None, - } - } - - /// Update the list of files groups to read - pub fn with_file_groups(mut self, file_groups: Vec>) -> Self { - self.file_scan_config.file_groups = file_groups; - self - } - - /// Set the filter predicate when reading. - /// - /// See the "Predicate Pushdown" section of the [`ParquetExec`] documentation - /// for more details. - pub fn with_predicate(mut self, predicate: Arc) -> Self { - self.predicate = Some(predicate); - self - } - - /// Set the metadata size hint - /// - /// This value determines how many bytes at the end of the file the default - /// [`ParquetFileReaderFactory`] will request in the initial IO. If this is - /// too small, the ParquetExec will need to make additional IO requests to - /// read the footer. - pub fn with_metadata_size_hint(mut self, metadata_size_hint: usize) -> Self { - self.metadata_size_hint = Some(metadata_size_hint); - self - } - - /// Set the options for controlling how the ParquetExec reads parquet files. - /// - /// See also [`Self::new_with_options`] - pub fn with_table_parquet_options( - mut self, - table_parquet_options: TableParquetOptions, - ) -> Self { - self.table_parquet_options = table_parquet_options; - self - } - - /// Set optional user defined parquet file reader factory. - /// - /// You can use [`ParquetFileReaderFactory`] to more precisely control how - /// data is read from parquet files (e.g. skip re-reading metadata, coalesce - /// I/O operations, etc). - /// - /// The default reader factory reads directly from an [`ObjectStore`] - /// instance using individual I/O operations for the footer and each page. - /// - /// If a custom `ParquetFileReaderFactory` is provided, then data access - /// operations will be routed to this factory instead of [`ObjectStore`]. - /// - /// [`ObjectStore`]: object_store::ObjectStore - pub fn with_parquet_file_reader_factory( - mut self, - parquet_file_reader_factory: Arc, - ) -> Self { - self.parquet_file_reader_factory = Some(parquet_file_reader_factory); - self - } - - /// Set optional schema adapter factory. - /// - /// [`SchemaAdapterFactory`] allows user to specify how fields from the - /// parquet file get mapped to that of the table schema. The default schema - /// adapter uses arrow's cast library to map the parquet fields to the table - /// schema. - pub fn with_schema_adapter_factory( - mut self, - schema_adapter_factory: Arc, - ) -> Self { - self.schema_adapter_factory = Some(schema_adapter_factory); - self - } +//! Reexports the [`datafusion_datasource_parquet`] crate, containing Parquet based [`FileSource`]. +//! +//! [`FileSource`]: datafusion_datasource::file::FileSource - /// Convenience: build an `Arc`d `ParquetExec` from this builder - pub fn build_arc(self) -> Arc { - Arc::new(self.build()) - } - - /// Build a [`ParquetExec`] - #[must_use] - pub fn build(self) -> ParquetExec { - let Self { - file_scan_config, - predicate, - metadata_size_hint, - table_parquet_options, - parquet_file_reader_factory, - schema_adapter_factory, - } = self; - let mut parquet = ParquetSource::new(table_parquet_options); - if let Some(predicate) = predicate.clone() { - parquet = parquet - .with_predicate(Arc::clone(&file_scan_config.file_schema), predicate); - } - if let Some(metadata_size_hint) = metadata_size_hint { - parquet = parquet.with_metadata_size_hint(metadata_size_hint) - } - if let Some(parquet_reader_factory) = parquet_file_reader_factory { - parquet = parquet.with_parquet_file_reader_factory(parquet_reader_factory) - } - if let Some(schema_factory) = schema_adapter_factory { - parquet = parquet.with_schema_adapter_factory(schema_factory); - } - - let base_config = file_scan_config.with_source(Arc::new(parquet.clone())); - debug!("Creating ParquetExec, files: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", - base_config.file_groups, base_config.projection, predicate, base_config.limit); - - ParquetExec { - inner: DataSourceExec::new(Arc::new(base_config.clone())), - base_config, - predicate, - pruning_predicate: parquet.pruning_predicate, - schema_adapter_factory: parquet.schema_adapter_factory, - parquet_file_reader_factory: parquet.parquet_file_reader_factory, - table_parquet_options: parquet.table_parquet_options, - } - } -} - -#[allow(unused, deprecated)] -impl ParquetExec { - /// Create a new Parquet reader execution plan provided file list and schema. - pub fn new( - base_config: FileScanConfig, - predicate: Option>, - metadata_size_hint: Option, - table_parquet_options: TableParquetOptions, - ) -> Self { - let mut builder = - ParquetExecBuilder::new_with_options(base_config, table_parquet_options); - if let Some(predicate) = predicate { - builder = builder.with_predicate(predicate); - } - if let Some(metadata_size_hint) = metadata_size_hint { - builder = builder.with_metadata_size_hint(metadata_size_hint); - } - builder.build() - } - /// Return a [`ParquetExecBuilder`]. - /// - /// See example on [`ParquetExec`] and [`ParquetExecBuilder`] for specifying - /// parquet table options. - pub fn builder(file_scan_config: FileScanConfig) -> ParquetExecBuilder { - ParquetExecBuilder::new(file_scan_config) - } - - /// Convert this `ParquetExec` into a builder for modification - pub fn into_builder(self) -> ParquetExecBuilder { - // list out fields so it is clear what is being dropped - // (note the fields which are dropped are re-created as part of calling - // `build` on the builder) - let file_scan_config = self.file_scan_config(); - let parquet = self.parquet_source(); - - ParquetExecBuilder { - file_scan_config, - predicate: parquet.predicate, - metadata_size_hint: parquet.metadata_size_hint, - table_parquet_options: parquet.table_parquet_options, - parquet_file_reader_factory: parquet.parquet_file_reader_factory, - schema_adapter_factory: parquet.schema_adapter_factory, - } - } - fn file_scan_config(&self) -> FileScanConfig { - self.inner - .data_source() - .as_any() - .downcast_ref::() - .unwrap() - .clone() - } - - fn parquet_source(&self) -> ParquetSource { - self.file_scan_config() - .file_source() - .as_any() - .downcast_ref::() - .unwrap() - .clone() - } - - /// [`FileScanConfig`] that controls this scan (such as which files to read) - pub fn base_config(&self) -> &FileScanConfig { - &self.base_config - } - /// Options passed to the parquet reader for this scan - pub fn table_parquet_options(&self) -> &TableParquetOptions { - &self.table_parquet_options - } - /// Optional predicate. - pub fn predicate(&self) -> Option<&Arc> { - self.predicate.as_ref() - } - /// Optional reference to this parquet scan's pruning predicate - pub fn pruning_predicate(&self) -> Option<&Arc> { - self.pruning_predicate.as_ref() - } - /// return the optional file reader factory - pub fn parquet_file_reader_factory( - &self, - ) -> Option<&Arc> { - self.parquet_file_reader_factory.as_ref() - } - /// Optional user defined parquet file reader factory. - pub fn with_parquet_file_reader_factory( - mut self, - parquet_file_reader_factory: Arc, - ) -> Self { - let mut parquet = self.parquet_source(); - parquet.parquet_file_reader_factory = - Some(Arc::clone(&parquet_file_reader_factory)); - let file_source = self.file_scan_config(); - self.inner = self - .inner - .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); - self.parquet_file_reader_factory = Some(parquet_file_reader_factory); - self - } - /// return the optional schema adapter factory - pub fn schema_adapter_factory(&self) -> Option<&Arc> { - self.schema_adapter_factory.as_ref() - } - /// Set optional schema adapter factory. - /// - /// [`SchemaAdapterFactory`] allows user to specify how fields from the - /// parquet file get mapped to that of the table schema. The default schema - /// adapter uses arrow's cast library to map the parquet fields to the table - /// schema. - pub fn with_schema_adapter_factory( - mut self, - schema_adapter_factory: Arc, - ) -> Self { - let mut parquet = self.parquet_source(); - parquet.schema_adapter_factory = Some(Arc::clone(&schema_adapter_factory)); - let file_source = self.file_scan_config(); - self.inner = self - .inner - .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); - self.schema_adapter_factory = Some(schema_adapter_factory); - self - } - /// If true, the predicate will be used during the parquet scan. - /// Defaults to false - /// - /// [`Expr`]: datafusion_expr::Expr - pub fn with_pushdown_filters(mut self, pushdown_filters: bool) -> Self { - let mut parquet = self.parquet_source(); - parquet.table_parquet_options.global.pushdown_filters = pushdown_filters; - let file_source = self.file_scan_config(); - self.inner = self - .inner - .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); - self.table_parquet_options.global.pushdown_filters = pushdown_filters; - self - } - - /// Return the value described in [`Self::with_pushdown_filters`] - fn pushdown_filters(&self) -> bool { - self.parquet_source() - .table_parquet_options - .global - .pushdown_filters - } - /// If true, the `RowFilter` made by `pushdown_filters` may try to - /// minimize the cost of filter evaluation by reordering the - /// predicate [`Expr`]s. If false, the predicates are applied in - /// the same order as specified in the query. Defaults to false. - /// - /// [`Expr`]: datafusion_expr::Expr - pub fn with_reorder_filters(mut self, reorder_filters: bool) -> Self { - let mut parquet = self.parquet_source(); - parquet.table_parquet_options.global.reorder_filters = reorder_filters; - let file_source = self.file_scan_config(); - self.inner = self - .inner - .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); - self.table_parquet_options.global.reorder_filters = reorder_filters; - self - } - /// Return the value described in [`Self::with_reorder_filters`] - fn reorder_filters(&self) -> bool { - self.parquet_source() - .table_parquet_options - .global - .reorder_filters - } - /// If enabled, the reader will read the page index - /// This is used to optimize filter pushdown - /// via `RowSelector` and `RowFilter` by - /// eliminating unnecessary IO and decoding - fn bloom_filter_on_read(&self) -> bool { - self.parquet_source() - .table_parquet_options - .global - .bloom_filter_on_read - } - /// Return the value described in [`ParquetSource::with_enable_page_index`] - fn enable_page_index(&self) -> bool { - self.parquet_source() - .table_parquet_options - .global - .enable_page_index - } - - fn output_partitioning_helper(file_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_config.file_groups.len()) - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - orderings: &[LexOrdering], - constraints: Constraints, - file_config: &FileScanConfig, - ) -> PlanProperties { - PlanProperties::new( - EquivalenceProperties::new_with_orderings(schema, orderings) - .with_constraints(constraints), - Self::output_partitioning_helper(file_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - /// Updates the file groups to read and recalculates the output partitioning - /// - /// Note this function does not update statistics or other properties - /// that depend on the file groups. - fn with_file_groups_and_update_partitioning( - mut self, - file_groups: Vec>, - ) -> Self { - let mut config = self.file_scan_config(); - config.file_groups = file_groups; - self.inner = self.inner.with_data_source(Arc::new(config)); - self - } -} - -#[allow(unused, deprecated)] -impl DisplayAs for ParquetExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - self.inner.fmt_as(t, f) - } -} - -#[allow(unused, deprecated)] -impl ExecutionPlan for ParquetExec { - fn name(&self) -> &'static str { - "ParquetExec" - } - - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - self.inner.properties() - } - - fn children(&self) -> Vec<&Arc> { - // this is a leaf node and has no children - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - /// Redistribute files across partitions according to their size - /// See comments on `FileGroupPartitioner` for more detail. - fn repartitioned( - &self, - target_partitions: usize, - config: &ConfigOptions, - ) -> Result>> { - self.inner.repartitioned(target_partitions, config) - } - - fn execute( - &self, - partition_index: usize, - ctx: Arc, - ) -> Result { - self.inner.execute(partition_index, ctx) - } - fn metrics(&self) -> Option { - self.inner.metrics() - } - fn statistics(&self) -> Result { - self.inner.statistics() - } - fn fetch(&self) -> Option { - self.inner.fetch() - } - - fn with_fetch(&self, limit: Option) -> Option> { - self.inner.with_fetch(limit) - } -} - -fn should_enable_page_index( - enable_page_index: bool, - page_pruning_predicate: &Option>, -) -> bool { - enable_page_index - && page_pruning_predicate.is_some() - && page_pruning_predicate - .as_ref() - .map(|p| p.filter_number() > 0) - .unwrap_or(false) -} +pub use datafusion_datasource_parquet::*; #[cfg(test)] mod tests { // See also `parquet_exec` integration test use std::fs::{self, File}; use std::io::Write; + use std::sync::Arc; use std::sync::Mutex; - use super::*; use crate::dataframe::DataFrameWriteOptions; use crate::datasource::file_format::options::CsvReadOptions; use crate::datasource::file_format::parquet::test_util::store_parquet; use crate::datasource::file_format::test_util::scan_format; - use crate::datasource::listing::{FileRange, ListingOptions, PartitionedFile}; - use crate::datasource::object_store::ObjectStoreUrl; + use crate::datasource::listing::ListingOptions; use crate::execution::context::SessionState; - use crate::physical_plan::displayable; use crate::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; use crate::test::object_store::local_unpartitioned_file; - use crate::{ - assert_batches_sorted_eq, - datasource::file_format::{parquet::ParquetFormat, FileFormat}, - physical_plan::collect, - }; - use arrow::array::{ ArrayRef, Date64Array, Int32Array, Int64Array, Int8Array, StringArray, StructArray, }; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaBuilder}; use arrow::record_batch::RecordBatch; + use arrow_schema::SchemaRef; use bytes::{BufMut, BytesMut}; - use datafusion_common::{assert_contains, ScalarValue}; + use datafusion_common::config::TableParquetOptions; + use datafusion_common::{ + assert_batches_eq, assert_batches_sorted_eq, assert_contains, Result, ScalarValue, + }; + use datafusion_datasource::file_format::FileFormat; + use datafusion_datasource::file_meta::FileMeta; + use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; + + use datafusion_datasource::{FileRange, PartitionedFile}; + use datafusion_datasource_parquet::source::ParquetSource; + use datafusion_datasource_parquet::{ + DefaultParquetFileReaderFactory, ParquetFileReaderFactory, ParquetFormat, + }; + use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{col, lit, when, Expr}; use datafusion_physical_expr::planner::logical2physical; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; + use datafusion_physical_plan::{collect, displayable}; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; - use crate::datasource::physical_plan::parquet::source::ParquetSource; use chrono::{TimeZone, Utc}; use futures::StreamExt; use object_store::local::LocalFileSystem; @@ -743,28 +224,6 @@ mod tests { ) } - #[tokio::test] - async fn write_parquet_results_error_handling() -> Result<()> { - let ctx = SessionContext::new(); - // register a local file system object store for /tmp directory - let tmp_dir = TempDir::new()?; - let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); - let local_url = Url::parse("file://local").unwrap(); - ctx.register_object_store(&local_url, local); - - let options = CsvReadOptions::default() - .schema_infer_max_records(2) - .has_header(true); - let df = ctx.read_csv("tests/data/corrupt.csv", options).await?; - let out_dir_url = "file://local/out"; - let e = df - .write_parquet(out_dir_url, DataFrameWriteOptions::new(), None) - .await - .expect_err("should fail because input file does not match inferred schema"); - assert_eq!(e.strip_backtrace(), "Arrow error: Parser error: Error while parsing value d for column 0 at line 4"); - Ok(()) - } - #[tokio::test] async fn evolved_schema() { let c1: ArrayRef = @@ -1288,7 +747,7 @@ mod tests { #[tokio::test] async fn parquet_exec_with_projection() -> Result<()> { - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let filename = "alltypes_plain.parquet"; let session_ctx = SessionContext::new(); let state = session_ctx.state(); @@ -1376,7 +835,7 @@ mod tests { let session_ctx = SessionContext::new(); let state = session_ctx.state(); - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let filename = format!("{testdata}/alltypes_plain.parquet"); let meta = local_unpartitioned_file(filename); @@ -1409,7 +868,7 @@ mod tests { let object_store_url = ObjectStoreUrl::local_filesystem(); let store = state.runtime_env().object_store(&object_store_url).unwrap(); - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let filename = format!("{testdata}/alltypes_plain.parquet"); let meta = local_unpartitioned_file(filename); @@ -1492,7 +951,7 @@ mod tests { "| 1 | false | 1 | 10 | 26 |", "+----+----------+-------------+-------+-----+", ]; - crate::assert_batches_eq!(expected, &[batch]); + assert_batches_eq!(expected, &[batch]); let batch = results.next().await; assert!(batch.is_none()); @@ -1675,7 +1134,7 @@ mod tests { .await; // should have a pruning predicate - let pruning_predicate = &rt.parquet_source.pruning_predicate; + let pruning_predicate = rt.parquet_source.pruning_predicate(); assert!(pruning_predicate.is_some()); // convert to explain plan form @@ -1716,7 +1175,7 @@ mod tests { .round_trip(vec![batches.clone()]) .await; - let pruning_predicate = &rt0.parquet_source.pruning_predicate; + let pruning_predicate = rt0.parquet_source.pruning_predicate(); assert!(pruning_predicate.is_some()); let display0 = displayable(rt0.parquet_exec.as_ref()) @@ -1758,9 +1217,9 @@ mod tests { .await; // should have a pruning predicate - let pruning_predicate = &rt1.parquet_source.pruning_predicate; + let pruning_predicate = rt1.parquet_source.pruning_predicate(); assert!(pruning_predicate.is_some()); - let pruning_predicate = &rt2.parquet_source.predicate; + let pruning_predicate = rt2.parquet_source.predicate(); assert!(pruning_predicate.is_some()); // convert to explain plan form @@ -1801,14 +1260,14 @@ mod tests { .await; // Should not contain a pruning predicate (since nothing can be pruned) - let pruning_predicate = &rt.parquet_source.pruning_predicate; + let pruning_predicate = rt.parquet_source.pruning_predicate(); assert!( pruning_predicate.is_none(), "Still had pruning predicate: {pruning_predicate:?}" ); // but does still has a pushdown down predicate - let predicate = rt.parquet_source.predicate.as_ref(); + let predicate = rt.parquet_source.predicate(); let filter_phys = logical2physical(&filter, rt.parquet_exec.schema().as_ref()); assert_eq!(predicate.unwrap().to_string(), filter_phys.to_string()); } @@ -1836,7 +1295,7 @@ mod tests { .await; // Should have a pruning predicate - let pruning_predicate = &rt.parquet_source.pruning_predicate; + let pruning_predicate = rt.parquet_source.pruning_predicate(); assert!(pruning_predicate.is_some()); } @@ -1996,7 +1455,7 @@ mod tests { "| {id: 4, name: aaa2} | 2 | test02 |", "+---------------------+----+--------+", ]; - crate::assert_batches_eq!(expected, &batch); + assert_batches_eq!(expected, &batch); Ok(()) } @@ -2026,7 +1485,7 @@ mod tests { "| {id: 4, name: aaa2} | 2 | test02 |", "+---------------------+----+--------+", ]; - crate::assert_batches_eq!(expected, &batch); + assert_batches_eq!(expected, &batch); Ok(()) } @@ -2108,7 +1567,7 @@ mod tests { fn create_reader( &self, partition_index: usize, - file_meta: crate::datasource::physical_plan::FileMeta, + file_meta: FileMeta, metadata_size_hint: Option, metrics: &ExecutionPlanMetricsSet, ) -> Result> diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index f02927619a7d..acc16e0687c9 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -27,13 +27,6 @@ use crate::arrow::datatypes::SchemaRef; use crate::error::Result; use crate::physical_plan::{ColumnStatistics, Statistics}; -#[cfg(feature = "parquet")] -use crate::{ - arrow::datatypes::Schema, - functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}, - physical_plan::Accumulator, -}; - use super::listing::PartitionedFile; /// Get all files as well as the file level summary statistics (no statistic for partition columns). @@ -152,28 +145,6 @@ pub async fn get_statistics_with_limit( Ok((result_files, statistics)) } -// only adding this cfg b/c this is the only feature it's used with currently -#[cfg(feature = "parquet")] -pub(crate) fn create_max_min_accs( - schema: &Schema, -) -> (Vec>, Vec>) { - let max_values: Vec> = schema - .fields() - .iter() - .map(|field| { - MaxAccumulator::try_new(min_max_aggregate_data_type(field.data_type())).ok() - }) - .collect(); - let min_values: Vec> = schema - .fields() - .iter() - .map(|field| { - MinAccumulator::try_new(min_max_aggregate_data_type(field.data_type())).ok() - }) - .collect(); - (max_values, min_values) -} - fn add_row_stats( file_num_rows: Precision, num_rows: Precision, @@ -185,52 +156,6 @@ fn add_row_stats( } } -// only adding this cfg b/c this is the only feature it's used with currently -#[cfg(feature = "parquet")] -pub(crate) fn get_col_stats( - schema: &Schema, - null_counts: Vec>, - max_values: &mut [Option], - min_values: &mut [Option], -) -> Vec { - (0..schema.fields().len()) - .map(|i| { - let max_value = match max_values.get_mut(i).unwrap() { - Some(max_value) => max_value.evaluate().ok(), - None => None, - }; - let min_value = match min_values.get_mut(i).unwrap() { - Some(min_value) => min_value.evaluate().ok(), - None => None, - }; - ColumnStatistics { - null_count: null_counts[i], - max_value: max_value.map(Precision::Exact).unwrap_or(Precision::Absent), - min_value: min_value.map(Precision::Exact).unwrap_or(Precision::Absent), - sum_value: Precision::Absent, - distinct_count: Precision::Absent, - } - }) - .collect() -} - -// Min/max aggregation can take Dictionary encode input but always produces unpacked -// (aka non Dictionary) output. We need to adjust the output data type to reflect this. -// The reason min/max aggregate produces unpacked output because there is only one -// min/max value per group; there is no needs to keep them Dictionary encode -// -// only adding this cfg b/c this is the only feature it's used with currently -#[cfg(feature = "parquet")] -fn min_max_aggregate_data_type( - input_type: &arrow_schema::DataType, -) -> &arrow_schema::DataType { - if let arrow_schema::DataType::Dictionary(_, value_type) = input_type { - value_type.as_ref() - } else { - input_type - } -} - /// If the given value is numerically greater than the original maximum value, /// return the new maximum value with appropriate exactness information. fn set_max_if_greater( diff --git a/datafusion/core/src/execution/context/csv.rs b/datafusion/core/src/execution/context/csv.rs index 9b4c0e3b2964..3e7db1caa20f 100644 --- a/datafusion/core/src/execution/context/csv.rs +++ b/datafusion/core/src/execution/context/csv.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::datasource::physical_plan::plan_to_csv; use datafusion_common::TableReference; +use datafusion_datasource_csv::source::plan_to_csv; use std::sync::Arc; use super::super::options::{CsvReadOptions, ReadOptions}; diff --git a/datafusion/core/src/execution/context/json.rs b/datafusion/core/src/execution/context/json.rs index 013c47d046fc..e9d799400863 100644 --- a/datafusion/core/src/execution/context/json.rs +++ b/datafusion/core/src/execution/context/json.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::datasource::physical_plan::plan_to_json; use datafusion_common::TableReference; +use datafusion_datasource_json::source::plan_to_json; use std::sync::Arc; use super::super::options::{NdJsonReadOptions, ReadOptions}; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c27d1e4fd46b..ad0993ed43ca 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -83,12 +83,14 @@ use object_store::ObjectStore; use parking_lot::RwLock; use url::Url; -mod avro; mod csv; mod json; #[cfg(feature = "parquet")] mod parquet; +#[cfg(feature = "avro")] +mod avro; + /// DataFilePaths adds a method to convert strings and vector of strings to vector of [`ListingTableUrl`] URLs. /// This allows methods such [`SessionContext::read_csv`] and [`SessionContext::read_avro`] /// to take either a single file or multiple files. diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index 67ccacaea666..6ec9796fe90d 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use super::super::options::{ParquetReadOptions, ReadOptions}; use super::{DataFilePaths, DataFrame, ExecutionPlan, Result, SessionContext}; -use crate::datasource::physical_plan::parquet::plan_to_parquet; +use datafusion_datasource_parquet::plan_to_parquet; use datafusion_common::TableReference; use parquet::file::properties::WriterProperties; diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index ad2b69f25abc..dc43af9fa5da 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -258,6 +258,10 @@ impl Session for SessionState { fn as_any(&self) -> &dyn Any { self } + + fn table_options(&self) -> &TableOptions { + self.table_options() + } } impl SessionState { @@ -832,17 +836,16 @@ impl SessionState { self.config.options() } - /// return the TableOptions options with its extensions - pub fn default_table_options(&self) -> TableOptions { - self.table_options - .combine_with_session_config(self.config_options()) - } - /// Return the table options pub fn table_options(&self) -> &TableOptions { &self.table_options } + /// return the TableOptions options with its extensions + pub fn default_table_options(&self) -> TableOptions { + Session::default_table_options(self) + } + /// Return mutable table options pub fn table_options_mut(&mut self) -> &mut TableOptions { &mut self.table_options diff --git a/datafusion/core/src/execution/session_state_defaults.rs b/datafusion/core/src/execution/session_state_defaults.rs index 33bf01cf35cd..b48ef90f2bd5 100644 --- a/datafusion/core/src/execution/session_state_defaults.rs +++ b/datafusion/core/src/execution/session_state_defaults.rs @@ -18,6 +18,7 @@ use crate::catalog::{CatalogProvider, TableProviderFactory}; use crate::catalog_common::listing_schema::ListingSchemaProvider; use crate::datasource::file_format::arrow::ArrowFormatFactory; +#[cfg(feature = "avro")] use crate::datasource::file_format::avro::AvroFormatFactory; use crate::datasource::file_format::csv::CsvFormatFactory; use crate::datasource::file_format::json::JsonFormatFactory; @@ -135,6 +136,7 @@ impl SessionStateDefaults { Arc::new(JsonFormatFactory::new()), Arc::new(CsvFormatFactory::new()), Arc::new(ArrowFormatFactory::new()), + #[cfg(feature = "avro")] Arc::new(AvroFormatFactory::new()), ]; diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 23885c41c61a..be707f7e19d0 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -29,30 +29,29 @@ use crate::datasource::file_format::csv::CsvFormat; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::FileFormat; use crate::datasource::listing::PartitionedFile; -use crate::datasource::object_store::ObjectStoreUrl; + use crate::datasource::physical_plan::CsvSource; use crate::datasource::{MemTable, TableProvider}; use crate::error::Result; use crate::logical_expr::LogicalPlan; -use crate::test::object_store::local_unpartitioned_file; use crate::test_util::{aggr_test_schema, arrow_test_data}; use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::DataFusionError; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::source::DataSourceExec; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; #[cfg(feature = "compression")] use bzip2::Compression as BzCompression; +use datafusion_datasource_csv::partitioned_csv_config; #[cfg(feature = "compression")] use flate2::write::GzEncoder; #[cfg(feature = "compression")] use flate2::Compression as GzCompression; +use object_store::local_unpartitioned_file; #[cfg(feature = "compression")] use xz2::write::XzEncoder; #[cfg(feature = "compression")] @@ -186,16 +185,6 @@ pub fn partitioned_file_groups( .collect::>()) } -/// Returns a [`FileScanConfig`] for given `file_groups` -pub fn partitioned_csv_config( - schema: SchemaRef, - file_groups: Vec>, - file_source: Arc, -) -> FileScanConfig { - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema, file_source) - .with_file_groups(file_groups) -} - pub fn assert_fields_eq(plan: &LogicalPlan, expected: Vec<&str>) { let actual: Vec = plan .schema() diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 50e33b27e1bb..d6865ca3d532 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -27,9 +27,7 @@ use std::collections::HashMap; use std::fs::File; use std::io::Write; use std::path::Path; -use std::pin::Pin; use std::sync::Arc; -use std::task::{Context, Poll}; use crate::catalog::{TableProvider, TableProviderFactory}; use crate::dataframe::DataFrame; @@ -37,7 +35,7 @@ use crate::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use crate::datasource::{empty::EmptyTable, provider_as_source}; use crate::error::Result; use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; -use crate::physical_plan::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; +use crate::physical_plan::ExecutionPlan; use crate::prelude::{CsvReadOptions, SessionContext}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -47,7 +45,7 @@ use datafusion_common::TableReference; use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; use async_trait::async_trait; -use futures::Stream; + use tempfile::TempDir; // backwards compatibility #[cfg(feature = "parquet")] @@ -236,39 +234,3 @@ pub fn register_unbounded_file_with_ordering( ctx.register_table(table_name, Arc::new(StreamTable::new(Arc::new(config))))?; Ok(()) } - -struct BoundedStream { - limit: usize, - count: usize, - batch: RecordBatch, -} - -impl Stream for BoundedStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { - if self.count >= self.limit { - return Poll::Ready(None); - } - self.count += 1; - Poll::Ready(Some(Ok(self.batch.clone()))) - } -} - -impl RecordBatchStream for BoundedStream { - fn schema(&self) -> SchemaRef { - self.batch.schema() - } -} - -/// Creates an bounded stream for testing purposes. -pub fn bounded_stream(batch: RecordBatch, limit: usize) -> SendableRecordBatchStream { - Box::pin(BoundedStream { - count: 0, - limit, - batch, - }) -} diff --git a/datafusion/core/tests/parquet/external_access_plan.rs b/datafusion/core/tests/parquet/external_access_plan.rs index 1eacbe42c525..31c685378a21 100644 --- a/datafusion/core/tests/parquet/external_access_plan.rs +++ b/datafusion/core/tests/parquet/external_access_plan.rs @@ -27,10 +27,10 @@ use arrow::datatypes::SchemaRef; use arrow::util::pretty::pretty_format_batches; use datafusion::common::Result; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::parquet::{ParquetAccessPlan, RowGroupAccess}; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::prelude::SessionContext; use datafusion_common::{assert_contains, DFSchema}; +use datafusion_datasource_parquet::{ParquetAccessPlan, RowGroupAccess}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_plan::metrics::MetricsSet; diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index 4cbbcf12f32b..bb20246bf9d5 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -26,10 +26,10 @@ use datafusion::assert_batches_sorted_eq; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::physical_plan::collect; use datafusion::prelude::SessionContext; +use datafusion::test::object_store::local_unpartitioned_file; use datafusion_common::Result; use datafusion_execution::object_store::ObjectStoreUrl; -use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; @@ -168,16 +168,3 @@ pub async fn store_parquet( let meta: Vec<_> = files.iter().map(local_unpartitioned_file).collect(); Ok((meta, files)) } - -/// Helper method to fetch the file size and date at given path and create a `ObjectMeta` -pub fn local_unpartitioned_file(path: impl AsRef) -> ObjectMeta { - let location = Path::from_filesystem_path(path.as_ref()).unwrap(); - let metadata = std::fs::metadata(path).expect("Local file metadata"); - ObjectMeta { - location, - last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), - size: metadata.len() as usize, - e_tag: None, - version: None, - } -} diff --git a/datafusion/datasource-avro/Cargo.toml b/datafusion/datasource-avro/Cargo.toml new file mode 100644 index 000000000000..e6bb2ef4d5a9 --- /dev/null +++ b/datafusion/datasource-avro/Cargo.toml @@ -0,0 +1,60 @@ +# 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. + +[package] +name = "datafusion-datasource-avro" +description = "datafusion-datasource-avro" +authors.workspace = true +edition.workspace = true +homepage.workspace = true +license.workspace = true +readme.workspace = true +repository.workspace = true +rust-version.workspace = true +version.workspace = true + +[package.metadata.docs.rs] +all-features = true + +[dependencies] +apache-avro = { workspace = true } +arrow = { workspace = true } +async-trait = { workspace = true } +bytes = { workspace = true } +chrono = { workspace = true } +datafusion-catalog = { workspace = true } +datafusion-common = { workspace = true, features = ["object_store", "avro"] } +datafusion-datasource = { workspace = true } +datafusion-execution = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-plan = { workspace = true } +futures = { workspace = true } +num-traits = { version = "0.2" } +object_store = { workspace = true } +tokio = { workspace = true } + +[dev-dependencies] +rstest = { workspace = true } +serde_json = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_datasource_avro" +path = "src/mod.rs" diff --git a/datafusion/datasource-avro/LICENSE.txt b/datafusion/datasource-avro/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/datasource-avro/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/datasource-avro/NOTICE.txt b/datafusion/datasource-avro/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/datasource-avro/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/datasource-avro/README.md b/datafusion/datasource-avro/README.md new file mode 100644 index 000000000000..f8d7aebdcad1 --- /dev/null +++ b/datafusion/datasource-avro/README.md @@ -0,0 +1,26 @@ + + +# DataFusion datasource + +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that defines a Avro based file source. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs b/datafusion/datasource-avro/src/avro_to_arrow/arrow_array_reader.rs similarity index 99% rename from datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs rename to datafusion/datasource-avro/src/avro_to_arrow/arrow_array_reader.rs index 8f0e3792ffec..9a1b54b872ad 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/datasource-avro/src/avro_to_arrow/arrow_array_reader.rs @@ -17,13 +17,20 @@ //! Avro to Arrow array readers -use crate::arrow::array::{ +use apache_avro::schema::RecordSchema; +use apache_avro::{ + schema::{Schema as AvroSchema, SchemaKind}, + types::Value, + AvroResult, Error as AvroError, Reader as AvroReader, +}; +use arrow::array::{ make_array, Array, ArrayBuilder, ArrayData, ArrayDataBuilder, ArrayRef, BooleanBuilder, LargeStringArray, ListBuilder, NullArray, OffsetSizeTrait, PrimitiveArray, StringArray, StringBuilder, StringDictionaryBuilder, }; -use crate::arrow::buffer::{Buffer, MutableBuffer}; -use crate::arrow::datatypes::{ +use arrow::array::{BinaryArray, FixedSizeBinaryArray, GenericListArray}; +use arrow::buffer::{Buffer, MutableBuffer}; +use arrow::datatypes::{ ArrowDictionaryKeyType, ArrowNumericType, ArrowPrimitiveType, DataType, Date32Type, Date64Type, Field, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, Schema, Time32MillisecondType, Time32SecondType, Time64MicrosecondType, @@ -31,21 +38,14 @@ use crate::arrow::datatypes::{ TimestampNanosecondType, TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; -use crate::arrow::error::ArrowError; -use crate::arrow::record_batch::RecordBatch; -use crate::arrow::util::bit_util; -use crate::error::{DataFusionError, Result}; -use apache_avro::schema::RecordSchema; -use apache_avro::{ - schema::{Schema as AvroSchema, SchemaKind}, - types::Value, - AvroResult, Error as AvroError, Reader as AvroReader, -}; -use arrow::array::{BinaryArray, FixedSizeBinaryArray, GenericListArray}; use arrow::datatypes::{Fields, SchemaRef}; +use arrow::error::ArrowError; use arrow::error::ArrowError::SchemaError; use arrow::error::Result as ArrowResult; +use arrow::record_batch::RecordBatch; +use arrow::util::bit_util; use datafusion_common::arrow_err; +use datafusion_common::error::{DataFusionError, Result}; use num_traits::NumCast; use std::collections::BTreeMap; use std::io::Read; @@ -1071,10 +1071,10 @@ where #[cfg(test)] mod test { - use crate::arrow::array::Array; - use crate::arrow::datatypes::{Field, TimeUnit}; - use crate::datasource::avro_to_arrow::{Reader, ReaderBuilder}; + use crate::avro_to_arrow::{Reader, ReaderBuilder}; + use arrow::array::Array; use arrow::datatypes::DataType; + use arrow::datatypes::{Field, TimeUnit}; use datafusion_common::assert_batches_eq; use datafusion_common::cast::{ as_int32_array, as_int64_array, as_list_array, as_timestamp_microsecond_array, @@ -1083,7 +1083,7 @@ mod test { use std::sync::Arc; fn build_reader(name: &str, batch_size: usize) -> Reader { - let testdata = crate::test_util::arrow_test_data(); + let testdata = datafusion_common::test_util::arrow_test_data(); let filename = format!("{testdata}/avro/{name}"); let builder = ReaderBuilder::new() .read_schema() diff --git a/datafusion/core/src/datasource/avro_to_arrow/mod.rs b/datafusion/datasource-avro/src/avro_to_arrow/mod.rs similarity index 67% rename from datafusion/core/src/datasource/avro_to_arrow/mod.rs rename to datafusion/datasource-avro/src/avro_to_arrow/mod.rs index 71184a78c96f..c1530a488020 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/mod.rs +++ b/datafusion/datasource-avro/src/avro_to_arrow/mod.rs @@ -19,33 +19,21 @@ //! //! [Avro]: https://avro.apache.org/docs/1.2.0/ -#[cfg(feature = "avro")] mod arrow_array_reader; -#[cfg(feature = "avro")] mod reader; -#[cfg(feature = "avro")] mod schema; -use crate::arrow::datatypes::Schema; -use crate::error::Result; -#[cfg(feature = "avro")] +use arrow::datatypes::Schema; pub use reader::{Reader, ReaderBuilder}; -#[cfg(feature = "avro")] + pub use schema::to_arrow_schema; use std::io::Read; -#[cfg(feature = "avro")] /// Read Avro schema given a reader -pub fn read_avro_schema_from_reader(reader: &mut R) -> Result { +pub fn read_avro_schema_from_reader( + reader: &mut R, +) -> datafusion_common::Result { let avro_reader = apache_avro::Reader::new(reader)?; let schema = avro_reader.writer_schema(); to_arrow_schema(schema) } - -#[cfg(not(feature = "avro"))] -/// Read Avro schema given a reader (requires the avro feature) -pub fn read_avro_schema_from_reader(_: &mut R) -> Result { - Err(crate::error::DataFusionError::NotImplemented( - "cannot read avro schema without the 'avro' feature enabled".to_string(), - )) -} diff --git a/datafusion/core/src/datasource/avro_to_arrow/reader.rs b/datafusion/datasource-avro/src/avro_to_arrow/reader.rs similarity index 95% rename from datafusion/core/src/datasource/avro_to_arrow/reader.rs rename to datafusion/datasource-avro/src/avro_to_arrow/reader.rs index dbc24da46366..bc7b50a9cdc3 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/reader.rs +++ b/datafusion/datasource-avro/src/avro_to_arrow/reader.rs @@ -16,10 +16,10 @@ // under the License. use super::arrow_array_reader::AvroArrowArrayReader; -use crate::arrow::datatypes::SchemaRef; -use crate::arrow::record_batch::RecordBatch; -use crate::error::Result; +use arrow::datatypes::SchemaRef; use arrow::error::Result as ArrowResult; +use arrow::record_batch::RecordBatch; +use datafusion_common::Result; use std::io::{Read, Seek}; use std::sync::Arc; @@ -58,7 +58,7 @@ impl ReaderBuilder { /// ``` /// use std::fs::File; /// - /// use datafusion::datasource::avro_to_arrow::{Reader, ReaderBuilder}; + /// use datafusion_datasource_avro::avro_to_arrow::{Reader, ReaderBuilder}; /// /// fn example() -> Reader<'static, File> { /// let file = File::open("test/data/basic.avro").unwrap(); @@ -170,13 +170,17 @@ impl Iterator for Reader<'_, R> { #[cfg(test)] mod tests { use super::*; - use crate::arrow::array::*; - use crate::arrow::datatypes::{DataType, Field}; + use arrow::array::*; + use arrow::array::{ + BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array, + TimestampMicrosecondArray, + }; use arrow::datatypes::TimeUnit; + use arrow::datatypes::{DataType, Field}; use std::fs::File; fn build_reader(name: &str) -> Reader { - let testdata = crate::test_util::arrow_test_data(); + let testdata = datafusion_common::test_util::arrow_test_data(); let filename = format!("{testdata}/avro/{name}"); let builder = ReaderBuilder::new().read_schema().with_batch_size(64); builder.build(File::open(filename).unwrap()).unwrap() diff --git a/datafusion/core/src/datasource/avro_to_arrow/schema.rs b/datafusion/datasource-avro/src/avro_to_arrow/schema.rs similarity index 98% rename from datafusion/core/src/datasource/avro_to_arrow/schema.rs rename to datafusion/datasource-avro/src/avro_to_arrow/schema.rs index 991f648e58bd..276056c24c01 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/schema.rs +++ b/datafusion/datasource-avro/src/avro_to_arrow/schema.rs @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -use crate::arrow::datatypes::{DataType, IntervalUnit, Schema, TimeUnit, UnionMode}; -use crate::error::{DataFusionError, Result}; use apache_avro::schema::{ Alias, DecimalSchema, EnumSchema, FixedSchema, Name, RecordSchema, }; use apache_avro::types::Value; use apache_avro::Schema as AvroSchema; +use arrow::datatypes::{DataType, IntervalUnit, Schema, TimeUnit, UnionMode}; use arrow::datatypes::{Field, UnionFields}; +use datafusion_common::error::{DataFusionError, Result}; use std::collections::HashMap; use std::sync::Arc; @@ -309,12 +309,12 @@ pub fn aliased( #[cfg(test)] mod test { use super::{aliased, external_props, to_arrow_schema}; - use crate::arrow::datatypes::DataType::{Binary, Float32, Float64, Timestamp, Utf8}; - use crate::arrow::datatypes::TimeUnit::Microsecond; - use crate::arrow::datatypes::{Field, Schema}; use apache_avro::schema::{Alias, EnumSchema, FixedSchema, Name, RecordSchema}; use apache_avro::Schema as AvroSchema; + use arrow::datatypes::DataType::{Binary, Float32, Float64, Timestamp, Utf8}; use arrow::datatypes::DataType::{Boolean, Int32, Int64}; + use arrow::datatypes::TimeUnit::Microsecond; + use arrow::datatypes::{Field, Schema}; fn alias(name: &str) -> Alias { Alias::new(name).unwrap() diff --git a/datafusion/datasource-avro/src/file_format.rs b/datafusion/datasource-avro/src/file_format.rs new file mode 100644 index 000000000000..00a96121aa3b --- /dev/null +++ b/datafusion/datasource-avro/src/file_format.rs @@ -0,0 +1,160 @@ +// 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. + +//! Apache Avro [`FileFormat`] abstractions + +use std::any::Any; +use std::collections::HashMap; +use std::fmt; +use std::sync::Arc; + +use datafusion_common::{Result, Statistics}; +use datafusion_datasource::file_compression_type::FileCompressionType; +use datafusion_datasource::file_format::{FileFormat, FileFormatFactory}; + +use arrow::datatypes::Schema; +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; +use datafusion_catalog::Session; +use datafusion_common::internal_err; +use datafusion_common::parsers::CompressionTypeVariant; +use datafusion_common::GetExt; +use datafusion_common::DEFAULT_AVRO_EXTENSION; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::ExecutionPlan; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; + +use crate::avro_to_arrow::read_avro_schema_from_reader; +use crate::source::AvroSource; + +#[derive(Default)] +/// Factory struct used to create [`AvroFormat`] +pub struct AvroFormatFactory; + +impl AvroFormatFactory { + /// Creates an instance of [`AvroFormatFactory`] + pub fn new() -> Self { + Self {} + } +} + +impl FileFormatFactory for AvroFormatFactory { + fn create( + &self, + _state: &dyn Session, + _format_options: &HashMap, + ) -> Result> { + Ok(Arc::new(AvroFormat)) + } + + fn default(&self) -> Arc { + Arc::new(AvroFormat) + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +impl fmt::Debug for AvroFormatFactory { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("AvroFormatFactory").finish() + } +} + +impl GetExt for AvroFormatFactory { + fn get_ext(&self) -> String { + // Removes the dot, i.e. ".avro" -> "avro" + DEFAULT_AVRO_EXTENSION[1..].to_string() + } +} + +/// Avro [`FileFormat`] implementation. +#[derive(Default, Debug)] +pub struct AvroFormat; + +#[async_trait] +impl FileFormat for AvroFormat { + fn as_any(&self) -> &dyn Any { + self + } + + fn get_ext(&self) -> String { + AvroFormatFactory::new().get_ext() + } + + fn get_ext_with_compression( + &self, + file_compression_type: &FileCompressionType, + ) -> Result { + let ext = self.get_ext(); + match file_compression_type.get_variant() { + CompressionTypeVariant::UNCOMPRESSED => Ok(ext), + _ => internal_err!("Avro FileFormat does not support compression."), + } + } + + async fn infer_schema( + &self, + _state: &dyn Session, + store: &Arc, + objects: &[ObjectMeta], + ) -> Result { + let mut schemas = vec![]; + for object in objects { + let r = store.as_ref().get(&object.location).await?; + let schema = match r.payload { + GetResultPayload::File(mut file, _) => { + read_avro_schema_from_reader(&mut file)? + } + GetResultPayload::Stream(_) => { + // TODO: Fetching entire file to get schema is potentially wasteful + let data = r.bytes().await?; + read_avro_schema_from_reader(&mut data.as_ref())? + } + }; + schemas.push(schema); + } + let merged_schema = Schema::try_merge(schemas)?; + Ok(Arc::new(merged_schema)) + } + + async fn infer_stats( + &self, + _state: &dyn Session, + _store: &Arc, + table_schema: SchemaRef, + _object: &ObjectMeta, + ) -> Result { + Ok(Statistics::new_unknown(&table_schema)) + } + + async fn create_physical_plan( + &self, + _state: &dyn Session, + conf: FileScanConfig, + _filters: Option<&Arc>, + ) -> Result> { + Ok(conf.with_source(self.file_source()).build()) + } + + fn file_source(&self) -> Arc { + Arc::new(AvroSource::new()) + } +} diff --git a/datafusion/datasource-avro/src/mod.rs b/datafusion/datasource-avro/src/mod.rs new file mode 100644 index 000000000000..7d00b14e5119 --- /dev/null +++ b/datafusion/datasource-avro/src/mod.rs @@ -0,0 +1,30 @@ +// 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. + +#![doc( + html_logo_url = "https://raw.githubusercontent.com/apache/datafusion/19fe44cf2f30cbdd63d4a4f52c74055163c6cc38/docs/logos/standalone_logo/logo_original.svg", + html_favicon_url = "https://raw.githubusercontent.com/apache/datafusion/19fe44cf2f30cbdd63d4a4f52c74055163c6cc38/docs/logos/standalone_logo/logo_original.svg" +)] +#![cfg_attr(docsrs, feature(doc_auto_cfg))] + +//! An [Avro](https://avro.apache.org/) based [`FileSource`](datafusion_datasource::file::FileSource) implementation and related functionality. + +pub mod avro_to_arrow; +pub mod file_format; +pub mod source; + +pub use file_format::*; diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs new file mode 100644 index 000000000000..ce3722e7b11e --- /dev/null +++ b/datafusion/datasource-avro/src/source.rs @@ -0,0 +1,282 @@ +// 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. + +//! Execution plan for reading line-delimited Avro files + +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + +use crate::avro_to_arrow::Reader as AvroReader; + +use datafusion_common::error::Result; + +use arrow::datatypes::SchemaRef; +use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::source::DataSourceExec; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; + +use object_store::ObjectStore; + +/// Execution plan for scanning Avro data source +#[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] +pub struct AvroExec { + inner: DataSourceExec, + base_config: FileScanConfig, +} + +#[allow(unused, deprecated)] +impl AvroExec { + /// Create a new Avro reader execution plan provided base configurations + pub fn new(base_config: FileScanConfig) -> Self { + let ( + projected_schema, + projected_constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); + let cache = Self::compute_properties( + Arc::clone(&projected_schema), + &projected_output_ordering, + projected_constraints, + &base_config, + ); + let base_config = base_config.with_source(Arc::new(AvroSource::default())); + Self { + inner: DataSourceExec::new(Arc::new(base_config.clone())), + base_config, + } + } + + /// Ref to the base configs + pub fn base_config(&self) -> &FileScanConfig { + &self.base_config + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + constraints: Constraints, + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints); + let n_partitions = file_scan_config.file_groups.len(); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +#[allow(unused, deprecated)] +impl DisplayAs for AvroExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + self.inner.fmt_as(t, f) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for AvroExec { + fn name(&self) -> &'static str { + "AvroExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + self.inner.properties() + } + fn children(&self) -> Vec<&Arc> { + Vec::new() + } + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + self.inner.execute(partition, context) + } + + fn statistics(&self) -> Result { + self.inner.statistics() + } + + fn metrics(&self) -> Option { + self.inner.metrics() + } + + fn fetch(&self) -> Option { + self.inner.fetch() + } + + fn with_fetch(&self, limit: Option) -> Option> { + self.inner.with_fetch(limit) + } +} + +/// AvroSource holds the extra configuration that is necessary for opening avro files +#[derive(Clone, Default)] +pub struct AvroSource { + schema: Option, + batch_size: Option, + projection: Option>, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} + +impl AvroSource { + /// Initialize an AvroSource with default values + pub fn new() -> Self { + Self::default() + } + + fn open(&self, reader: R) -> Result> { + AvroReader::try_new( + reader, + Arc::clone(self.schema.as_ref().expect("Schema must set before open")), + self.batch_size.expect("Batch size must set before open"), + self.projection.clone(), + ) + } +} + +impl FileSource for AvroSource { + fn create_file_opener( + &self, + object_store: Arc, + _base_config: &FileScanConfig, + _partition: usize, + ) -> Arc { + Arc::new(private::AvroOpener { + config: Arc::new(self.clone()), + object_store, + }) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) + } + + fn with_schema(&self, schema: SchemaRef) -> Arc { + let mut conf = self.clone(); + conf.schema = Some(schema); + Arc::new(conf) + } + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, config: &FileScanConfig) -> Arc { + let mut conf = self.clone(); + conf.projection = config.projected_file_column_names(); + Arc::new(conf) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn statistics(&self) -> Result { + let statistics = &self.projected_statistics; + Ok(statistics + .clone() + .expect("projected_statistics must be set")) + } + + fn file_type(&self) -> &str { + "avro" + } + + fn repartitioned( + &self, + _target_partitions: usize, + _repartition_file_min_size: usize, + _output_ordering: Option, + _config: &FileScanConfig, + ) -> Result> { + Ok(None) + } +} + +mod private { + use super::*; + + use bytes::Buf; + use datafusion_datasource::{file_meta::FileMeta, file_stream::FileOpenFuture}; + use futures::StreamExt; + use object_store::{GetResultPayload, ObjectStore}; + + pub struct AvroOpener { + pub config: Arc, + pub object_store: Arc, + } + + impl FileOpener for AvroOpener { + fn open(&self, file_meta: FileMeta) -> Result { + let config = Arc::clone(&self.config); + let object_store = Arc::clone(&self.object_store); + Ok(Box::pin(async move { + let r = object_store.get(file_meta.location()).await?; + match r.payload { + GetResultPayload::File(file, _) => { + let reader = config.open(file)?; + Ok(futures::stream::iter(reader).boxed()) + } + GetResultPayload::Stream(_) => { + let bytes = r.bytes().await?; + let reader = config.open(bytes.reader())?; + Ok(futures::stream::iter(reader).boxed()) + } + } + })) + } + } +} diff --git a/datafusion/datasource-csv/Cargo.toml b/datafusion/datasource-csv/Cargo.toml new file mode 100644 index 000000000000..689531758cad --- /dev/null +++ b/datafusion/datasource-csv/Cargo.toml @@ -0,0 +1,60 @@ +# 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. + +[package] +name = "datafusion-datasource-csv" +description = "datafusion-datasource-csv" +authors.workspace = true +edition.workspace = true +homepage.workspace = true +license.workspace = true +readme.workspace = true +repository.workspace = true +rust-version.workspace = true +version.workspace = true + +[package.metadata.docs.rs] +all-features = true + +[dependencies] +arrow = { workspace = true } +async-trait = { workspace = true } +bytes = { workspace = true } +datafusion-catalog = { workspace = true } +datafusion-common = { workspace = true, features = ["object_store"] } +datafusion-common-runtime = { workspace = true } +datafusion-datasource = { workspace = true } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-plan = { workspace = true } +futures = { workspace = true } +itertools = { workspace = true } +log = { workspace = true } +object_store = { workspace = true } +rand = { workspace = true } +regex = { workspace = true } +tokio = { workspace = true } +url = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_datasource_csv" +path = "src/mod.rs" diff --git a/datafusion/datasource-csv/LICENSE.txt b/datafusion/datasource-csv/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/datasource-csv/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/datasource-csv/NOTICE.txt b/datafusion/datasource-csv/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/datasource-csv/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/datasource-csv/README.md b/datafusion/datasource-csv/README.md new file mode 100644 index 000000000000..c5944f9e438f --- /dev/null +++ b/datafusion/datasource-csv/README.md @@ -0,0 +1,26 @@ + + +# DataFusion datasource + +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that defines a CSV based file source. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/datasource-csv/src/file_format.rs b/datafusion/datasource-csv/src/file_format.rs new file mode 100644 index 000000000000..2e083f9ef22f --- /dev/null +++ b/datafusion/datasource-csv/src/file_format.rs @@ -0,0 +1,736 @@ +// 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. + +//! [`CsvFormat`], Comma Separated Value (CSV) [`FileFormat`] abstractions + +use std::any::Any; +use std::collections::{HashMap, HashSet}; +use std::fmt::{self, Debug}; +use std::sync::Arc; + +use arrow::array::RecordBatch; +use arrow::csv::WriterBuilder; +use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; +use arrow::error::ArrowError; +use datafusion_catalog::Session; +use datafusion_common::config::{ConfigField, ConfigFileType, CsvOptions}; +use datafusion_common::file_options::csv_writer::CsvWriterOptions; +use datafusion_common::{ + exec_err, not_impl_err, DataFusionError, GetExt, Result, Statistics, + DEFAULT_CSV_EXTENSION, +}; +use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::decoder::Decoder; +use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_compression_type::FileCompressionType; +use datafusion_datasource::file_format::{ + FileFormat, FileFormatFactory, DEFAULT_SCHEMA_INFER_MAX_RECORD, +}; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig}; +use datafusion_datasource::write::demux::DemuxedStreamReceiver; +use datafusion_datasource::write::orchestration::spawn_writer_tasks_and_join; +use datafusion_datasource::write::BatchSerializer; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_expr::dml::InsertOp; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexRequirement; + +use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; +use futures::stream::BoxStream; +use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; +use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; +use regex::Regex; + +use crate::source::CsvSource; + +#[derive(Default)] +/// Factory used to create [`CsvFormat`] +pub struct CsvFormatFactory { + /// the options for csv file read + pub options: Option, +} + +impl CsvFormatFactory { + /// Creates an instance of [`CsvFormatFactory`] + pub fn new() -> Self { + Self { options: None } + } + + /// Creates an instance of [`CsvFormatFactory`] with customized default options + pub fn new_with_options(options: CsvOptions) -> Self { + Self { + options: Some(options), + } + } +} + +impl Debug for CsvFormatFactory { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("CsvFormatFactory") + .field("options", &self.options) + .finish() + } +} + +impl FileFormatFactory for CsvFormatFactory { + fn create( + &self, + state: &dyn Session, + format_options: &HashMap, + ) -> Result> { + let csv_options = match &self.options { + None => { + let mut table_options = state.default_table_options(); + table_options.set_config_format(ConfigFileType::CSV); + table_options.alter_with_string_hash_map(format_options)?; + table_options.csv + } + Some(csv_options) => { + let mut csv_options = csv_options.clone(); + for (k, v) in format_options { + csv_options.set(k, v)?; + } + csv_options + } + }; + + Ok(Arc::new(CsvFormat::default().with_options(csv_options))) + } + + fn default(&self) -> Arc { + Arc::new(CsvFormat::default()) + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +impl GetExt for CsvFormatFactory { + fn get_ext(&self) -> String { + // Removes the dot, i.e. ".csv" -> "csv" + DEFAULT_CSV_EXTENSION[1..].to_string() + } +} + +/// Character Separated Value [`FileFormat`] implementation. +#[derive(Debug, Default)] +pub struct CsvFormat { + options: CsvOptions, +} + +impl CsvFormat { + /// Return a newline delimited stream from the specified file on + /// Stream, decompressing if necessary + /// Each returned `Bytes` has a whole number of newline delimited rows + async fn read_to_delimited_chunks<'a>( + &self, + store: &Arc, + object: &ObjectMeta, + ) -> BoxStream<'a, Result> { + // stream to only read as many rows as needed into memory + let stream = store + .get(&object.location) + .await + .map_err(DataFusionError::ObjectStore); + let stream = match stream { + Ok(stream) => self + .read_to_delimited_chunks_from_stream( + stream + .into_stream() + .map_err(DataFusionError::ObjectStore) + .boxed(), + ) + .await + .map_err(DataFusionError::from) + .left_stream(), + Err(e) => { + futures::stream::once(futures::future::ready(Err(e))).right_stream() + } + }; + stream.boxed() + } + + /// Convert a stream of bytes into a stream of of [`Bytes`] containing newline + /// delimited CSV records, while accounting for `\` and `"`. + pub async fn read_to_delimited_chunks_from_stream<'a>( + &self, + stream: BoxStream<'a, Result>, + ) -> BoxStream<'a, Result> { + let file_compression_type: FileCompressionType = self.options.compression.into(); + let decoder = file_compression_type.convert_stream(stream); + let stream = match decoder { + Ok(decoded_stream) => { + newline_delimited_stream(decoded_stream.map_err(|e| match e { + DataFusionError::ObjectStore(e) => e, + err => object_store::Error::Generic { + store: "read to delimited chunks failed", + source: Box::new(err), + }, + })) + .map_err(DataFusionError::from) + .left_stream() + } + Err(e) => { + futures::stream::once(futures::future::ready(Err(e))).right_stream() + } + }; + stream.boxed() + } + + /// Set the csv options + pub fn with_options(mut self, options: CsvOptions) -> Self { + self.options = options; + self + } + + /// Retrieve the csv options + pub fn options(&self) -> &CsvOptions { + &self.options + } + + /// Set a limit in terms of records to scan to infer the schema + /// - default to `DEFAULT_SCHEMA_INFER_MAX_RECORD` + pub fn with_schema_infer_max_rec(mut self, max_rec: usize) -> Self { + self.options.schema_infer_max_rec = Some(max_rec); + self + } + + /// Set true to indicate that the first line is a header. + /// - default to true + pub fn with_has_header(mut self, has_header: bool) -> Self { + self.options.has_header = Some(has_header); + self + } + + /// Set the regex to use for null values in the CSV reader. + /// - default to treat empty values as null. + pub fn with_null_regex(mut self, null_regex: Option) -> Self { + self.options.null_regex = null_regex; + self + } + + /// Returns `Some(true)` if the first line is a header, `Some(false)` if + /// it is not, and `None` if it is not specified. + pub fn has_header(&self) -> Option { + self.options.has_header + } + + /// Lines beginning with this byte are ignored. + pub fn with_comment(mut self, comment: Option) -> Self { + self.options.comment = comment; + self + } + + /// The character separating values within a row. + /// - default to ',' + pub fn with_delimiter(mut self, delimiter: u8) -> Self { + self.options.delimiter = delimiter; + self + } + + /// The quote character in a row. + /// - default to '"' + pub fn with_quote(mut self, quote: u8) -> Self { + self.options.quote = quote; + self + } + + /// The escape character in a row. + /// - default is None + pub fn with_escape(mut self, escape: Option) -> Self { + self.options.escape = escape; + self + } + + /// The character used to indicate the end of a row. + /// - default to None (CRLF) + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.options.terminator = terminator; + self + } + + /// Specifies whether newlines in (quoted) values are supported. + /// + /// Parsing newlines in quoted values may be affected by execution behaviour such as + /// parallel file scanning. Setting this to `true` ensures that newlines in values are + /// parsed successfully, which may reduce performance. + /// + /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. + pub fn with_newlines_in_values(mut self, newlines_in_values: bool) -> Self { + self.options.newlines_in_values = Some(newlines_in_values); + self + } + + /// Set a `FileCompressionType` of CSV + /// - defaults to `FileCompressionType::UNCOMPRESSED` + pub fn with_file_compression_type( + mut self, + file_compression_type: FileCompressionType, + ) -> Self { + self.options.compression = file_compression_type.into(); + self + } + + /// The delimiter character. + pub fn delimiter(&self) -> u8 { + self.options.delimiter + } + + /// The quote character. + pub fn quote(&self) -> u8 { + self.options.quote + } + + /// The escape character. + pub fn escape(&self) -> Option { + self.options.escape + } +} + +#[derive(Debug)] +pub struct CsvDecoder { + inner: arrow::csv::reader::Decoder, +} + +impl CsvDecoder { + pub fn new(decoder: arrow::csv::reader::Decoder) -> Self { + Self { inner: decoder } + } +} + +impl Decoder for CsvDecoder { + fn decode(&mut self, buf: &[u8]) -> Result { + self.inner.decode(buf) + } + + fn flush(&mut self) -> Result, ArrowError> { + self.inner.flush() + } + + fn can_flush_early(&self) -> bool { + self.inner.capacity() == 0 + } +} + +impl Debug for CsvSerializer { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("CsvSerializer") + .field("header", &self.header) + .finish() + } +} + +#[async_trait] +impl FileFormat for CsvFormat { + fn as_any(&self) -> &dyn Any { + self + } + + fn get_ext(&self) -> String { + CsvFormatFactory::new().get_ext() + } + + fn get_ext_with_compression( + &self, + file_compression_type: &FileCompressionType, + ) -> Result { + let ext = self.get_ext(); + Ok(format!("{}{}", ext, file_compression_type.get_ext())) + } + + async fn infer_schema( + &self, + state: &dyn Session, + store: &Arc, + objects: &[ObjectMeta], + ) -> Result { + let mut schemas = vec![]; + + let mut records_to_read = self + .options + .schema_infer_max_rec + .unwrap_or(DEFAULT_SCHEMA_INFER_MAX_RECORD); + + for object in objects { + let stream = self.read_to_delimited_chunks(store, object).await; + let (schema, records_read) = self + .infer_schema_from_stream(state, records_to_read, stream) + .await + .map_err(|err| { + DataFusionError::Context( + format!("Error when processing CSV file {}", &object.location), + Box::new(err), + ) + })?; + records_to_read -= records_read; + schemas.push(schema); + if records_to_read == 0 { + break; + } + } + + let merged_schema = Schema::try_merge(schemas)?; + Ok(Arc::new(merged_schema)) + } + + async fn infer_stats( + &self, + _state: &dyn Session, + _store: &Arc, + table_schema: SchemaRef, + _object: &ObjectMeta, + ) -> Result { + Ok(Statistics::new_unknown(&table_schema)) + } + + async fn create_physical_plan( + &self, + state: &dyn Session, + mut conf: FileScanConfig, + _filters: Option<&Arc>, + ) -> Result> { + conf.file_compression_type = self.options.compression.into(); + // Consult configuration options for default values + let has_header = self + .options + .has_header + .unwrap_or(state.config_options().catalog.has_header); + let newlines_in_values = self + .options + .newlines_in_values + .unwrap_or(state.config_options().catalog.newlines_in_values); + conf.new_lines_in_values = newlines_in_values; + + let source = Arc::new( + CsvSource::new(has_header, self.options.delimiter, self.options.quote) + .with_escape(self.options.escape) + .with_terminator(self.options.terminator) + .with_comment(self.options.comment), + ); + Ok(conf.with_source(source).build()) + } + + async fn create_writer_physical_plan( + &self, + input: Arc, + state: &dyn Session, + conf: FileSinkConfig, + order_requirements: Option, + ) -> Result> { + if conf.insert_op != InsertOp::Append { + return not_impl_err!("Overwrites are not implemented yet for CSV"); + } + + // `has_header` and `newlines_in_values` fields of CsvOptions may inherit + // their values from session from configuration settings. To support + // this logic, writer options are built from the copy of `self.options` + // with updated values of these special fields. + let has_header = self + .options() + .has_header + .unwrap_or(state.config_options().catalog.has_header); + let newlines_in_values = self + .options() + .newlines_in_values + .unwrap_or(state.config_options().catalog.newlines_in_values); + + let options = self + .options() + .clone() + .with_has_header(has_header) + .with_newlines_in_values(newlines_in_values); + + let writer_options = CsvWriterOptions::try_from(&options)?; + + let sink = Arc::new(CsvSink::new(conf, writer_options)); + + Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) + } + + fn file_source(&self) -> Arc { + Arc::new(CsvSource::default()) + } +} + +impl CsvFormat { + /// Return the inferred schema reading up to records_to_read from a + /// stream of delimited chunks returning the inferred schema and the + /// number of lines that were read + pub async fn infer_schema_from_stream( + &self, + state: &dyn Session, + mut records_to_read: usize, + stream: impl Stream>, + ) -> Result<(Schema, usize)> { + let mut total_records_read = 0; + let mut column_names = vec![]; + let mut column_type_possibilities = vec![]; + let mut record_number = -1; + + pin_mut!(stream); + + while let Some(chunk) = stream.next().await.transpose()? { + record_number += 1; + let first_chunk = record_number == 0; + let mut format = arrow::csv::reader::Format::default() + .with_header( + first_chunk + && self + .options + .has_header + .unwrap_or(state.config_options().catalog.has_header), + ) + .with_delimiter(self.options.delimiter) + .with_quote(self.options.quote); + + if let Some(null_regex) = &self.options.null_regex { + let regex = Regex::new(null_regex.as_str()) + .expect("Unable to parse CSV null regex."); + format = format.with_null_regex(regex); + } + + if let Some(escape) = self.options.escape { + format = format.with_escape(escape); + } + + if let Some(comment) = self.options.comment { + format = format.with_comment(comment); + } + + let (Schema { fields, .. }, records_read) = + format.infer_schema(chunk.reader(), Some(records_to_read))?; + + records_to_read -= records_read; + total_records_read += records_read; + + if first_chunk { + // set up initial structures for recording inferred schema across chunks + (column_names, column_type_possibilities) = fields + .into_iter() + .map(|field| { + let mut possibilities = HashSet::new(); + if records_read > 0 { + // at least 1 data row read, record the inferred datatype + possibilities.insert(field.data_type().clone()); + } + (field.name().clone(), possibilities) + }) + .unzip(); + } else { + if fields.len() != column_type_possibilities.len() { + return exec_err!( + "Encountered unequal lengths between records on CSV file whilst inferring schema. \ + Expected {} fields, found {} fields at record {}", + column_type_possibilities.len(), + fields.len(), + record_number + 1 + ); + } + + column_type_possibilities.iter_mut().zip(&fields).for_each( + |(possibilities, field)| { + possibilities.insert(field.data_type().clone()); + }, + ); + } + + if records_to_read == 0 { + break; + } + } + + let schema = build_schema_helper(column_names, &column_type_possibilities); + Ok((schema, total_records_read)) + } +} + +fn build_schema_helper(names: Vec, types: &[HashSet]) -> Schema { + let fields = names + .into_iter() + .zip(types) + .map(|(field_name, data_type_possibilities)| { + // ripped from arrow::csv::reader::infer_reader_schema_with_csv_options + // determine data type based on possible types + // if there are incompatible types, use DataType::Utf8 + match data_type_possibilities.len() { + 1 => Field::new( + field_name, + data_type_possibilities.iter().next().unwrap().clone(), + true, + ), + 2 => { + if data_type_possibilities.contains(&DataType::Int64) + && data_type_possibilities.contains(&DataType::Float64) + { + // we have an integer and double, fall down to double + Field::new(field_name, DataType::Float64, true) + } else { + // default to Utf8 for conflicting datatypes (e.g bool and int) + Field::new(field_name, DataType::Utf8, true) + } + } + _ => Field::new(field_name, DataType::Utf8, true), + } + }) + .collect::(); + Schema::new(fields) +} + +impl Default for CsvSerializer { + fn default() -> Self { + Self::new() + } +} + +/// Define a struct for serializing CSV records to a stream +pub struct CsvSerializer { + // CSV writer builder + builder: WriterBuilder, + // Flag to indicate whether there will be a header + header: bool, +} + +impl CsvSerializer { + /// Constructor for the CsvSerializer object + pub fn new() -> Self { + Self { + builder: WriterBuilder::new(), + header: true, + } + } + + /// Method for setting the CSV writer builder + pub fn with_builder(mut self, builder: WriterBuilder) -> Self { + self.builder = builder; + self + } + + /// Method for setting the CSV writer header status + pub fn with_header(mut self, header: bool) -> Self { + self.header = header; + self + } +} + +impl BatchSerializer for CsvSerializer { + fn serialize(&self, batch: RecordBatch, initial: bool) -> Result { + let mut buffer = Vec::with_capacity(4096); + let builder = self.builder.clone(); + let header = self.header && initial; + let mut writer = builder.with_header(header).build(&mut buffer); + writer.write(&batch)?; + drop(writer); + Ok(Bytes::from(buffer)) + } +} + +/// Implements [`DataSink`] for writing to a CSV file. +pub struct CsvSink { + /// Config options for writing data + config: FileSinkConfig, + writer_options: CsvWriterOptions, +} + +impl Debug for CsvSink { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("CsvSink").finish() + } +} + +impl DisplayAs for CsvSink { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "CsvSink(file_groups=",)?; + FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; + write!(f, ")") + } + } + } +} + +impl CsvSink { + /// Create from config. + pub fn new(config: FileSinkConfig, writer_options: CsvWriterOptions) -> Self { + Self { + config, + writer_options, + } + } + + /// Retrieve the writer options + pub fn writer_options(&self) -> &CsvWriterOptions { + &self.writer_options + } +} + +#[async_trait] +impl FileSink for CsvSink { + fn config(&self) -> &FileSinkConfig { + &self.config + } + + async fn spawn_writer_tasks_and_join( + &self, + context: &Arc, + demux_task: SpawnedTask>, + file_stream_rx: DemuxedStreamReceiver, + object_store: Arc, + ) -> Result { + let builder = self.writer_options.writer_options.clone(); + let header = builder.header(); + let serializer = Arc::new( + CsvSerializer::new() + .with_builder(builder) + .with_header(header), + ) as _; + spawn_writer_tasks_and_join( + context, + serializer, + self.writer_options.compression.into(), + object_store, + demux_task, + file_stream_rx, + ) + .await + } +} + +#[async_trait] +impl DataSink for CsvSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> &SchemaRef { + self.config.output_schema() + } + + async fn write_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + FileSink::write_all(self, data, context).await + } +} diff --git a/datafusion/datasource-csv/src/mod.rs b/datafusion/datasource-csv/src/mod.rs new file mode 100644 index 000000000000..4117d1fee5fc --- /dev/null +++ b/datafusion/datasource-csv/src/mod.rs @@ -0,0 +1,38 @@ +// 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. + +pub mod file_format; +pub mod source; + +use std::sync::Arc; + +use arrow::datatypes::SchemaRef; +use datafusion_datasource::{ + file::FileSource, file_scan_config::FileScanConfig, PartitionedFile, +}; +use datafusion_execution::object_store::ObjectStoreUrl; +pub use file_format::*; + +/// Returns a [`FileScanConfig`] for given `file_groups` +pub fn partitioned_csv_config( + schema: SchemaRef, + file_groups: Vec>, + file_source: Arc, +) -> FileScanConfig { + FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema, file_source) + .with_file_groups(file_groups) +} diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs new file mode 100644 index 000000000000..124b32d4f181 --- /dev/null +++ b/datafusion/datasource-csv/src/source.rs @@ -0,0 +1,781 @@ +// 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. + +//! Execution plan for reading CSV files + +use std::any::Any; +use std::fmt; +use std::io::{Read, Seek, SeekFrom}; +use std::sync::Arc; +use std::task::Poll; + +use datafusion_datasource::decoder::{deserialize_stream, DecoderDeserializer}; +use datafusion_datasource::file_compression_type::FileCompressionType; +use datafusion_datasource::file_meta::FileMeta; +use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; +use datafusion_datasource::{ + calculate_range, FileRange, ListingTableUrl, PartitionedFile, RangeCalculation, +}; + +use arrow::csv; +use arrow::datatypes::SchemaRef; +use datafusion_common::config::ConfigOptions; +use datafusion_common::{Constraints, DataFusionError, Result, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::source::DataSourceExec; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, +}; + +use futures::{StreamExt, TryStreamExt}; +use object_store::buffered::BufWriter; +use object_store::{GetOptions, GetResultPayload, ObjectStore}; +use tokio::io::AsyncWriteExt; +use tokio::task::JoinSet; + +use crate::file_format::CsvDecoder; + +/// Old Csv source, deprecated with DataSourceExec implementation and CsvSource +/// +/// See examples on `CsvSource` +#[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] +pub struct CsvExec { + base_config: FileScanConfig, + inner: DataSourceExec, +} + +/// Builder for [`CsvExec`]. +/// +/// See example on [`CsvExec`]. +#[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use FileScanConfig instead")] +pub struct CsvExecBuilder { + file_scan_config: FileScanConfig, + file_compression_type: FileCompressionType, + // TODO: it seems like these format options could be reused across all the various CSV config + has_header: bool, + delimiter: u8, + quote: u8, + terminator: Option, + escape: Option, + comment: Option, + newlines_in_values: bool, +} + +#[allow(unused, deprecated)] +impl CsvExecBuilder { + /// Create a new builder to read the provided file scan configuration. + pub fn new(file_scan_config: FileScanConfig) -> Self { + Self { + file_scan_config, + // TODO: these defaults are duplicated from `CsvOptions` - should they be computed? + has_header: false, + delimiter: b',', + quote: b'"', + terminator: None, + escape: None, + comment: None, + newlines_in_values: false, + file_compression_type: FileCompressionType::UNCOMPRESSED, + } + } + + /// Set whether the first row defines the column names. + /// + /// The default value is `false`. + pub fn with_has_header(mut self, has_header: bool) -> Self { + self.has_header = has_header; + self + } + + /// Set the column delimeter. + /// + /// The default is `,`. + pub fn with_delimeter(mut self, delimiter: u8) -> Self { + self.delimiter = delimiter; + self + } + + /// Set the quote character. + /// + /// The default is `"`. + pub fn with_quote(mut self, quote: u8) -> Self { + self.quote = quote; + self + } + + /// Set the line terminator. If not set, the default is CRLF. + /// + /// The default is None. + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + + /// Set the escape character. + /// + /// The default is `None` (i.e. quotes cannot be escaped). + pub fn with_escape(mut self, escape: Option) -> Self { + self.escape = escape; + self + } + + /// Set the comment character. + /// + /// The default is `None` (i.e. comments are not supported). + pub fn with_comment(mut self, comment: Option) -> Self { + self.comment = comment; + self + } + + /// Set whether newlines in (quoted) values are supported. + /// + /// Parsing newlines in quoted values may be affected by execution behaviour such as + /// parallel file scanning. Setting this to `true` ensures that newlines in values are + /// parsed successfully, which may reduce performance. + /// + /// The default value is `false`. + pub fn with_newlines_in_values(mut self, newlines_in_values: bool) -> Self { + self.newlines_in_values = newlines_in_values; + self + } + + /// Set the file compression type. + /// + /// The default is [`FileCompressionType::UNCOMPRESSED`]. + pub fn with_file_compression_type( + mut self, + file_compression_type: FileCompressionType, + ) -> Self { + self.file_compression_type = file_compression_type; + self + } + + /// Build a [`CsvExec`]. + #[must_use] + pub fn build(self) -> CsvExec { + let Self { + file_scan_config: base_config, + file_compression_type, + has_header, + delimiter, + quote, + terminator, + escape, + comment, + newlines_in_values, + } = self; + + let ( + projected_schema, + projected_constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); + let cache = CsvExec::compute_properties( + projected_schema, + &projected_output_ordering, + projected_constraints, + &base_config, + ); + let csv = CsvSource::new(has_header, delimiter, quote) + .with_comment(comment) + .with_escape(escape) + .with_terminator(terminator); + let base_config = base_config + .with_newlines_in_values(newlines_in_values) + .with_file_compression_type(file_compression_type) + .with_source(Arc::new(csv)); + + CsvExec { + inner: DataSourceExec::new(Arc::new(base_config.clone())), + base_config, + } + } +} + +#[allow(unused, deprecated)] +impl CsvExec { + /// Create a new CSV reader execution plan provided base and specific configurations + #[allow(clippy::too_many_arguments)] + pub fn new( + base_config: FileScanConfig, + has_header: bool, + delimiter: u8, + quote: u8, + terminator: Option, + escape: Option, + comment: Option, + newlines_in_values: bool, + file_compression_type: FileCompressionType, + ) -> Self { + CsvExecBuilder::new(base_config) + .with_has_header(has_header) + .with_delimeter(delimiter) + .with_quote(quote) + .with_terminator(terminator) + .with_escape(escape) + .with_comment(comment) + .with_newlines_in_values(newlines_in_values) + .with_file_compression_type(file_compression_type) + .build() + } + + /// Return a [`CsvExecBuilder`]. + /// + /// See example on [`CsvExec`] and [`CsvExecBuilder`] for specifying CSV table options. + pub fn builder(file_scan_config: FileScanConfig) -> CsvExecBuilder { + CsvExecBuilder::new(file_scan_config) + } + + /// Ref to the base configs + pub fn base_config(&self) -> &FileScanConfig { + &self.base_config + } + + fn file_scan_config(&self) -> FileScanConfig { + self.inner + .data_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + fn csv_source(&self) -> CsvSource { + let source = self.file_scan_config(); + source + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + /// true if the first line of each file is a header + pub fn has_header(&self) -> bool { + self.csv_source().has_header() + } + + /// Specifies whether newlines in (quoted) values are supported. + /// + /// Parsing newlines in quoted values may be affected by execution behaviour such as + /// parallel file scanning. Setting this to `true` ensures that newlines in values are + /// parsed successfully, which may reduce performance. + /// + /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. + pub fn newlines_in_values(&self) -> bool { + let source = self.file_scan_config(); + source.newlines_in_values() + } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + constraints: Constraints, + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + fn with_file_groups(mut self, file_groups: Vec>) -> Self { + self.base_config.file_groups = file_groups.clone(); + let mut file_source = self.file_scan_config(); + file_source = file_source.with_file_groups(file_groups); + self.inner = self.inner.with_data_source(Arc::new(file_source)); + self + } +} + +#[allow(unused, deprecated)] +impl DisplayAs for CsvExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + self.inner.fmt_as(t, f) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for CsvExec { + fn name(&self) -> &'static str { + "CsvExec" + } + + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + self.inner.properties() + } + + fn children(&self) -> Vec<&Arc> { + // this is a leaf node and has no children + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + /// Redistribute files across partitions according to their size + /// See comments on `FileGroupPartitioner` for more detail. + /// + /// Return `None` if can't get repartitioned (empty, compressed file, or `newlines_in_values` set). + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> Result>> { + self.inner.repartitioned(target_partitions, config) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + self.inner.execute(partition, context) + } + + fn statistics(&self) -> Result { + self.inner.statistics() + } + + fn metrics(&self) -> Option { + self.inner.metrics() + } + + fn fetch(&self) -> Option { + self.inner.fetch() + } + + fn with_fetch(&self, limit: Option) -> Option> { + self.inner.with_fetch(limit) + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + self.inner.try_swapping_with_projection(projection) + } +} + +/// A Config for [`CsvOpener`] +/// +/// # Example: create a `DataSourceExec` for CSV +/// ``` +/// # use std::sync::Arc; +/// # use arrow::datatypes::Schema; +/// # use datafusion_datasource::file_scan_config::FileScanConfig; +/// # use datafusion_datasource::PartitionedFile; +/// # use datafusion_datasource_csv::source::CsvSource; +/// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # use datafusion_datasource::source::DataSourceExec; +/// +/// # let object_store_url = ObjectStoreUrl::local_filesystem(); +/// # let file_schema = Arc::new(Schema::empty()); +/// +/// let source = Arc::new(CsvSource::new( +/// true, +/// b',', +/// b'"', +/// ) +/// .with_terminator(Some(b'#') +/// )); +/// // Create a DataSourceExec for reading the first 100MB of `file1.csv` +/// let file_scan_config = FileScanConfig::new(object_store_url, file_schema, source) +/// .with_file(PartitionedFile::new("file1.csv", 100*1024*1024)) +/// .with_newlines_in_values(true); // The file contains newlines in values; +/// let exec = file_scan_config.build(); +/// ``` +#[derive(Debug, Clone, Default)] +pub struct CsvSource { + batch_size: Option, + file_schema: Option, + file_projection: Option>, + pub(crate) has_header: bool, + delimiter: u8, + quote: u8, + terminator: Option, + escape: Option, + comment: Option, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} + +impl CsvSource { + /// Returns a [`CsvSource`] + pub fn new(has_header: bool, delimiter: u8, quote: u8) -> Self { + Self { + has_header, + delimiter, + quote, + ..Self::default() + } + } + + /// true if the first line of each file is a header + pub fn has_header(&self) -> bool { + self.has_header + } + /// A column delimiter + pub fn delimiter(&self) -> u8 { + self.delimiter + } + + /// The quote character + pub fn quote(&self) -> u8 { + self.quote + } + + /// The line terminator + pub fn terminator(&self) -> Option { + self.terminator + } + + /// Lines beginning with this byte are ignored. + pub fn comment(&self) -> Option { + self.comment + } + + /// The escape character + pub fn escape(&self) -> Option { + self.escape + } + + /// Initialize a CsvSource with escape + pub fn with_escape(&self, escape: Option) -> Self { + let mut conf = self.clone(); + conf.escape = escape; + conf + } + + /// Initialize a CsvSource with terminator + pub fn with_terminator(&self, terminator: Option) -> Self { + let mut conf = self.clone(); + conf.terminator = terminator; + conf + } + + /// Initialize a CsvSource with comment + pub fn with_comment(&self, comment: Option) -> Self { + let mut conf = self.clone(); + conf.comment = comment; + conf + } +} + +impl CsvSource { + fn open(&self, reader: R) -> Result> { + Ok(self.builder().build(reader)?) + } + + fn builder(&self) -> csv::ReaderBuilder { + let mut builder = csv::ReaderBuilder::new(Arc::clone( + self.file_schema + .as_ref() + .expect("Schema must be set before initializing builder"), + )) + .with_delimiter(self.delimiter) + .with_batch_size( + self.batch_size + .expect("Batch size must be set before initializing builder"), + ) + .with_header(self.has_header) + .with_quote(self.quote); + if let Some(terminator) = self.terminator { + builder = builder.with_terminator(terminator); + } + if let Some(proj) = &self.file_projection { + builder = builder.with_projection(proj.clone()); + } + if let Some(escape) = self.escape { + builder = builder.with_escape(escape) + } + if let Some(comment) = self.comment { + builder = builder.with_comment(comment); + } + + builder + } +} + +/// A [`FileOpener`] that opens a CSV file and yields a [`FileOpenFuture`] +pub struct CsvOpener { + config: Arc, + file_compression_type: FileCompressionType, + object_store: Arc, +} + +impl CsvOpener { + /// Returns a [`CsvOpener`] + pub fn new( + config: Arc, + file_compression_type: FileCompressionType, + object_store: Arc, + ) -> Self { + Self { + config, + file_compression_type, + object_store, + } + } +} + +impl FileSource for CsvSource { + fn create_file_opener( + &self, + object_store: Arc, + base_config: &FileScanConfig, + _partition: usize, + ) -> Arc { + Arc::new(CsvOpener { + config: Arc::new(self.clone()), + file_compression_type: base_config.file_compression_type, + object_store, + }) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) + } + + fn with_schema(&self, schema: SchemaRef) -> Arc { + let mut conf = self.clone(); + conf.file_schema = Some(schema); + Arc::new(conf) + } + + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, config: &FileScanConfig) -> Arc { + let mut conf = self.clone(); + conf.file_projection = config.file_column_projection_indices(); + Arc::new(conf) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + fn statistics(&self) -> Result { + let statistics = &self.projected_statistics; + Ok(statistics + .clone() + .expect("projected_statistics must be set")) + } + fn file_type(&self) -> &str { + "csv" + } + fn fmt_extra(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, ", has_header={}", self.has_header) + } +} + +impl FileOpener for CsvOpener { + /// Open a partitioned CSV file. + /// + /// If `file_meta.range` is `None`, the entire file is opened. + /// If `file_meta.range` is `Some(FileRange {start, end})`, this signifies that the partition + /// corresponds to the byte range [start, end) within the file. + /// + /// Note: `start` or `end` might be in the middle of some lines. In such cases, the following rules + /// are applied to determine which lines to read: + /// 1. The first line of the partition is the line in which the index of the first character >= `start`. + /// 2. The last line of the partition is the line in which the byte at position `end - 1` resides. + /// + /// Examples: + /// Consider the following partitions enclosed by braces `{}`: + /// + /// {A,1,2,3,4,5,6,7,8,9\n + /// A,1,2,3,4,5,6,7,8,9\n} + /// A,1,2,3,4,5,6,7,8,9\n + /// The lines read would be: [0, 1] + /// + /// A,{1,2,3,4,5,6,7,8,9\n + /// A,1,2,3,4,5,6,7,8,9\n + /// A},1,2,3,4,5,6,7,8,9\n + /// The lines read would be: [1, 2] + fn open(&self, file_meta: FileMeta) -> Result { + // `self.config.has_header` controls whether to skip reading the 1st line header + // If the .csv file is read in parallel and this `CsvOpener` is only reading some middle + // partition, then don't skip first line + let mut csv_has_header = self.config.has_header; + if let Some(FileRange { start, .. }) = file_meta.range { + if start != 0 { + csv_has_header = false; + } + } + + let config = CsvSource { + has_header: csv_has_header, + ..(*self.config).clone() + }; + + let file_compression_type = self.file_compression_type.to_owned(); + + if file_meta.range.is_some() { + assert!( + !file_compression_type.is_compressed(), + "Reading compressed .csv in parallel is not supported" + ); + } + + let store = Arc::clone(&self.object_store); + let terminator = self.config.terminator; + + Ok(Box::pin(async move { + // Current partition contains bytes [start_byte, end_byte) (might contain incomplete lines at boundaries) + + let calculated_range = + calculate_range(&file_meta, &store, terminator).await?; + + let range = match calculated_range { + RangeCalculation::Range(None) => None, + RangeCalculation::Range(Some(range)) => Some(range.into()), + RangeCalculation::TerminateEarly => { + return Ok( + futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() + ) + } + }; + + let options = GetOptions { + range, + ..Default::default() + }; + + let result = store.get_opts(file_meta.location(), options).await?; + + match result.payload { + GetResultPayload::File(mut file, _) => { + let is_whole_file_scanned = file_meta.range.is_none(); + let decoder = if is_whole_file_scanned { + // Don't seek if no range as breaks FIFO files + file_compression_type.convert_read(file)? + } else { + file.seek(SeekFrom::Start(result.range.start as _))?; + file_compression_type.convert_read( + file.take((result.range.end - result.range.start) as u64), + )? + }; + + Ok(futures::stream::iter(config.open(decoder)?).boxed()) + } + GetResultPayload::Stream(s) => { + let decoder = config.builder().build_decoder(); + let s = s.map_err(DataFusionError::from); + let input = file_compression_type.convert_stream(s.boxed())?.fuse(); + + Ok(deserialize_stream( + input, + DecoderDeserializer::new(CsvDecoder::new(decoder)), + )) + } + } + })) + } +} + +pub async fn plan_to_csv( + task_ctx: Arc, + plan: Arc, + path: impl AsRef, +) -> Result<()> { + let path = path.as_ref(); + let parsed = ListingTableUrl::parse(path)?; + let object_store_url = parsed.object_store(); + let store = task_ctx.runtime_env().object_store(&object_store_url)?; + let mut join_set = JoinSet::new(); + for i in 0..plan.output_partitioning().partition_count() { + let storeref = Arc::clone(&store); + let plan: Arc = Arc::clone(&plan); + let filename = format!("{}/part-{i}.csv", parsed.prefix()); + let file = object_store::path::Path::parse(filename)?; + + let mut stream = plan.execute(i, Arc::clone(&task_ctx))?; + join_set.spawn(async move { + let mut buf_writer = BufWriter::new(storeref, file.clone()); + let mut buffer = Vec::with_capacity(1024); + //only write headers on first iteration + let mut write_headers = true; + while let Some(batch) = stream.next().await.transpose()? { + let mut writer = csv::WriterBuilder::new() + .with_header(write_headers) + .build(buffer); + writer.write(&batch)?; + buffer = writer.into_inner(); + buf_writer.write_all(&buffer).await?; + buffer.clear(); + //prevent writing headers more than once + write_headers = false; + } + buf_writer.shutdown().await.map_err(DataFusionError::from) + }); + } + + while let Some(result) = join_set.join_next().await { + match result { + Ok(res) => res?, // propagate DataFusion error + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + } + + Ok(()) +} diff --git a/datafusion/datasource-json/Cargo.toml b/datafusion/datasource-json/Cargo.toml new file mode 100644 index 000000000000..78547c592ba3 --- /dev/null +++ b/datafusion/datasource-json/Cargo.toml @@ -0,0 +1,56 @@ +# 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. + +[package] +name = "datafusion-datasource-json" +description = "datafusion-datasource-json" +authors.workspace = true +edition.workspace = true +homepage.workspace = true +license.workspace = true +readme.workspace = true +repository.workspace = true +rust-version.workspace = true +version.workspace = true + +[package.metadata.docs.rs] +all-features = true + +[dependencies] +arrow = { workspace = true } +async-trait = { workspace = true } +bytes = { workspace = true } +datafusion-catalog = { workspace = true } +datafusion-common = { workspace = true, features = ["object_store"] } +datafusion-common-runtime = { workspace = true } +datafusion-datasource = { workspace = true } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-plan = { workspace = true } +futures = { workspace = true } +object_store = { workspace = true } +serde_json = { workspace = true } +tokio = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_datasource_json" +path = "src/mod.rs" diff --git a/datafusion/datasource-json/LICENSE.txt b/datafusion/datasource-json/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/datasource-json/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/datasource-json/NOTICE.txt b/datafusion/datasource-json/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/datasource-json/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/datasource-json/README.md b/datafusion/datasource-json/README.md new file mode 100644 index 000000000000..64181814736d --- /dev/null +++ b/datafusion/datasource-json/README.md @@ -0,0 +1,26 @@ + + +# DataFusion datasource + +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that defines a JSON based file source. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/datasource-json/src/file_format.rs b/datafusion/datasource-json/src/file_format.rs new file mode 100644 index 000000000000..71106c316b25 --- /dev/null +++ b/datafusion/datasource-json/src/file_format.rs @@ -0,0 +1,414 @@ +// 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. + +//! [`JsonFormat`]: Line delimited JSON [`FileFormat`] abstractions + +use std::any::Any; +use std::collections::HashMap; +use std::fmt; +use std::fmt::Debug; +use std::io::BufReader; +use std::sync::Arc; + +use arrow::array::RecordBatch; +use arrow::datatypes::{Schema, SchemaRef}; +use arrow::error::ArrowError; +use arrow::json; +use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; +use datafusion_catalog::Session; +use datafusion_common::config::{ConfigField, ConfigFileType, JsonOptions}; +use datafusion_common::file_options::json_writer::JsonWriterOptions; +use datafusion_common::{ + not_impl_err, GetExt, Result, Statistics, DEFAULT_JSON_EXTENSION, +}; +use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::decoder::Decoder; +use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_compression_type::FileCompressionType; +use datafusion_datasource::file_format::{ + FileFormat, FileFormatFactory, DEFAULT_SCHEMA_INFER_MAX_RECORD, +}; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig}; +use datafusion_datasource::write::demux::DemuxedStreamReceiver; +use datafusion_datasource::write::orchestration::spawn_writer_tasks_and_join; +use datafusion_datasource::write::BatchSerializer; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_expr::dml::InsertOp; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; + +use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; + +use crate::source::JsonSource; + +#[derive(Default)] +/// Factory struct used to create [JsonFormat] +pub struct JsonFormatFactory { + /// the options carried by format factory + pub options: Option, +} + +impl JsonFormatFactory { + /// Creates an instance of [JsonFormatFactory] + pub fn new() -> Self { + Self { options: None } + } + + /// Creates an instance of [JsonFormatFactory] with customized default options + pub fn new_with_options(options: JsonOptions) -> Self { + Self { + options: Some(options), + } + } +} + +impl FileFormatFactory for JsonFormatFactory { + fn create( + &self, + state: &dyn Session, + format_options: &HashMap, + ) -> Result> { + let json_options = match &self.options { + None => { + let mut table_options = state.default_table_options(); + table_options.set_config_format(ConfigFileType::JSON); + table_options.alter_with_string_hash_map(format_options)?; + table_options.json + } + Some(json_options) => { + let mut json_options = json_options.clone(); + for (k, v) in format_options { + json_options.set(k, v)?; + } + json_options + } + }; + + Ok(Arc::new(JsonFormat::default().with_options(json_options))) + } + + fn default(&self) -> Arc { + Arc::new(JsonFormat::default()) + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +impl GetExt for JsonFormatFactory { + fn get_ext(&self) -> String { + // Removes the dot, i.e. ".parquet" -> "parquet" + DEFAULT_JSON_EXTENSION[1..].to_string() + } +} + +impl Debug for JsonFormatFactory { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("JsonFormatFactory") + .field("options", &self.options) + .finish() + } +} + +/// New line delimited JSON `FileFormat` implementation. +#[derive(Debug, Default)] +pub struct JsonFormat { + options: JsonOptions, +} + +impl JsonFormat { + /// Set JSON options + pub fn with_options(mut self, options: JsonOptions) -> Self { + self.options = options; + self + } + + /// Retrieve JSON options + pub fn options(&self) -> &JsonOptions { + &self.options + } + + /// Set a limit in terms of records to scan to infer the schema + /// - defaults to `DEFAULT_SCHEMA_INFER_MAX_RECORD` + pub fn with_schema_infer_max_rec(mut self, max_rec: usize) -> Self { + self.options.schema_infer_max_rec = Some(max_rec); + self + } + + /// Set a [`FileCompressionType`] of JSON + /// - defaults to `FileCompressionType::UNCOMPRESSED` + pub fn with_file_compression_type( + mut self, + file_compression_type: FileCompressionType, + ) -> Self { + self.options.compression = file_compression_type.into(); + self + } +} + +#[async_trait] +impl FileFormat for JsonFormat { + fn as_any(&self) -> &dyn Any { + self + } + + fn get_ext(&self) -> String { + JsonFormatFactory::new().get_ext() + } + + fn get_ext_with_compression( + &self, + file_compression_type: &FileCompressionType, + ) -> Result { + let ext = self.get_ext(); + Ok(format!("{}{}", ext, file_compression_type.get_ext())) + } + + async fn infer_schema( + &self, + _state: &dyn Session, + store: &Arc, + objects: &[ObjectMeta], + ) -> Result { + let mut schemas = Vec::new(); + let mut records_to_read = self + .options + .schema_infer_max_rec + .unwrap_or(DEFAULT_SCHEMA_INFER_MAX_RECORD); + let file_compression_type = FileCompressionType::from(self.options.compression); + for object in objects { + let mut take_while = || { + let should_take = records_to_read > 0; + if should_take { + records_to_read -= 1; + } + should_take + }; + + let r = store.as_ref().get(&object.location).await?; + let schema = match r.payload { + GetResultPayload::File(file, _) => { + let decoder = file_compression_type.convert_read(file)?; + let mut reader = BufReader::new(decoder); + let iter = ValueIter::new(&mut reader, None); + infer_json_schema_from_iterator(iter.take_while(|_| take_while()))? + } + GetResultPayload::Stream(_) => { + let data = r.bytes().await?; + let decoder = file_compression_type.convert_read(data.reader())?; + let mut reader = BufReader::new(decoder); + let iter = ValueIter::new(&mut reader, None); + infer_json_schema_from_iterator(iter.take_while(|_| take_while()))? + } + }; + + schemas.push(schema); + if records_to_read == 0 { + break; + } + } + + let schema = Schema::try_merge(schemas)?; + Ok(Arc::new(schema)) + } + + async fn infer_stats( + &self, + _state: &dyn Session, + _store: &Arc, + table_schema: SchemaRef, + _object: &ObjectMeta, + ) -> Result { + Ok(Statistics::new_unknown(&table_schema)) + } + + async fn create_physical_plan( + &self, + _state: &dyn Session, + mut conf: FileScanConfig, + _filters: Option<&Arc>, + ) -> Result> { + let source = Arc::new(JsonSource::new()); + conf.file_compression_type = FileCompressionType::from(self.options.compression); + Ok(conf.with_source(source).build()) + } + + async fn create_writer_physical_plan( + &self, + input: Arc, + _state: &dyn Session, + conf: FileSinkConfig, + order_requirements: Option, + ) -> Result> { + if conf.insert_op != InsertOp::Append { + return not_impl_err!("Overwrites are not implemented yet for Json"); + } + + let writer_options = JsonWriterOptions::try_from(&self.options)?; + + let sink = Arc::new(JsonSink::new(conf, writer_options)); + + Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) + } + + fn file_source(&self) -> Arc { + Arc::new(JsonSource::default()) + } +} + +impl Default for JsonSerializer { + fn default() -> Self { + Self::new() + } +} + +/// Define a struct for serializing Json records to a stream +pub struct JsonSerializer {} + +impl JsonSerializer { + /// Constructor for the JsonSerializer object + pub fn new() -> Self { + Self {} + } +} + +impl BatchSerializer for JsonSerializer { + fn serialize(&self, batch: RecordBatch, _initial: bool) -> Result { + let mut buffer = Vec::with_capacity(4096); + let mut writer = json::LineDelimitedWriter::new(&mut buffer); + writer.write(&batch)?; + Ok(Bytes::from(buffer)) + } +} + +/// Implements [`DataSink`] for writing to a Json file. +pub struct JsonSink { + /// Config options for writing data + config: FileSinkConfig, + /// Writer options for underlying Json writer + writer_options: JsonWriterOptions, +} + +impl Debug for JsonSink { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("JsonSink").finish() + } +} + +impl DisplayAs for JsonSink { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "JsonSink(file_groups=",)?; + FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; + write!(f, ")") + } + } + } +} + +impl JsonSink { + /// Create from config. + pub fn new(config: FileSinkConfig, writer_options: JsonWriterOptions) -> Self { + Self { + config, + writer_options, + } + } + + /// Retrieve the writer options + pub fn writer_options(&self) -> &JsonWriterOptions { + &self.writer_options + } +} + +#[async_trait] +impl FileSink for JsonSink { + fn config(&self) -> &FileSinkConfig { + &self.config + } + + async fn spawn_writer_tasks_and_join( + &self, + context: &Arc, + demux_task: SpawnedTask>, + file_stream_rx: DemuxedStreamReceiver, + object_store: Arc, + ) -> Result { + let serializer = Arc::new(JsonSerializer::new()) as _; + spawn_writer_tasks_and_join( + context, + serializer, + self.writer_options.compression.into(), + object_store, + demux_task, + file_stream_rx, + ) + .await + } +} + +#[async_trait] +impl DataSink for JsonSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> &SchemaRef { + self.config.output_schema() + } + + async fn write_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + FileSink::write_all(self, data, context).await + } +} + +#[derive(Debug)] +pub struct JsonDecoder { + inner: json::reader::Decoder, +} + +impl JsonDecoder { + pub fn new(decoder: json::reader::Decoder) -> Self { + Self { inner: decoder } + } +} + +impl Decoder for JsonDecoder { + fn decode(&mut self, buf: &[u8]) -> Result { + self.inner.decode(buf) + } + + fn flush(&mut self) -> Result, ArrowError> { + self.inner.flush() + } + + fn can_flush_early(&self) -> bool { + false + } +} diff --git a/datafusion/datasource-json/src/mod.rs b/datafusion/datasource-json/src/mod.rs new file mode 100644 index 000000000000..35dabfa109fc --- /dev/null +++ b/datafusion/datasource-json/src/mod.rs @@ -0,0 +1,21 @@ +// 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. + +pub mod file_format; +pub mod source; + +pub use file_format::*; diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs new file mode 100644 index 000000000000..249593587b82 --- /dev/null +++ b/datafusion/datasource-json/src/source.rs @@ -0,0 +1,440 @@ +// 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. + +//! Execution plan for reading line-delimited JSON files + +use std::any::Any; +use std::io::{BufReader, Read, Seek, SeekFrom}; +use std::sync::Arc; +use std::task::Poll; + +use crate::file_format::JsonDecoder; + +use datafusion_common::error::{DataFusionError, Result}; +use datafusion_datasource::decoder::{deserialize_stream, DecoderDeserializer}; +use datafusion_datasource::file_compression_type::FileCompressionType; +use datafusion_datasource::file_meta::FileMeta; +use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; +use datafusion_datasource::{ + calculate_range, ListingTableUrl, PartitionedFile, RangeCalculation, +}; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; + +use arrow::json::ReaderBuilder; +use arrow::{datatypes::SchemaRef, json}; +use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::source::DataSourceExec; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; + +use futures::{StreamExt, TryStreamExt}; +use object_store::buffered::BufWriter; +use object_store::{GetOptions, GetResultPayload, ObjectStore}; +use tokio::io::AsyncWriteExt; +use tokio::task::JoinSet; + +/// Execution plan for scanning NdJson data source +#[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] +pub struct NdJsonExec { + inner: DataSourceExec, + base_config: FileScanConfig, + file_compression_type: FileCompressionType, +} + +#[allow(unused, deprecated)] +impl NdJsonExec { + /// Create a new JSON reader execution plan provided base configurations + pub fn new( + base_config: FileScanConfig, + file_compression_type: FileCompressionType, + ) -> Self { + let ( + projected_schema, + projected_constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); + let cache = Self::compute_properties( + projected_schema, + &projected_output_ordering, + projected_constraints, + &base_config, + ); + + let json = JsonSource::default(); + let base_config = base_config + .with_file_compression_type(file_compression_type) + .with_source(Arc::new(json)); + + Self { + inner: DataSourceExec::new(Arc::new(base_config.clone())), + file_compression_type: base_config.file_compression_type, + base_config, + } + } + + /// Ref to the base configs + pub fn base_config(&self) -> &FileScanConfig { + &self.base_config + } + + /// Ref to file compression type + pub fn file_compression_type(&self) -> &FileCompressionType { + &self.file_compression_type + } + + fn file_scan_config(&self) -> FileScanConfig { + self.inner + .data_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + fn json_source(&self) -> JsonSource { + let source = self.file_scan_config(); + source + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + constraints: Constraints, + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + fn with_file_groups(mut self, file_groups: Vec>) -> Self { + self.base_config.file_groups = file_groups.clone(); + let mut file_source = self.file_scan_config(); + file_source = file_source.with_file_groups(file_groups); + self.inner = self.inner.with_data_source(Arc::new(file_source)); + self + } +} + +#[allow(unused, deprecated)] +impl DisplayAs for NdJsonExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + self.inner.fmt_as(t, f) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for NdJsonExec { + fn name(&self) -> &'static str { + "NdJsonExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + fn properties(&self) -> &PlanProperties { + self.inner.properties() + } + + fn children(&self) -> Vec<&Arc> { + Vec::new() + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn repartitioned( + &self, + target_partitions: usize, + config: &datafusion_common::config::ConfigOptions, + ) -> Result>> { + self.inner.repartitioned(target_partitions, config) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + self.inner.execute(partition, context) + } + + fn statistics(&self) -> Result { + self.inner.statistics() + } + + fn metrics(&self) -> Option { + self.inner.metrics() + } + + fn fetch(&self) -> Option { + self.inner.fetch() + } + + fn with_fetch(&self, limit: Option) -> Option> { + self.inner.with_fetch(limit) + } +} + +/// A [`FileOpener`] that opens a JSON file and yields a [`FileOpenFuture`] +pub struct JsonOpener { + batch_size: usize, + projected_schema: SchemaRef, + file_compression_type: FileCompressionType, + object_store: Arc, +} + +impl JsonOpener { + /// Returns a [`JsonOpener`] + pub fn new( + batch_size: usize, + projected_schema: SchemaRef, + file_compression_type: FileCompressionType, + object_store: Arc, + ) -> Self { + Self { + batch_size, + projected_schema, + file_compression_type, + object_store, + } + } +} + +/// JsonSource holds the extra configuration that is necessary for [`JsonOpener`] +#[derive(Clone, Default)] +pub struct JsonSource { + batch_size: Option, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} + +impl JsonSource { + /// Initialize a JsonSource with default values + pub fn new() -> Self { + Self::default() + } +} + +impl FileSource for JsonSource { + fn create_file_opener( + &self, + object_store: Arc, + base_config: &FileScanConfig, + _partition: usize, + ) -> Arc { + Arc::new(JsonOpener { + batch_size: self + .batch_size + .expect("Batch size must set before creating opener"), + projected_schema: base_config.projected_file_schema(), + file_compression_type: base_config.file_compression_type, + object_store, + }) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(Self { ..self.clone() }) + } + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn statistics(&self) -> Result { + let statistics = &self.projected_statistics; + Ok(statistics + .clone() + .expect("projected_statistics must be set to call")) + } + + fn file_type(&self) -> &str { + "json" + } +} + +impl FileOpener for JsonOpener { + /// Open a partitioned NDJSON file. + /// + /// If `file_meta.range` is `None`, the entire file is opened. + /// Else `file_meta.range` is `Some(FileRange{start, end})`, which corresponds to the byte range [start, end) within the file. + /// + /// Note: `start` or `end` might be in the middle of some lines. In such cases, the following rules + /// are applied to determine which lines to read: + /// 1. The first line of the partition is the line in which the index of the first character >= `start`. + /// 2. The last line of the partition is the line in which the byte at position `end - 1` resides. + fn open(&self, file_meta: FileMeta) -> Result { + let store = Arc::clone(&self.object_store); + let schema = Arc::clone(&self.projected_schema); + let batch_size = self.batch_size; + let file_compression_type = self.file_compression_type.to_owned(); + + Ok(Box::pin(async move { + let calculated_range = calculate_range(&file_meta, &store, None).await?; + + let range = match calculated_range { + RangeCalculation::Range(None) => None, + RangeCalculation::Range(Some(range)) => Some(range.into()), + RangeCalculation::TerminateEarly => { + return Ok( + futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() + ) + } + }; + + let options = GetOptions { + range, + ..Default::default() + }; + + let result = store.get_opts(file_meta.location(), options).await?; + + match result.payload { + GetResultPayload::File(mut file, _) => { + let bytes = match file_meta.range { + None => file_compression_type.convert_read(file)?, + Some(_) => { + file.seek(SeekFrom::Start(result.range.start as _))?; + let limit = result.range.end - result.range.start; + file_compression_type.convert_read(file.take(limit as u64))? + } + }; + + let reader = ReaderBuilder::new(schema) + .with_batch_size(batch_size) + .build(BufReader::new(bytes))?; + + Ok(futures::stream::iter(reader).boxed()) + } + GetResultPayload::Stream(s) => { + let s = s.map_err(DataFusionError::from); + + let decoder = ReaderBuilder::new(schema) + .with_batch_size(batch_size) + .build_decoder()?; + let input = file_compression_type.convert_stream(s.boxed())?.fuse(); + + Ok(deserialize_stream( + input, + DecoderDeserializer::new(JsonDecoder::new(decoder)), + )) + } + } + })) + } +} + +pub async fn plan_to_json( + task_ctx: Arc, + plan: Arc, + path: impl AsRef, +) -> Result<()> { + let path = path.as_ref(); + let parsed = ListingTableUrl::parse(path)?; + let object_store_url = parsed.object_store(); + let store = task_ctx.runtime_env().object_store(&object_store_url)?; + let mut join_set = JoinSet::new(); + for i in 0..plan.output_partitioning().partition_count() { + let storeref = Arc::clone(&store); + let plan: Arc = Arc::clone(&plan); + let filename = format!("{}/part-{i}.json", parsed.prefix()); + let file = object_store::path::Path::parse(filename)?; + + let mut stream = plan.execute(i, Arc::clone(&task_ctx))?; + join_set.spawn(async move { + let mut buf_writer = BufWriter::new(storeref, file.clone()); + + let mut buffer = Vec::with_capacity(1024); + while let Some(batch) = stream.next().await.transpose()? { + let mut writer = json::LineDelimitedWriter::new(buffer); + writer.write(&batch)?; + buffer = writer.into_inner(); + buf_writer.write_all(&buffer).await?; + buffer.clear(); + } + + buf_writer.shutdown().await.map_err(DataFusionError::from) + }); + } + + while let Some(result) = join_set.join_next().await { + match result { + Ok(res) => res?, // propagate DataFusion error + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + } + + Ok(()) +} diff --git a/datafusion/datasource-parquet/Cargo.toml b/datafusion/datasource-parquet/Cargo.toml new file mode 100644 index 000000000000..d84487bba9bd --- /dev/null +++ b/datafusion/datasource-parquet/Cargo.toml @@ -0,0 +1,65 @@ +# 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. + +[package] +name = "datafusion-datasource-parquet" +description = "datafusion-datasource-parquet" +authors.workspace = true +edition.workspace = true +homepage.workspace = true +license.workspace = true +readme.workspace = true +repository.workspace = true +rust-version.workspace = true +version.workspace = true + +[package.metadata.docs.rs] +all-features = true + +[dependencies] +arrow = { workspace = true } +async-trait = { workspace = true } +bytes = { workspace = true } +datafusion-catalog = { workspace = true } +datafusion-common = { workspace = true, features = ["object_store", "parquet"] } +datafusion-common-runtime = { workspace = true } +datafusion-datasource = { workspace = true, features = ["parquet"] } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-functions-aggregate = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-optimizer = { workspace = true } +datafusion-physical-plan = { workspace = true } +futures = { workspace = true } +itertools = { workspace = true } +log = { workspace = true } +object_store = { workspace = true } +parking_lot = { workspace = true } +parquet = { workspace = true } +rand = { workspace = true } +tokio = { workspace = true, features = ["fs"] } + +[dev-dependencies] +chrono = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_datasource_parquet" +path = "src/mod.rs" diff --git a/datafusion/datasource-parquet/LICENSE.txt b/datafusion/datasource-parquet/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/datasource-parquet/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/datasource-parquet/NOTICE.txt b/datafusion/datasource-parquet/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/datasource-parquet/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/datasource-parquet/README.md b/datafusion/datasource-parquet/README.md new file mode 100644 index 000000000000..abcdd5ab1340 --- /dev/null +++ b/datafusion/datasource-parquet/README.md @@ -0,0 +1,26 @@ + + +# DataFusion datasource + +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that defines a Parquet based file source. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs similarity index 99% rename from datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs rename to datafusion/datasource-parquet/src/access_plan.rs index d30549708bbd..0c30f3ff85b6 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -35,7 +35,7 @@ use parquet::file::metadata::RowGroupMetaData; /// /// ```rust /// # use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; -/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; +/// # use datafusion_datasource_parquet::ParquetAccessPlan; /// // Default to scan all row groups /// let mut access_plan = ParquetAccessPlan::new_all(4); /// access_plan.skip(0); // skip row group diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs new file mode 100644 index 000000000000..78f78aad4c15 --- /dev/null +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -0,0 +1,1416 @@ +// 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. + +//! [`ParquetFormat`]: Parquet [`FileFormat`] abstractions + +use std::any::Any; +use std::fmt; +use std::fmt::Debug; +use std::ops::Range; +use std::sync::Arc; + +use arrow::array::RecordBatch; +use arrow::datatypes::{Fields, Schema, SchemaRef}; +use datafusion_datasource::file_compression_type::FileCompressionType; +use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig}; +use datafusion_datasource::write::{create_writer, get_writer_schema, SharedBuffer}; + +use datafusion_datasource::file_format::{ + FileFormat, FileFormatFactory, FilePushdownSupport, +}; +use datafusion_datasource::write::demux::DemuxedStreamReceiver; + +use arrow::compute::sum; +use arrow::datatypes::{DataType, Field, FieldRef}; +use datafusion_catalog::Session; +use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; +use datafusion_common::parsers::CompressionTypeVariant; +use datafusion_common::stats::Precision; +use datafusion_common::{ + internal_datafusion_err, internal_err, not_impl_err, ColumnStatistics, + DataFusionError, GetExt, Result, DEFAULT_PARQUET_EXTENSION, +}; +use datafusion_common::{HashMap, Statistics}; +use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_expr::dml::InsertOp; +use datafusion_expr::Expr; +use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::Accumulator; + +use async_trait::async_trait; +use bytes::Bytes; +use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; +use futures::future::BoxFuture; +use futures::{FutureExt, StreamExt, TryStreamExt}; +use log::debug; +use object_store::buffered::BufWriter; +use object_store::path::Path; +use object_store::{ObjectMeta, ObjectStore}; +use parquet::arrow::arrow_reader::statistics::StatisticsConverter; +use parquet::arrow::arrow_writer::{ + compute_leaves, get_column_writers, ArrowColumnChunk, ArrowColumnWriter, + ArrowLeafColumn, ArrowWriterOptions, +}; +use parquet::arrow::async_reader::MetadataFetch; +use parquet::arrow::{parquet_to_arrow_schema, ArrowSchemaConverter, AsyncArrowWriter}; +use parquet::errors::ParquetError; +use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; +use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder}; +use parquet::file::writer::SerializedFileWriter; +use parquet::format::FileMetaData; +use tokio::io::{AsyncWrite, AsyncWriteExt}; +use tokio::sync::mpsc::{self, Receiver, Sender}; +use tokio::task::JoinSet; + +use crate::can_expr_be_pushed_down_with_schemas; +use crate::source::ParquetSource; + +/// Initial writing buffer size. Note this is just a size hint for efficiency. It +/// will grow beyond the set value if needed. +const INITIAL_BUFFER_BYTES: usize = 1048576; + +/// When writing parquet files in parallel, if the buffered Parquet data exceeds +/// this size, it is flushed to object store +const BUFFER_FLUSH_BYTES: usize = 1024000; + +#[derive(Default)] +/// Factory struct used to create [ParquetFormat] +pub struct ParquetFormatFactory { + /// inner options for parquet + pub options: Option, +} + +impl ParquetFormatFactory { + /// Creates an instance of [ParquetFormatFactory] + pub fn new() -> Self { + Self { options: None } + } + + /// Creates an instance of [ParquetFormatFactory] with customized default options + pub fn new_with_options(options: TableParquetOptions) -> Self { + Self { + options: Some(options), + } + } +} + +impl FileFormatFactory for ParquetFormatFactory { + fn create( + &self, + state: &dyn Session, + format_options: &std::collections::HashMap, + ) -> Result> { + let parquet_options = match &self.options { + None => { + let mut table_options = state.default_table_options(); + table_options.set_config_format(ConfigFileType::PARQUET); + table_options.alter_with_string_hash_map(format_options)?; + table_options.parquet + } + Some(parquet_options) => { + let mut parquet_options = parquet_options.clone(); + for (k, v) in format_options { + parquet_options.set(k, v)?; + } + parquet_options + } + }; + + Ok(Arc::new( + ParquetFormat::default().with_options(parquet_options), + )) + } + + fn default(&self) -> Arc { + Arc::new(ParquetFormat::default()) + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +impl GetExt for ParquetFormatFactory { + fn get_ext(&self) -> String { + // Removes the dot, i.e. ".parquet" -> "parquet" + DEFAULT_PARQUET_EXTENSION[1..].to_string() + } +} + +impl Debug for ParquetFormatFactory { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("ParquetFormatFactory") + .field("ParquetFormatFactory", &self.options) + .finish() + } +} +/// The Apache Parquet `FileFormat` implementation +#[derive(Debug, Default)] +pub struct ParquetFormat { + options: TableParquetOptions, +} + +impl ParquetFormat { + /// Construct a new Format with no local overrides + pub fn new() -> Self { + Self::default() + } + + /// Activate statistics based row group level pruning + /// - If `None`, defaults to value on `config_options` + pub fn with_enable_pruning(mut self, enable: bool) -> Self { + self.options.global.pruning = enable; + self + } + + /// Return `true` if pruning is enabled + pub fn enable_pruning(&self) -> bool { + self.options.global.pruning + } + + /// Provide a hint to the size of the file metadata. If a hint is provided + /// the reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. + /// Without a hint, two read are required. One read to fetch the 8-byte parquet footer and then + /// another read to fetch the metadata length encoded in the footer. + /// + /// - If `None`, defaults to value on `config_options` + pub fn with_metadata_size_hint(mut self, size_hint: Option) -> Self { + self.options.global.metadata_size_hint = size_hint; + self + } + + /// Return the metadata size hint if set + pub fn metadata_size_hint(&self) -> Option { + self.options.global.metadata_size_hint + } + + /// Tell the parquet reader to skip any metadata that may be in + /// the file Schema. This can help avoid schema conflicts due to + /// metadata. + /// + /// - If `None`, defaults to value on `config_options` + pub fn with_skip_metadata(mut self, skip_metadata: bool) -> Self { + self.options.global.skip_metadata = skip_metadata; + self + } + + /// Returns `true` if schema metadata will be cleared prior to + /// schema merging. + pub fn skip_metadata(&self) -> bool { + self.options.global.skip_metadata + } + + /// Set Parquet options for the ParquetFormat + pub fn with_options(mut self, options: TableParquetOptions) -> Self { + self.options = options; + self + } + + /// Parquet options + pub fn options(&self) -> &TableParquetOptions { + &self.options + } + + /// Return `true` if should use view types. + /// + /// If this returns true, DataFusion will instruct the parquet reader + /// to read string / binary columns using view `StringView` or `BinaryView` + /// if the table schema specifies those types, regardless of any embedded metadata + /// that may specify an alternate Arrow type. The parquet reader is optimized + /// for reading `StringView` and `BinaryView` and such queries are significantly faster. + /// + /// If this returns false, the parquet reader will read the columns according to the + /// defaults or any embedded Arrow type information. This may result in reading + /// `StringArrays` and then casting to `StringViewArray` which is less efficient. + pub fn force_view_types(&self) -> bool { + self.options.global.schema_force_view_types + } + + /// If true, will use view types. See [`Self::force_view_types`] for details + pub fn with_force_view_types(mut self, use_views: bool) -> Self { + self.options.global.schema_force_view_types = use_views; + self + } + + /// Return `true` if binary types will be read as strings. + /// + /// If this returns true, DataFusion will instruct the parquet reader + /// to read binary columns such as `Binary` or `BinaryView` as the + /// corresponding string type such as `Utf8` or `LargeUtf8`. + /// The parquet reader has special optimizations for `Utf8` and `LargeUtf8` + /// validation, and such queries are significantly faster than reading + /// binary columns and then casting to string columns. + pub fn binary_as_string(&self) -> bool { + self.options.global.binary_as_string + } + + /// If true, will read binary types as strings. See [`Self::binary_as_string`] for details + pub fn with_binary_as_string(mut self, binary_as_string: bool) -> Self { + self.options.global.binary_as_string = binary_as_string; + self + } +} + +/// Clears all metadata (Schema level and field level) on an iterator +/// of Schemas +fn clear_metadata( + schemas: impl IntoIterator, +) -> impl Iterator { + schemas.into_iter().map(|schema| { + let fields = schema + .fields() + .iter() + .map(|field| { + field.as_ref().clone().with_metadata(Default::default()) // clear meta + }) + .collect::(); + Schema::new(fields) + }) +} + +async fn fetch_schema_with_location( + store: &dyn ObjectStore, + file: &ObjectMeta, + metadata_size_hint: Option, +) -> Result<(Path, Schema)> { + let loc_path = file.location.clone(); + let schema = fetch_schema(store, file, metadata_size_hint).await?; + Ok((loc_path, schema)) +} + +#[async_trait] +impl FileFormat for ParquetFormat { + fn as_any(&self) -> &dyn Any { + self + } + + fn get_ext(&self) -> String { + ParquetFormatFactory::new().get_ext() + } + + fn get_ext_with_compression( + &self, + file_compression_type: &FileCompressionType, + ) -> Result { + let ext = self.get_ext(); + match file_compression_type.get_variant() { + CompressionTypeVariant::UNCOMPRESSED => Ok(ext), + _ => internal_err!("Parquet FileFormat does not support compression."), + } + } + + async fn infer_schema( + &self, + state: &dyn Session, + store: &Arc, + objects: &[ObjectMeta], + ) -> Result { + let mut schemas: Vec<_> = futures::stream::iter(objects) + .map(|object| { + fetch_schema_with_location( + store.as_ref(), + object, + self.metadata_size_hint(), + ) + }) + .boxed() // Workaround https://github.com/rust-lang/rust/issues/64552 + .buffered(state.config_options().execution.meta_fetch_concurrency) + .try_collect() + .await?; + + // Schema inference adds fields based the order they are seen + // which depends on the order the files are processed. For some + // object stores (like local file systems) the order returned from list + // is not deterministic. Thus, to ensure deterministic schema inference + // sort the files first. + // https://github.com/apache/datafusion/pull/6629 + schemas.sort_by(|(location1, _), (location2, _)| location1.cmp(location2)); + + let schemas = schemas + .into_iter() + .map(|(_, schema)| schema) + .collect::>(); + + let schema = if self.skip_metadata() { + Schema::try_merge(clear_metadata(schemas)) + } else { + Schema::try_merge(schemas) + }?; + + let schema = if self.binary_as_string() { + transform_binary_to_string(&schema) + } else { + schema + }; + + let schema = if self.force_view_types() { + transform_schema_to_view(&schema) + } else { + schema + }; + + Ok(Arc::new(schema)) + } + + async fn infer_stats( + &self, + _state: &dyn Session, + store: &Arc, + table_schema: SchemaRef, + object: &ObjectMeta, + ) -> Result { + let stats = fetch_statistics( + store.as_ref(), + table_schema, + object, + self.metadata_size_hint(), + ) + .await?; + Ok(stats) + } + + async fn create_physical_plan( + &self, + _state: &dyn Session, + conf: FileScanConfig, + filters: Option<&Arc>, + ) -> Result> { + let mut predicate = None; + let mut metadata_size_hint = None; + + // If enable pruning then combine the filters to build the predicate. + // If disable pruning then set the predicate to None, thus readers + // will not prune data based on the statistics. + if self.enable_pruning() { + if let Some(pred) = filters.cloned() { + predicate = Some(pred); + } + } + if let Some(metadata) = self.metadata_size_hint() { + metadata_size_hint = Some(metadata); + } + + let mut source = ParquetSource::new(self.options.clone()); + + if let Some(predicate) = predicate { + source = source.with_predicate(Arc::clone(&conf.file_schema), predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + source = source.with_metadata_size_hint(metadata_size_hint) + } + Ok(conf.with_source(Arc::new(source)).build()) + } + + async fn create_writer_physical_plan( + &self, + input: Arc, + _state: &dyn Session, + conf: FileSinkConfig, + order_requirements: Option, + ) -> Result> { + if conf.insert_op != InsertOp::Append { + return not_impl_err!("Overwrites are not implemented yet for Parquet"); + } + + let sink = Arc::new(ParquetSink::new(conf, self.options.clone())); + + Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) + } + + fn supports_filters_pushdown( + &self, + file_schema: &Schema, + table_schema: &Schema, + filters: &[&Expr], + ) -> Result { + if !self.options().global.pushdown_filters { + return Ok(FilePushdownSupport::NoSupport); + } + + let all_supported = filters.iter().all(|filter| { + can_expr_be_pushed_down_with_schemas(filter, file_schema, table_schema) + }); + + Ok(if all_supported { + FilePushdownSupport::Supported + } else { + FilePushdownSupport::NotSupportedForFilter + }) + } + + fn file_source(&self) -> Arc { + Arc::new(ParquetSource::default()) + } +} + +/// Coerces the file schema if the table schema uses a view type. +#[cfg(not(target_arch = "wasm32"))] +pub fn coerce_file_schema_to_view_type( + table_schema: &Schema, + file_schema: &Schema, +) -> Option { + let mut transform = false; + let table_fields: HashMap<_, _> = table_schema + .fields + .iter() + .map(|f| { + let dt = f.data_type(); + if dt.equals_datatype(&DataType::Utf8View) + || dt.equals_datatype(&DataType::BinaryView) + { + transform = true; + } + (f.name(), dt) + }) + .collect(); + + if !transform { + return None; + } + + let transformed_fields: Vec> = file_schema + .fields + .iter() + .map( + |field| match (table_fields.get(field.name()), field.data_type()) { + (Some(DataType::Utf8View), DataType::Utf8 | DataType::LargeUtf8) => { + field_with_new_type(field, DataType::Utf8View) + } + ( + Some(DataType::BinaryView), + DataType::Binary | DataType::LargeBinary, + ) => field_with_new_type(field, DataType::BinaryView), + _ => Arc::clone(field), + }, + ) + .collect(); + + Some(Schema::new_with_metadata( + transformed_fields, + file_schema.metadata.clone(), + )) +} + +/// If the table schema uses a string type, coerce the file schema to use a string type. +/// +/// See [ParquetFormat::binary_as_string] for details +#[cfg(not(target_arch = "wasm32"))] +pub fn coerce_file_schema_to_string_type( + table_schema: &Schema, + file_schema: &Schema, +) -> Option { + let mut transform = false; + let table_fields: HashMap<_, _> = table_schema + .fields + .iter() + .map(|f| (f.name(), f.data_type())) + .collect(); + let transformed_fields: Vec> = file_schema + .fields + .iter() + .map( + |field| match (table_fields.get(field.name()), field.data_type()) { + // table schema uses string type, coerce the file schema to use string type + ( + Some(DataType::Utf8), + DataType::Binary | DataType::LargeBinary | DataType::BinaryView, + ) => { + transform = true; + field_with_new_type(field, DataType::Utf8) + } + // table schema uses large string type, coerce the file schema to use large string type + ( + Some(DataType::LargeUtf8), + DataType::Binary | DataType::LargeBinary | DataType::BinaryView, + ) => { + transform = true; + field_with_new_type(field, DataType::LargeUtf8) + } + // table schema uses string view type, coerce the file schema to use view type + ( + Some(DataType::Utf8View), + DataType::Binary | DataType::LargeBinary | DataType::BinaryView, + ) => { + transform = true; + field_with_new_type(field, DataType::Utf8View) + } + _ => Arc::clone(field), + }, + ) + .collect(); + + if !transform { + None + } else { + Some(Schema::new_with_metadata( + transformed_fields, + file_schema.metadata.clone(), + )) + } +} + +/// Create a new field with the specified data type, copying the other +/// properties from the input field +fn field_with_new_type(field: &FieldRef, new_type: DataType) -> FieldRef { + Arc::new(field.as_ref().clone().with_data_type(new_type)) +} + +/// Transform a schema to use view types for Utf8 and Binary +/// +/// See [ParquetFormat::force_view_types] for details +pub fn transform_schema_to_view(schema: &Schema) -> Schema { + let transformed_fields: Vec> = schema + .fields + .iter() + .map(|field| match field.data_type() { + DataType::Utf8 | DataType::LargeUtf8 => { + field_with_new_type(field, DataType::Utf8View) + } + DataType::Binary | DataType::LargeBinary => { + field_with_new_type(field, DataType::BinaryView) + } + _ => Arc::clone(field), + }) + .collect(); + Schema::new_with_metadata(transformed_fields, schema.metadata.clone()) +} + +/// Transform a schema so that any binary types are strings +pub fn transform_binary_to_string(schema: &Schema) -> Schema { + let transformed_fields: Vec> = schema + .fields + .iter() + .map(|field| match field.data_type() { + DataType::Binary => field_with_new_type(field, DataType::Utf8), + DataType::LargeBinary => field_with_new_type(field, DataType::LargeUtf8), + DataType::BinaryView => field_with_new_type(field, DataType::Utf8View), + _ => Arc::clone(field), + }) + .collect(); + Schema::new_with_metadata(transformed_fields, schema.metadata.clone()) +} + +/// [`MetadataFetch`] adapter for reading bytes from an [`ObjectStore`] +struct ObjectStoreFetch<'a> { + store: &'a dyn ObjectStore, + meta: &'a ObjectMeta, +} + +impl<'a> ObjectStoreFetch<'a> { + fn new(store: &'a dyn ObjectStore, meta: &'a ObjectMeta) -> Self { + Self { store, meta } + } +} + +impl MetadataFetch for ObjectStoreFetch<'_> { + fn fetch( + &mut self, + range: Range, + ) -> BoxFuture<'_, Result> { + async { + self.store + .get_range(&self.meta.location, range) + .await + .map_err(ParquetError::from) + } + .boxed() + } +} + +/// Fetches parquet metadata from ObjectStore for given object +/// +/// This component is a subject to **change** in near future and is exposed for low level integrations +/// through [`ParquetFileReaderFactory`]. +/// +/// [`ParquetFileReaderFactory`]: crate::ParquetFileReaderFactory +pub async fn fetch_parquet_metadata( + store: &dyn ObjectStore, + meta: &ObjectMeta, + size_hint: Option, +) -> Result { + let file_size = meta.size; + let fetch = ObjectStoreFetch::new(store, meta); + + ParquetMetaDataReader::new() + .with_prefetch_hint(size_hint) + .load_and_finish(fetch, file_size) + .await + .map_err(DataFusionError::from) +} + +/// Read and parse the schema of the Parquet file at location `path` +async fn fetch_schema( + store: &dyn ObjectStore, + file: &ObjectMeta, + metadata_size_hint: Option, +) -> Result { + let metadata = fetch_parquet_metadata(store, file, metadata_size_hint).await?; + let file_metadata = metadata.file_metadata(); + let schema = parquet_to_arrow_schema( + file_metadata.schema_descr(), + file_metadata.key_value_metadata(), + )?; + Ok(schema) +} + +/// Read and parse the statistics of the Parquet file at location `path` +/// +/// See [`statistics_from_parquet_meta_calc`] for more details +pub async fn fetch_statistics( + store: &dyn ObjectStore, + table_schema: SchemaRef, + file: &ObjectMeta, + metadata_size_hint: Option, +) -> Result { + let metadata = fetch_parquet_metadata(store, file, metadata_size_hint).await?; + statistics_from_parquet_meta_calc(&metadata, table_schema) +} + +/// Convert statistics in [`ParquetMetaData`] into [`Statistics`] using ['StatisticsConverter`] +/// +/// The statistics are calculated for each column in the table schema +/// using the row group statistics in the parquet metadata. +pub fn statistics_from_parquet_meta_calc( + metadata: &ParquetMetaData, + table_schema: SchemaRef, +) -> Result { + let row_groups_metadata = metadata.row_groups(); + + let mut statistics = Statistics::new_unknown(&table_schema); + let mut has_statistics = false; + let mut num_rows = 0_usize; + let mut total_byte_size = 0_usize; + for row_group_meta in row_groups_metadata { + num_rows += row_group_meta.num_rows() as usize; + total_byte_size += row_group_meta.total_byte_size() as usize; + + if !has_statistics { + row_group_meta.columns().iter().for_each(|column| { + has_statistics = column.statistics().is_some(); + }); + } + } + statistics.num_rows = Precision::Exact(num_rows); + statistics.total_byte_size = Precision::Exact(total_byte_size); + + let file_metadata = metadata.file_metadata(); + let mut file_schema = parquet_to_arrow_schema( + file_metadata.schema_descr(), + file_metadata.key_value_metadata(), + )?; + if let Some(merged) = coerce_file_schema_to_string_type(&table_schema, &file_schema) { + file_schema = merged; + } + + if let Some(merged) = coerce_file_schema_to_view_type(&table_schema, &file_schema) { + file_schema = merged; + } + + statistics.column_statistics = if has_statistics { + let (mut max_accs, mut min_accs) = create_max_min_accs(&table_schema); + let mut null_counts_array = + vec![Precision::Exact(0); table_schema.fields().len()]; + + table_schema + .fields() + .iter() + .enumerate() + .for_each(|(idx, field)| { + match StatisticsConverter::try_new( + field.name(), + &file_schema, + file_metadata.schema_descr(), + ) { + Ok(stats_converter) => { + summarize_min_max_null_counts( + &mut min_accs, + &mut max_accs, + &mut null_counts_array, + idx, + num_rows, + &stats_converter, + row_groups_metadata, + ) + .ok(); + } + Err(e) => { + debug!("Failed to create statistics converter: {}", e); + null_counts_array[idx] = Precision::Exact(num_rows); + } + } + }); + + get_col_stats( + &table_schema, + null_counts_array, + &mut max_accs, + &mut min_accs, + ) + } else { + Statistics::unknown_column(&table_schema) + }; + + Ok(statistics) +} + +fn get_col_stats( + schema: &Schema, + null_counts: Vec>, + max_values: &mut [Option], + min_values: &mut [Option], +) -> Vec { + (0..schema.fields().len()) + .map(|i| { + let max_value = match max_values.get_mut(i).unwrap() { + Some(max_value) => max_value.evaluate().ok(), + None => None, + }; + let min_value = match min_values.get_mut(i).unwrap() { + Some(min_value) => min_value.evaluate().ok(), + None => None, + }; + ColumnStatistics { + null_count: null_counts[i], + max_value: max_value.map(Precision::Exact).unwrap_or(Precision::Absent), + min_value: min_value.map(Precision::Exact).unwrap_or(Precision::Absent), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + } + }) + .collect() +} + +/// Deprecated +/// Use [`statistics_from_parquet_meta_calc`] instead. +/// This method was deprecated because it didn't need to be async so a new method was created +/// that exposes a synchronous API. +#[deprecated( + since = "40.0.0", + note = "please use `statistics_from_parquet_meta_calc` instead" +)] +pub async fn statistics_from_parquet_meta( + metadata: &ParquetMetaData, + table_schema: SchemaRef, +) -> Result { + statistics_from_parquet_meta_calc(metadata, table_schema) +} + +fn summarize_min_max_null_counts( + min_accs: &mut [Option], + max_accs: &mut [Option], + null_counts_array: &mut [Precision], + arrow_schema_index: usize, + num_rows: usize, + stats_converter: &StatisticsConverter, + row_groups_metadata: &[RowGroupMetaData], +) -> Result<()> { + let max_values = stats_converter.row_group_maxes(row_groups_metadata)?; + let min_values = stats_converter.row_group_mins(row_groups_metadata)?; + let null_counts = stats_converter.row_group_null_counts(row_groups_metadata)?; + + if let Some(max_acc) = &mut max_accs[arrow_schema_index] { + max_acc.update_batch(&[max_values])?; + } + + if let Some(min_acc) = &mut min_accs[arrow_schema_index] { + min_acc.update_batch(&[min_values])?; + } + + null_counts_array[arrow_schema_index] = Precision::Exact(match sum(&null_counts) { + Some(null_count) => null_count as usize, + None => num_rows, + }); + + Ok(()) +} + +/// Implements [`DataSink`] for writing to a parquet file. +pub struct ParquetSink { + /// Config options for writing data + config: FileSinkConfig, + /// Underlying parquet options + parquet_options: TableParquetOptions, + /// File metadata from successfully produced parquet files. The Mutex is only used + /// to allow inserting to HashMap from behind borrowed reference in DataSink::write_all. + written: Arc>>, +} + +impl Debug for ParquetSink { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("ParquetSink").finish() + } +} + +impl DisplayAs for ParquetSink { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "ParquetSink(file_groups=",)?; + FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; + write!(f, ")") + } + } + } +} + +impl ParquetSink { + /// Create from config. + pub fn new(config: FileSinkConfig, parquet_options: TableParquetOptions) -> Self { + Self { + config, + parquet_options, + written: Default::default(), + } + } + + /// Retrieve the file metadata for the written files, keyed to the path + /// which may be partitioned (in the case of hive style partitioning). + pub fn written(&self) -> HashMap { + self.written.lock().clone() + } + + /// Create writer properties based upon configuration settings, + /// including partitioning and the inclusion of arrow schema metadata. + fn create_writer_props(&self) -> Result { + let schema = if self.parquet_options.global.allow_single_file_parallelism { + // If parallelizing writes, we may be also be doing hive style partitioning + // into multiple files which impacts the schema per file. + // Refer to `get_writer_schema()` + &get_writer_schema(&self.config) + } else { + self.config.output_schema() + }; + + // TODO: avoid this clone in follow up PR, where the writer properties & schema + // are calculated once on `ParquetSink::new` + let mut parquet_opts = self.parquet_options.clone(); + if !self.parquet_options.global.skip_arrow_metadata { + parquet_opts.arrow_schema(schema); + } + + Ok(WriterPropertiesBuilder::try_from(&parquet_opts)?.build()) + } + + /// Creates an AsyncArrowWriter which serializes a parquet file to an ObjectStore + /// AsyncArrowWriters are used when individual parquet file serialization is not parallelized + async fn create_async_arrow_writer( + &self, + location: &Path, + object_store: Arc, + parquet_props: WriterProperties, + ) -> Result> { + let buf_writer = BufWriter::new(object_store, location.clone()); + let options = ArrowWriterOptions::new() + .with_properties(parquet_props) + .with_skip_arrow_metadata(self.parquet_options.global.skip_arrow_metadata); + + let writer = AsyncArrowWriter::try_new_with_options( + buf_writer, + get_writer_schema(&self.config), + options, + )?; + Ok(writer) + } + + /// Parquet options + pub fn parquet_options(&self) -> &TableParquetOptions { + &self.parquet_options + } +} + +#[async_trait] +impl FileSink for ParquetSink { + fn config(&self) -> &FileSinkConfig { + &self.config + } + + async fn spawn_writer_tasks_and_join( + &self, + context: &Arc, + demux_task: SpawnedTask>, + mut file_stream_rx: DemuxedStreamReceiver, + object_store: Arc, + ) -> Result { + let parquet_opts = &self.parquet_options; + let allow_single_file_parallelism = + parquet_opts.global.allow_single_file_parallelism; + + let mut file_write_tasks: JoinSet< + std::result::Result<(Path, FileMetaData), DataFusionError>, + > = JoinSet::new(); + + let parquet_props = self.create_writer_props()?; + let parallel_options = ParallelParquetWriterOptions { + max_parallel_row_groups: parquet_opts + .global + .maximum_parallel_row_group_writers, + max_buffered_record_batches_per_stream: parquet_opts + .global + .maximum_buffered_record_batches_per_stream, + }; + + while let Some((path, mut rx)) = file_stream_rx.recv().await { + if !allow_single_file_parallelism { + let mut writer = self + .create_async_arrow_writer( + &path, + Arc::clone(&object_store), + parquet_props.clone(), + ) + .await?; + let mut reservation = + MemoryConsumer::new(format!("ParquetSink[{}]", path)) + .register(context.memory_pool()); + file_write_tasks.spawn(async move { + while let Some(batch) = rx.recv().await { + writer.write(&batch).await?; + reservation.try_resize(writer.memory_size())?; + } + let file_metadata = writer + .close() + .await + .map_err(DataFusionError::ParquetError)?; + Ok((path, file_metadata)) + }); + } else { + let writer = create_writer( + // Parquet files as a whole are never compressed, since they + // manage compressed blocks themselves. + FileCompressionType::UNCOMPRESSED, + &path, + Arc::clone(&object_store), + ) + .await?; + let schema = get_writer_schema(&self.config); + let props = parquet_props.clone(); + let parallel_options_clone = parallel_options.clone(); + let pool = Arc::clone(context.memory_pool()); + file_write_tasks.spawn(async move { + let file_metadata = output_single_parquet_file_parallelized( + writer, + rx, + schema, + &props, + parallel_options_clone, + pool, + ) + .await?; + Ok((path, file_metadata)) + }); + } + } + + let mut row_count = 0; + while let Some(result) = file_write_tasks.join_next().await { + match result { + Ok(r) => { + let (path, file_metadata) = r?; + row_count += file_metadata.num_rows; + let mut written_files = self.written.lock(); + written_files + .try_insert(path.clone(), file_metadata) + .map_err(|e| internal_datafusion_err!("duplicate entry detected for partitioned file {path}: {e}"))?; + drop(written_files); + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + } + + demux_task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)??; + + Ok(row_count as u64) + } +} + +#[async_trait] +impl DataSink for ParquetSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> &SchemaRef { + self.config.output_schema() + } + + async fn write_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + FileSink::write_all(self, data, context).await + } +} + +/// Consumes a stream of [ArrowLeafColumn] via a channel and serializes them using an [ArrowColumnWriter] +/// Once the channel is exhausted, returns the ArrowColumnWriter. +async fn column_serializer_task( + mut rx: Receiver, + mut writer: ArrowColumnWriter, + mut reservation: MemoryReservation, +) -> Result<(ArrowColumnWriter, MemoryReservation)> { + while let Some(col) = rx.recv().await { + writer.write(&col)?; + reservation.try_resize(writer.memory_size())?; + } + Ok((writer, reservation)) +} + +type ColumnWriterTask = SpawnedTask>; +type ColSender = Sender; + +/// Spawns a parallel serialization task for each column +/// Returns join handles for each columns serialization task along with a send channel +/// to send arrow arrays to each serialization task. +fn spawn_column_parallel_row_group_writer( + schema: Arc, + parquet_props: Arc, + max_buffer_size: usize, + pool: &Arc, +) -> Result<(Vec, Vec)> { + let schema_desc = ArrowSchemaConverter::new().convert(&schema)?; + let col_writers = get_column_writers(&schema_desc, &parquet_props, &schema)?; + let num_columns = col_writers.len(); + + let mut col_writer_tasks = Vec::with_capacity(num_columns); + let mut col_array_channels = Vec::with_capacity(num_columns); + for writer in col_writers.into_iter() { + // Buffer size of this channel limits the number of arrays queued up for column level serialization + let (send_array, receive_array) = + mpsc::channel::(max_buffer_size); + col_array_channels.push(send_array); + + let reservation = + MemoryConsumer::new("ParquetSink(ArrowColumnWriter)").register(pool); + let task = SpawnedTask::spawn(column_serializer_task( + receive_array, + writer, + reservation, + )); + col_writer_tasks.push(task); + } + + Ok((col_writer_tasks, col_array_channels)) +} + +/// Settings related to writing parquet files in parallel +#[derive(Clone)] +struct ParallelParquetWriterOptions { + max_parallel_row_groups: usize, + max_buffered_record_batches_per_stream: usize, +} + +/// This is the return type of calling [ArrowColumnWriter].close() on each column +/// i.e. the Vec of encoded columns which can be appended to a row group +type RBStreamSerializeResult = Result<(Vec, MemoryReservation, usize)>; + +/// Sends the ArrowArrays in passed [RecordBatch] through the channels to their respective +/// parallel column serializers. +async fn send_arrays_to_col_writers( + col_array_channels: &[ColSender], + rb: &RecordBatch, + schema: Arc, +) -> Result<()> { + // Each leaf column has its own channel, increment next_channel for each leaf column sent. + let mut next_channel = 0; + for (array, field) in rb.columns().iter().zip(schema.fields()) { + for c in compute_leaves(field, array)? { + // Do not surface error from closed channel (means something + // else hit an error, and the plan is shutting down). + if col_array_channels[next_channel].send(c).await.is_err() { + return Ok(()); + } + + next_channel += 1; + } + } + + Ok(()) +} + +/// Spawns a tokio task which joins the parallel column writer tasks, +/// and finalizes the row group +fn spawn_rg_join_and_finalize_task( + column_writer_tasks: Vec, + rg_rows: usize, + pool: &Arc, +) -> SpawnedTask { + let mut rg_reservation = + MemoryConsumer::new("ParquetSink(SerializedRowGroupWriter)").register(pool); + + SpawnedTask::spawn(async move { + let num_cols = column_writer_tasks.len(); + let mut finalized_rg = Vec::with_capacity(num_cols); + for task in column_writer_tasks.into_iter() { + let (writer, _col_reservation) = task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)??; + let encoded_size = writer.get_estimated_total_bytes(); + rg_reservation.grow(encoded_size); + finalized_rg.push(writer.close()?); + } + + Ok((finalized_rg, rg_reservation, rg_rows)) + }) +} + +/// This task coordinates the serialization of a parquet file in parallel. +/// As the query produces RecordBatches, these are written to a RowGroup +/// via parallel [ArrowColumnWriter] tasks. Once the desired max rows per +/// row group is reached, the parallel tasks are joined on another separate task +/// and sent to a concatenation task. This task immediately continues to work +/// on the next row group in parallel. So, parquet serialization is parallelized +/// across both columns and row_groups, with a theoretical max number of parallel tasks +/// given by n_columns * num_row_groups. +fn spawn_parquet_parallel_serialization_task( + mut data: Receiver, + serialize_tx: Sender>, + schema: Arc, + writer_props: Arc, + parallel_options: ParallelParquetWriterOptions, + pool: Arc, +) -> SpawnedTask> { + SpawnedTask::spawn(async move { + let max_buffer_rb = parallel_options.max_buffered_record_batches_per_stream; + let max_row_group_rows = writer_props.max_row_group_size(); + let (mut column_writer_handles, mut col_array_channels) = + spawn_column_parallel_row_group_writer( + Arc::clone(&schema), + Arc::clone(&writer_props), + max_buffer_rb, + &pool, + )?; + let mut current_rg_rows = 0; + + while let Some(mut rb) = data.recv().await { + // This loop allows the "else" block to repeatedly split the RecordBatch to handle the case + // when max_row_group_rows < execution.batch_size as an alternative to a recursive async + // function. + loop { + if current_rg_rows + rb.num_rows() < max_row_group_rows { + send_arrays_to_col_writers( + &col_array_channels, + &rb, + Arc::clone(&schema), + ) + .await?; + current_rg_rows += rb.num_rows(); + break; + } else { + let rows_left = max_row_group_rows - current_rg_rows; + let a = rb.slice(0, rows_left); + send_arrays_to_col_writers( + &col_array_channels, + &a, + Arc::clone(&schema), + ) + .await?; + + // Signal the parallel column writers that the RowGroup is done, join and finalize RowGroup + // on a separate task, so that we can immediately start on the next RG before waiting + // for the current one to finish. + drop(col_array_channels); + let finalize_rg_task = spawn_rg_join_and_finalize_task( + column_writer_handles, + max_row_group_rows, + &pool, + ); + + // Do not surface error from closed channel (means something + // else hit an error, and the plan is shutting down). + if serialize_tx.send(finalize_rg_task).await.is_err() { + return Ok(()); + } + + current_rg_rows = 0; + rb = rb.slice(rows_left, rb.num_rows() - rows_left); + + (column_writer_handles, col_array_channels) = + spawn_column_parallel_row_group_writer( + Arc::clone(&schema), + Arc::clone(&writer_props), + max_buffer_rb, + &pool, + )?; + } + } + } + + drop(col_array_channels); + // Handle leftover rows as final rowgroup, which may be smaller than max_row_group_rows + if current_rg_rows > 0 { + let finalize_rg_task = spawn_rg_join_and_finalize_task( + column_writer_handles, + current_rg_rows, + &pool, + ); + + // Do not surface error from closed channel (means something + // else hit an error, and the plan is shutting down). + if serialize_tx.send(finalize_rg_task).await.is_err() { + return Ok(()); + } + } + + Ok(()) + }) +} + +/// Consume RowGroups serialized by other parallel tasks and concatenate them in +/// to the final parquet file, while flushing finalized bytes to an [ObjectStore] +async fn concatenate_parallel_row_groups( + mut serialize_rx: Receiver>, + schema: Arc, + writer_props: Arc, + mut object_store_writer: Box, + pool: Arc, +) -> Result { + let merged_buff = SharedBuffer::new(INITIAL_BUFFER_BYTES); + + let mut file_reservation = + MemoryConsumer::new("ParquetSink(SerializedFileWriter)").register(&pool); + + let schema_desc = ArrowSchemaConverter::new().convert(schema.as_ref())?; + let mut parquet_writer = SerializedFileWriter::new( + merged_buff.clone(), + schema_desc.root_schema_ptr(), + writer_props, + )?; + + while let Some(task) = serialize_rx.recv().await { + let result = task.join_unwind().await; + let mut rg_out = parquet_writer.next_row_group()?; + let (serialized_columns, mut rg_reservation, _cnt) = + result.map_err(DataFusionError::ExecutionJoin)??; + for chunk in serialized_columns { + chunk.append_to_row_group(&mut rg_out)?; + rg_reservation.free(); + + let mut buff_to_flush = merged_buff.buffer.try_lock().unwrap(); + file_reservation.try_resize(buff_to_flush.len())?; + + if buff_to_flush.len() > BUFFER_FLUSH_BYTES { + object_store_writer + .write_all(buff_to_flush.as_slice()) + .await?; + buff_to_flush.clear(); + file_reservation.try_resize(buff_to_flush.len())?; // will set to zero + } + } + rg_out.close()?; + } + + let file_metadata = parquet_writer.close()?; + let final_buff = merged_buff.buffer.try_lock().unwrap(); + + object_store_writer.write_all(final_buff.as_slice()).await?; + object_store_writer.shutdown().await?; + file_reservation.free(); + + Ok(file_metadata) +} + +/// Parallelizes the serialization of a single parquet file, by first serializing N +/// independent RecordBatch streams in parallel to RowGroups in memory. Another +/// task then stitches these independent RowGroups together and streams this large +/// single parquet file to an ObjectStore in multiple parts. +async fn output_single_parquet_file_parallelized( + object_store_writer: Box, + data: Receiver, + output_schema: Arc, + parquet_props: &WriterProperties, + parallel_options: ParallelParquetWriterOptions, + pool: Arc, +) -> Result { + let max_rowgroups = parallel_options.max_parallel_row_groups; + // Buffer size of this channel limits maximum number of RowGroups being worked on in parallel + let (serialize_tx, serialize_rx) = + mpsc::channel::>(max_rowgroups); + + let arc_props = Arc::new(parquet_props.clone()); + let launch_serialization_task = spawn_parquet_parallel_serialization_task( + data, + serialize_tx, + Arc::clone(&output_schema), + Arc::clone(&arc_props), + parallel_options, + Arc::clone(&pool), + ); + let file_metadata = concatenate_parallel_row_groups( + serialize_rx, + Arc::clone(&output_schema), + Arc::clone(&arc_props), + object_store_writer, + pool, + ) + .await?; + + launch_serialization_task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)??; + Ok(file_metadata) +} + +/// Min/max aggregation can take Dictionary encode input but always produces unpacked +/// (aka non Dictionary) output. We need to adjust the output data type to reflect this. +/// The reason min/max aggregate produces unpacked output because there is only one +/// min/max value per group; there is no needs to keep them Dictionary encode +fn min_max_aggregate_data_type(input_type: &DataType) -> &DataType { + if let DataType::Dictionary(_, value_type) = input_type { + value_type.as_ref() + } else { + input_type + } +} + +fn create_max_min_accs( + schema: &Schema, +) -> (Vec>, Vec>) { + let max_values: Vec> = schema + .fields() + .iter() + .map(|field| { + MaxAccumulator::try_new(min_max_aggregate_data_type(field.data_type())).ok() + }) + .collect(); + let min_values: Vec> = schema + .fields() + .iter() + .map(|field| { + MinAccumulator::try_new(min_max_aggregate_data_type(field.data_type())).ok() + }) + .collect(); + (max_values, min_values) +} diff --git a/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs b/datafusion/datasource-parquet/src/metrics.rs similarity index 99% rename from datafusion/core/src/datasource/physical_plan/parquet/metrics.rs rename to datafusion/datasource-parquet/src/metrics.rs index f1b5f71530dc..3213d0201295 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs +++ b/datafusion/datasource-parquet/src/metrics.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::metrics::{ +use datafusion_physical_plan::metrics::{ Count, ExecutionPlanMetricsSet, MetricBuilder, Time, }; diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs new file mode 100644 index 000000000000..fb1f2d55169f --- /dev/null +++ b/datafusion/datasource-parquet/src/mod.rs @@ -0,0 +1,547 @@ +// 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. + +//! [`ParquetExec`] FileSource for reading Parquet files + +pub mod access_plan; +pub mod file_format; +mod metrics; +mod opener; +mod page_filter; +mod reader; +mod row_filter; +mod row_group_filter; +pub mod source; +mod writer; + +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + +pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; +use arrow::datatypes::SchemaRef; +use datafusion_common::config::{ConfigOptions, TableParquetOptions}; +use datafusion_common::Result; +use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::schema_adapter::SchemaAdapterFactory; +use datafusion_datasource::source::DataSourceExec; +use datafusion_datasource::PartitionedFile; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{ + EquivalenceProperties, LexOrdering, Partitioning, PhysicalExpr, +}; +use datafusion_physical_optimizer::pruning::PruningPredicate; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; +pub use file_format::*; +pub use metrics::ParquetFileMetrics; +pub use page_filter::PagePruningAccessPlanFilter; +pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; +pub use row_filter::build_row_filter; +pub use row_filter::can_expr_be_pushed_down_with_schemas; +pub use row_group_filter::RowGroupAccessPlanFilter; +use source::ParquetSource; +pub use writer::plan_to_parquet; + +use log::debug; + +#[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] +/// Deprecated Execution plan replaced with DataSourceExec +pub struct ParquetExec { + inner: DataSourceExec, + base_config: FileScanConfig, + table_parquet_options: TableParquetOptions, + /// Optional predicate for row filtering during parquet scan + predicate: Option>, + /// Optional predicate for pruning row groups (derived from `predicate`) + pruning_predicate: Option>, + /// Optional user defined parquet file reader factory + parquet_file_reader_factory: Option>, + /// Optional user defined schema adapter + schema_adapter_factory: Option>, +} + +#[allow(unused, deprecated)] +impl From for ParquetExecBuilder { + fn from(exec: ParquetExec) -> Self { + exec.into_builder() + } +} + +/// [`ParquetExecBuilder`], deprecated builder for [`ParquetExec`]. +/// +/// ParquetExec is replaced with `DataSourceExec` and it includes `ParquetSource` +/// +/// See example on [`ParquetSource`]. +#[deprecated( + since = "46.0.0", + note = "use DataSourceExec with ParquetSource instead" +)] +#[allow(unused, deprecated)] +pub struct ParquetExecBuilder { + file_scan_config: FileScanConfig, + predicate: Option>, + metadata_size_hint: Option, + table_parquet_options: TableParquetOptions, + parquet_file_reader_factory: Option>, + schema_adapter_factory: Option>, +} + +#[allow(unused, deprecated)] +impl ParquetExecBuilder { + /// Create a new builder to read the provided file scan configuration + pub fn new(file_scan_config: FileScanConfig) -> Self { + Self::new_with_options(file_scan_config, TableParquetOptions::default()) + } + + /// Create a new builder to read the data specified in the file scan + /// configuration with the provided `TableParquetOptions`. + pub fn new_with_options( + file_scan_config: FileScanConfig, + table_parquet_options: TableParquetOptions, + ) -> Self { + Self { + file_scan_config, + predicate: None, + metadata_size_hint: None, + table_parquet_options, + parquet_file_reader_factory: None, + schema_adapter_factory: None, + } + } + + /// Update the list of files groups to read + pub fn with_file_groups(mut self, file_groups: Vec>) -> Self { + self.file_scan_config.file_groups = file_groups; + self + } + + /// Set the filter predicate when reading. + /// + /// See the "Predicate Pushdown" section of the [`ParquetExec`] documentation + /// for more details. + pub fn with_predicate(mut self, predicate: Arc) -> Self { + self.predicate = Some(predicate); + self + } + + /// Set the metadata size hint + /// + /// This value determines how many bytes at the end of the file the default + /// [`ParquetFileReaderFactory`] will request in the initial IO. If this is + /// too small, the ParquetExec will need to make additional IO requests to + /// read the footer. + pub fn with_metadata_size_hint(mut self, metadata_size_hint: usize) -> Self { + self.metadata_size_hint = Some(metadata_size_hint); + self + } + + /// Set the options for controlling how the ParquetExec reads parquet files. + /// + /// See also [`Self::new_with_options`] + pub fn with_table_parquet_options( + mut self, + table_parquet_options: TableParquetOptions, + ) -> Self { + self.table_parquet_options = table_parquet_options; + self + } + + /// Set optional user defined parquet file reader factory. + /// + /// You can use [`ParquetFileReaderFactory`] to more precisely control how + /// data is read from parquet files (e.g. skip re-reading metadata, coalesce + /// I/O operations, etc). + /// + /// The default reader factory reads directly from an [`ObjectStore`] + /// instance using individual I/O operations for the footer and each page. + /// + /// If a custom `ParquetFileReaderFactory` is provided, then data access + /// operations will be routed to this factory instead of [`ObjectStore`]. + /// + /// [`ObjectStore`]: object_store::ObjectStore + pub fn with_parquet_file_reader_factory( + mut self, + parquet_file_reader_factory: Arc, + ) -> Self { + self.parquet_file_reader_factory = Some(parquet_file_reader_factory); + self + } + + /// Set optional schema adapter factory. + /// + /// [`SchemaAdapterFactory`] allows user to specify how fields from the + /// parquet file get mapped to that of the table schema. The default schema + /// adapter uses arrow's cast library to map the parquet fields to the table + /// schema. + pub fn with_schema_adapter_factory( + mut self, + schema_adapter_factory: Arc, + ) -> Self { + self.schema_adapter_factory = Some(schema_adapter_factory); + self + } + + /// Convenience: build an `Arc`d `ParquetExec` from this builder + pub fn build_arc(self) -> Arc { + Arc::new(self.build()) + } + + /// Build a [`ParquetExec`] + #[must_use] + pub fn build(self) -> ParquetExec { + let Self { + file_scan_config, + predicate, + metadata_size_hint, + table_parquet_options, + parquet_file_reader_factory, + schema_adapter_factory, + } = self; + let mut parquet = ParquetSource::new(table_parquet_options); + if let Some(predicate) = predicate.clone() { + parquet = parquet + .with_predicate(Arc::clone(&file_scan_config.file_schema), predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + parquet = parquet.with_metadata_size_hint(metadata_size_hint) + } + if let Some(parquet_reader_factory) = parquet_file_reader_factory { + parquet = parquet.with_parquet_file_reader_factory(parquet_reader_factory) + } + if let Some(schema_factory) = schema_adapter_factory { + parquet = parquet.with_schema_adapter_factory(schema_factory); + } + + let base_config = file_scan_config.with_source(Arc::new(parquet.clone())); + debug!("Creating ParquetExec, files: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", + base_config.file_groups, base_config.projection, predicate, base_config.limit); + + ParquetExec { + inner: DataSourceExec::new(Arc::new(base_config.clone())), + base_config, + predicate, + pruning_predicate: parquet.pruning_predicate, + schema_adapter_factory: parquet.schema_adapter_factory, + parquet_file_reader_factory: parquet.parquet_file_reader_factory, + table_parquet_options: parquet.table_parquet_options, + } + } +} + +#[allow(unused, deprecated)] +impl ParquetExec { + /// Create a new Parquet reader execution plan provided file list and schema. + pub fn new( + base_config: FileScanConfig, + predicate: Option>, + metadata_size_hint: Option, + table_parquet_options: TableParquetOptions, + ) -> Self { + let mut builder = + ParquetExecBuilder::new_with_options(base_config, table_parquet_options); + if let Some(predicate) = predicate { + builder = builder.with_predicate(predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + builder = builder.with_metadata_size_hint(metadata_size_hint); + } + builder.build() + } + /// Return a [`ParquetExecBuilder`]. + /// + /// See example on [`ParquetExec`] and [`ParquetExecBuilder`] for specifying + /// parquet table options. + pub fn builder(file_scan_config: FileScanConfig) -> ParquetExecBuilder { + ParquetExecBuilder::new(file_scan_config) + } + + /// Convert this `ParquetExec` into a builder for modification + pub fn into_builder(self) -> ParquetExecBuilder { + // list out fields so it is clear what is being dropped + // (note the fields which are dropped are re-created as part of calling + // `build` on the builder) + let file_scan_config = self.file_scan_config(); + let parquet = self.parquet_source(); + + ParquetExecBuilder { + file_scan_config, + predicate: parquet.predicate, + metadata_size_hint: parquet.metadata_size_hint, + table_parquet_options: parquet.table_parquet_options, + parquet_file_reader_factory: parquet.parquet_file_reader_factory, + schema_adapter_factory: parquet.schema_adapter_factory, + } + } + fn file_scan_config(&self) -> FileScanConfig { + self.inner + .data_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + fn parquet_source(&self) -> ParquetSource { + self.file_scan_config() + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + /// [`FileScanConfig`] that controls this scan (such as which files to read) + pub fn base_config(&self) -> &FileScanConfig { + &self.base_config + } + /// Options passed to the parquet reader for this scan + pub fn table_parquet_options(&self) -> &TableParquetOptions { + &self.table_parquet_options + } + /// Optional predicate. + pub fn predicate(&self) -> Option<&Arc> { + self.predicate.as_ref() + } + /// Optional reference to this parquet scan's pruning predicate + pub fn pruning_predicate(&self) -> Option<&Arc> { + self.pruning_predicate.as_ref() + } + /// return the optional file reader factory + pub fn parquet_file_reader_factory( + &self, + ) -> Option<&Arc> { + self.parquet_file_reader_factory.as_ref() + } + /// Optional user defined parquet file reader factory. + pub fn with_parquet_file_reader_factory( + mut self, + parquet_file_reader_factory: Arc, + ) -> Self { + let mut parquet = self.parquet_source(); + parquet.parquet_file_reader_factory = + Some(Arc::clone(&parquet_file_reader_factory)); + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); + self.parquet_file_reader_factory = Some(parquet_file_reader_factory); + self + } + /// return the optional schema adapter factory + pub fn schema_adapter_factory(&self) -> Option<&Arc> { + self.schema_adapter_factory.as_ref() + } + /// Set optional schema adapter factory. + /// + /// [`SchemaAdapterFactory`] allows user to specify how fields from the + /// parquet file get mapped to that of the table schema. The default schema + /// adapter uses arrow's cast library to map the parquet fields to the table + /// schema. + pub fn with_schema_adapter_factory( + mut self, + schema_adapter_factory: Arc, + ) -> Self { + let mut parquet = self.parquet_source(); + parquet.schema_adapter_factory = Some(Arc::clone(&schema_adapter_factory)); + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); + self.schema_adapter_factory = Some(schema_adapter_factory); + self + } + /// If true, the predicate will be used during the parquet scan. + /// Defaults to false + /// + /// [`Expr`]: datafusion_expr::Expr + pub fn with_pushdown_filters(mut self, pushdown_filters: bool) -> Self { + let mut parquet = self.parquet_source(); + parquet.table_parquet_options.global.pushdown_filters = pushdown_filters; + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); + self.table_parquet_options.global.pushdown_filters = pushdown_filters; + self + } + + /// Return the value described in [`Self::with_pushdown_filters`] + fn pushdown_filters(&self) -> bool { + self.parquet_source() + .table_parquet_options + .global + .pushdown_filters + } + /// If true, the `RowFilter` made by `pushdown_filters` may try to + /// minimize the cost of filter evaluation by reordering the + /// predicate [`Expr`]s. If false, the predicates are applied in + /// the same order as specified in the query. Defaults to false. + /// + /// [`Expr`]: datafusion_expr::Expr + pub fn with_reorder_filters(mut self, reorder_filters: bool) -> Self { + let mut parquet = self.parquet_source(); + parquet.table_parquet_options.global.reorder_filters = reorder_filters; + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_data_source(Arc::new(file_source.with_source(Arc::new(parquet)))); + self.table_parquet_options.global.reorder_filters = reorder_filters; + self + } + /// Return the value described in [`Self::with_reorder_filters`] + fn reorder_filters(&self) -> bool { + self.parquet_source() + .table_parquet_options + .global + .reorder_filters + } + /// If enabled, the reader will read the page index + /// This is used to optimize filter pushdown + /// via `RowSelector` and `RowFilter` by + /// eliminating unnecessary IO and decoding + fn bloom_filter_on_read(&self) -> bool { + self.parquet_source() + .table_parquet_options + .global + .bloom_filter_on_read + } + /// Return the value described in [`ParquetSource::with_enable_page_index`] + fn enable_page_index(&self) -> bool { + self.parquet_source() + .table_parquet_options + .global + .enable_page_index + } + + fn output_partitioning_helper(file_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + constraints: Constraints, + file_config: &FileScanConfig, + ) -> PlanProperties { + PlanProperties::new( + EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints), + Self::output_partitioning_helper(file_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + /// Updates the file groups to read and recalculates the output partitioning + /// + /// Note this function does not update statistics or other properties + /// that depend on the file groups. + fn with_file_groups_and_update_partitioning( + mut self, + file_groups: Vec>, + ) -> Self { + let mut config = self.file_scan_config(); + config.file_groups = file_groups; + self.inner = self.inner.with_data_source(Arc::new(config)); + self + } +} + +#[allow(unused, deprecated)] +impl DisplayAs for ParquetExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + self.inner.fmt_as(t, f) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for ParquetExec { + fn name(&self) -> &'static str { + "ParquetExec" + } + + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + self.inner.properties() + } + + fn children(&self) -> Vec<&Arc> { + // this is a leaf node and has no children + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + /// Redistribute files across partitions according to their size + /// See comments on `FileGroupPartitioner` for more detail. + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> Result>> { + self.inner.repartitioned(target_partitions, config) + } + + fn execute( + &self, + partition_index: usize, + ctx: Arc, + ) -> Result { + self.inner.execute(partition_index, ctx) + } + fn metrics(&self) -> Option { + self.inner.metrics() + } + fn statistics(&self) -> Result { + self.inner.statistics() + } + fn fetch(&self) -> Option { + self.inner.fetch() + } + + fn with_fetch(&self, limit: Option) -> Option> { + self.inner.with_fetch(limit) + } +} + +fn should_enable_page_index( + enable_page_index: bool, + page_pruning_predicate: &Option>, +) -> bool { + enable_page_index + && page_pruning_predicate.is_some() + && page_pruning_predicate + .as_ref() + .map(|p| p.filter_number() > 0) + .unwrap_or(false) +} diff --git a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs b/datafusion/datasource-parquet/src/opener.rs similarity index 95% rename from datafusion/core/src/datasource/physical_plan/parquet/opener.rs rename to datafusion/datasource-parquet/src/opener.rs index 4230a1bdce38..3c623f558e43 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -19,18 +19,18 @@ use std::sync::Arc; -use crate::datasource::file_format::parquet::{ +use crate::file_format::{ coerce_file_schema_to_string_type, coerce_file_schema_to_view_type, }; -use crate::datasource::physical_plan::parquet::page_filter::PagePruningAccessPlanFilter; -use crate::datasource::physical_plan::parquet::row_group_filter::RowGroupAccessPlanFilter; -use crate::datasource::physical_plan::parquet::{ - row_filter, should_enable_page_index, ParquetAccessPlan, +use crate::page_filter::PagePruningAccessPlanFilter; +use crate::row_group_filter::RowGroupAccessPlanFilter; +use crate::{ + row_filter, should_enable_page_index, ParquetAccessPlan, ParquetFileMetrics, + ParquetFileReaderFactory, }; -use crate::datasource::physical_plan::{ - FileMeta, FileOpenFuture, FileOpener, ParquetFileMetrics, ParquetFileReaderFactory, -}; -use crate::datasource::schema_adapter::SchemaAdapterFactory; +use datafusion_datasource::file_meta::FileMeta; +use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; +use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use arrow::datatypes::SchemaRef; use arrow::error::ArrowError; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/datasource-parquet/src/page_filter.rs similarity index 99% rename from datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs rename to datafusion/datasource-parquet/src/page_filter.rs index 02329effb09a..ef832d808647 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/datasource-parquet/src/page_filter.rs @@ -21,7 +21,7 @@ use std::collections::HashSet; use std::sync::Arc; use super::metrics::ParquetFileMetrics; -use crate::datasource::physical_plan::parquet::ParquetAccessPlan; +use crate::ParquetAccessPlan; use arrow::array::BooleanArray; use arrow::{ diff --git a/datafusion/core/src/datasource/physical_plan/parquet/reader.rs b/datafusion/datasource-parquet/src/reader.rs similarity index 98% rename from datafusion/core/src/datasource/physical_plan/parquet/reader.rs rename to datafusion/datasource-parquet/src/reader.rs index 8a4ba136fc96..5924a5b5038f 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/reader.rs +++ b/datafusion/datasource-parquet/src/reader.rs @@ -18,8 +18,8 @@ //! [`ParquetFileReaderFactory`] and [`DefaultParquetFileReaderFactory`] for //! low level control of parquet file readers -use crate::datasource::physical_plan::{FileMeta, ParquetFileMetrics}; use bytes::Bytes; +use datafusion_datasource::file_meta::FileMeta; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use futures::future::BoxFuture; use object_store::ObjectStore; @@ -29,6 +29,8 @@ use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; +use crate::ParquetFileMetrics; + /// Interface for reading parquet files. /// /// The combined implementations of [`ParquetFileReaderFactory`] and diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/datasource-parquet/src/row_filter.rs similarity index 98% rename from datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs rename to datafusion/datasource-parquet/src/row_filter.rs index ac6eaf2c8f63..39fcecf37c6d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/datasource-parquet/src/row_filter.rs @@ -71,17 +71,17 @@ use parquet::arrow::arrow_reader::{ArrowPredicate, RowFilter}; use parquet::arrow::ProjectionMask; use parquet::file::metadata::ParquetMetaData; -use crate::datasource::schema_adapter::SchemaMapper; use datafusion_common::cast::as_boolean_array; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter, }; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result, ScalarValue}; +use datafusion_datasource::schema_adapter::SchemaMapper; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::reassign_predicate_columns; use datafusion_physical_expr::{split_conjunction, PhysicalExpr}; -use crate::physical_plan::metrics; +use datafusion_physical_plan::metrics; use super::ParquetFileMetrics; @@ -584,7 +584,7 @@ pub fn build_row_filter( #[cfg(test)] mod test { use super::*; - use crate::datasource::schema_adapter::{ + use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; @@ -601,7 +601,7 @@ mod test { // We should ignore predicate that read non-primitive columns #[test] fn test_filter_candidate_builder_ignore_complex_types() { - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let file = std::fs::File::open(format!("{testdata}/list_columns.parquet")) .expect("opening file"); @@ -626,7 +626,7 @@ mod test { // If a column exists in the table schema but not the file schema it should be rewritten to a null expression #[test] fn test_filter_candidate_builder_rewrite_missing_column() { - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let file = std::fs::File::open(format!("{testdata}/alltypes_plain.parquet")) .expect("opening file"); @@ -665,7 +665,7 @@ mod test { #[test] fn test_filter_type_coercion() { - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let file = std::fs::File::open(format!("{testdata}/alltypes_plain.parquet")) .expect("opening file"); @@ -835,7 +835,7 @@ mod test { } fn get_basic_table_schema() -> Schema { - let testdata = crate::test_util::parquet_test_data(); + let testdata = datafusion_common::test_util::parquet_test_data(); let file = std::fs::File::open(format!("{testdata}/alltypes_plain.parquet")) .expect("opening file"); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs b/datafusion/datasource-parquet/src/row_group_filter.rs similarity index 99% rename from datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs rename to datafusion/datasource-parquet/src/row_group_filter.rs index 27bfb26902e5..9d5f9fa16b6e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs +++ b/datafusion/datasource-parquet/src/row_group_filter.rs @@ -19,13 +19,11 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use super::{ParquetAccessPlan, ParquetFileMetrics}; -use crate::datasource::listing::FileRange; - use arrow::array::{ArrayRef, BooleanArray}; use arrow::datatypes::Schema; use datafusion_common::{Column, Result, ScalarValue}; +use datafusion_datasource::FileRange; use datafusion_physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; - use parquet::arrow::arrow_reader::statistics::StatisticsConverter; use parquet::arrow::parquet_column; use parquet::basic::Type; @@ -435,15 +433,14 @@ mod tests { use std::sync::Arc; use super::*; - use crate::datasource::physical_plan::parquet::reader::ParquetFileReader; - use crate::physical_plan::metrics::ExecutionPlanMetricsSet; + use crate::reader::ParquetFileReader; use arrow::datatypes::DataType::Decimal128; use arrow::datatypes::{DataType, Field}; use datafusion_common::Result; use datafusion_expr::{cast, col, lit, Expr}; use datafusion_physical_expr::planner::logical2physical; - + use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use parquet::arrow::async_reader::ParquetObjectReader; use parquet::arrow::ArrowSchemaConverter; use parquet::basic::LogicalType; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/datasource-parquet/src/source.rs similarity index 94% rename from datafusion/core/src/datasource/physical_plan/parquet/source.rs rename to datafusion/datasource-parquet/src/source.rs index 142725524f1b..b9a2f3e999d0 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -20,11 +20,12 @@ use std::any::Any; use std::fmt::Formatter; use std::sync::Arc; -use crate::datasource::physical_plan::parquet::opener::ParquetOpener; -use crate::datasource::physical_plan::parquet::page_filter::PagePruningAccessPlanFilter; -use crate::datasource::physical_plan::parquet::DefaultParquetFileReaderFactory; -use crate::datasource::physical_plan::{FileOpener, ParquetFileReaderFactory}; -use crate::datasource::schema_adapter::{ +use crate::opener::ParquetOpener; +use crate::page_filter::PagePruningAccessPlanFilter; +use crate::DefaultParquetFileReaderFactory; +use crate::ParquetFileReaderFactory; +use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; @@ -75,12 +76,12 @@ use object_store::ObjectStore; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; -/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion_datasource::file_scan_config::FileScanConfig; +/// # use datafusion_datasource_parquet::source::ParquetSource; +/// # use datafusion_datasource::PartitionedFile; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_physical_expr::expressions::lit; -/// # use datafusion::datasource::source::DataSourceExec; +/// # use datafusion_datasource::source::DataSourceExec; /// # use datafusion_common::config::TableParquetOptions; /// /// # let file_schema = Arc::new(Schema::empty()); @@ -157,9 +158,9 @@ use object_store::ObjectStore; /// ```no_run /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::source::DataSourceExec; +/// # use datafusion_datasource::file_scan_config::FileScanConfig; +/// # use datafusion_datasource::PartitionedFile; +/// # use datafusion_datasource::source::DataSourceExec; /// /// # fn parquet_exec() -> DataSourceExec { unimplemented!() } /// // Split a single DataSourceExec into multiple DataSourceExecs, one for each file @@ -196,12 +197,12 @@ use object_store::ObjectStore; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{Schema, SchemaRef}; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; +/// # use datafusion_datasource::PartitionedFile; +/// # use datafusion_datasource_parquet::ParquetAccessPlan; +/// # use datafusion_datasource::file_scan_config::FileScanConfig; +/// # use datafusion_datasource_parquet::source::ParquetSource; /// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion::datasource::source::DataSourceExec; +/// # use datafusion_datasource::source::DataSourceExec; /// /// # fn schema() -> SchemaRef { /// # Arc::new(Schema::empty()) @@ -247,7 +248,7 @@ use object_store::ObjectStore; /// filled with nulls, but this can be customized via [`SchemaAdapterFactory`]. /// /// [`RecordBatch`]: arrow::record_batch::RecordBatch -/// [`SchemaAdapter`]: crate::datasource::schema_adapter::SchemaAdapter +/// [`SchemaAdapter`]: datafusion_datasource::schema_adapter::SchemaAdapter /// [`ParquetMetadata`]: parquet::file::metadata::ParquetMetaData #[derive(Clone, Default, Debug)] pub struct ParquetSource { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/writer.rs b/datafusion/datasource-parquet/src/writer.rs similarity index 98% rename from datafusion/core/src/datasource/physical_plan/parquet/writer.rs rename to datafusion/datasource-parquet/src/writer.rs index 00926dc2330b..cfdb057a4bc4 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/writer.rs +++ b/datafusion/datasource-parquet/src/writer.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::datasource::listing::ListingTableUrl; use datafusion_common::DataFusionError; +use datafusion_datasource::ListingTableUrl; use datafusion_execution::TaskContext; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use futures::StreamExt; diff --git a/datafusion/datasource/Cargo.toml b/datafusion/datasource/Cargo.toml index d75b27c2f685..5f0b2b4f2e77 100644 --- a/datafusion/datasource/Cargo.toml +++ b/datafusion/datasource/Cargo.toml @@ -31,8 +31,7 @@ version.workspace = true all-features = true [features] -# Temporary feature while I move things around -avro = [] +parquet = ["dep:parquet", "tempfile"] compression = ["async-compression", "xz2", "bzip2", "flate2", "zstd", "tokio-util"] default = ["compression"] @@ -63,7 +62,9 @@ glob = "0.3.0" itertools = { workspace = true } log = { workspace = true } object_store = { workspace = true } +parquet = { workspace = true, optional = true } rand = { workspace = true } +tempfile = { workspace = true, optional = true } tokio = { workspace = true } tokio-util = { version = "0.7.4", features = ["io"], optional = true } url = { workspace = true } diff --git a/datafusion/datasource/src/decoder.rs b/datafusion/datasource/src/decoder.rs new file mode 100644 index 000000000000..654569f74113 --- /dev/null +++ b/datafusion/datasource/src/decoder.rs @@ -0,0 +1,191 @@ +// 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. + +//! Module containing helper methods for the various file formats +//! See write.rs for write related helper methods + +use ::arrow::array::RecordBatch; + +use arrow::error::ArrowError; +use bytes::Buf; +use bytes::Bytes; +use datafusion_common::Result; +use futures::stream::BoxStream; +use futures::StreamExt as _; +use futures::{ready, Stream}; +use std::collections::VecDeque; +use std::fmt; +use std::task::Poll; + +/// Possible outputs of a [`BatchDeserializer`]. +#[derive(Debug, PartialEq)] +pub enum DeserializerOutput { + /// A successfully deserialized [`RecordBatch`]. + RecordBatch(RecordBatch), + /// The deserializer requires more data to make progress. + RequiresMoreData, + /// The input data has been exhausted. + InputExhausted, +} + +/// Trait defining a scheme for deserializing byte streams into structured data. +/// Implementors of this trait are responsible for converting raw bytes into +/// `RecordBatch` objects. +pub trait BatchDeserializer: Send + fmt::Debug { + /// Feeds a message for deserialization, updating the internal state of + /// this `BatchDeserializer`. Note that one can call this function multiple + /// times before calling `next`, which will queue multiple messages for + /// deserialization. Returns the number of bytes consumed. + fn digest(&mut self, message: T) -> usize; + + /// Attempts to deserialize any pending messages and returns a + /// `DeserializerOutput` to indicate progress. + fn next(&mut self) -> Result; + + /// Informs the deserializer that no more messages will be provided for + /// deserialization. + fn finish(&mut self); +} + +/// A general interface for decoders such as [`arrow::json::reader::Decoder`] and +/// [`arrow::csv::reader::Decoder`]. Defines an interface similar to +/// [`Decoder::decode`] and [`Decoder::flush`] methods, but also includes +/// a method to check if the decoder can flush early. Intended to be used in +/// conjunction with [`DecoderDeserializer`]. +/// +/// [`arrow::json::reader::Decoder`]: ::arrow::json::reader::Decoder +/// [`arrow::csv::reader::Decoder`]: ::arrow::csv::reader::Decoder +/// [`Decoder::decode`]: ::arrow::json::reader::Decoder::decode +/// [`Decoder::flush`]: ::arrow::json::reader::Decoder::flush +pub trait Decoder: Send + fmt::Debug { + /// See [`arrow::json::reader::Decoder::decode`]. + /// + /// [`arrow::json::reader::Decoder::decode`]: ::arrow::json::reader::Decoder::decode + fn decode(&mut self, buf: &[u8]) -> Result; + + /// See [`arrow::json::reader::Decoder::flush`]. + /// + /// [`arrow::json::reader::Decoder::flush`]: ::arrow::json::reader::Decoder::flush + fn flush(&mut self) -> Result, ArrowError>; + + /// Whether the decoder can flush early in its current state. + fn can_flush_early(&self) -> bool; +} + +impl fmt::Debug for DecoderDeserializer { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Deserializer") + .field("buffered_queue", &self.buffered_queue) + .field("finalized", &self.finalized) + .finish() + } +} + +impl BatchDeserializer for DecoderDeserializer { + fn digest(&mut self, message: Bytes) -> usize { + if message.is_empty() { + return 0; + } + + let consumed = message.len(); + self.buffered_queue.push_back(message); + consumed + } + + fn next(&mut self) -> Result { + while let Some(buffered) = self.buffered_queue.front_mut() { + let decoded = self.decoder.decode(buffered)?; + buffered.advance(decoded); + + if buffered.is_empty() { + self.buffered_queue.pop_front(); + } + + // Flush when the stream ends or batch size is reached + // Certain implementations can flush early + if decoded == 0 || self.decoder.can_flush_early() { + return match self.decoder.flush() { + Ok(Some(batch)) => Ok(DeserializerOutput::RecordBatch(batch)), + Ok(None) => continue, + Err(e) => Err(e), + }; + } + } + if self.finalized { + Ok(DeserializerOutput::InputExhausted) + } else { + Ok(DeserializerOutput::RequiresMoreData) + } + } + + fn finish(&mut self) { + self.finalized = true; + // Ensure the decoder is flushed: + self.buffered_queue.push_back(Bytes::new()); + } +} + +/// A generic, decoder-based deserialization scheme for processing encoded data. +/// +/// This struct is responsible for converting a stream of bytes, which represent +/// encoded data, into a stream of `RecordBatch` objects, following the specified +/// schema and formatting options. It also handles any buffering necessary to satisfy +/// the `Decoder` interface. +pub struct DecoderDeserializer { + /// The underlying decoder used for deserialization + pub(crate) decoder: T, + /// The buffer used to store the remaining bytes to be decoded + pub(crate) buffered_queue: VecDeque, + /// Whether the input stream has been fully consumed + pub(crate) finalized: bool, +} + +impl DecoderDeserializer { + /// Creates a new `DecoderDeserializer` with the provided decoder. + pub fn new(decoder: T) -> Self { + DecoderDeserializer { + decoder, + buffered_queue: VecDeque::new(), + finalized: false, + } + } +} + +/// Deserializes a stream of bytes into a stream of [`RecordBatch`] objects using the +/// provided deserializer. +/// +/// Returns a boxed stream of `Result`. The stream yields [`RecordBatch`] +/// objects as they are produced by the deserializer, or an [`ArrowError`] if an error +/// occurs while polling the input or deserializing. +pub fn deserialize_stream<'a>( + mut input: impl Stream> + Unpin + Send + 'a, + mut deserializer: impl BatchDeserializer + 'a, +) -> BoxStream<'a, Result> { + futures::stream::poll_fn(move |cx| loop { + match ready!(input.poll_next_unpin(cx)).transpose()? { + Some(b) => _ = deserializer.digest(b), + None => deserializer.finish(), + }; + + return match deserializer.next()? { + DeserializerOutput::RecordBatch(rb) => Poll::Ready(Some(Ok(rb))), + DeserializerOutput::InputExhausted => Poll::Ready(None), + DeserializerOutput::RequiresMoreData => continue, + }; + }) + .boxed() +} diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 79279b5c8231..d00552b19adf 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -501,7 +501,7 @@ impl FileScanConfig { (schema, constraints, stats, output_ordering) } - #[cfg_attr(not(feature = "avro"), allow(unused))] // Only used by avro + #[allow(unused)] // Only used by avro pub fn projected_file_column_names(&self) -> Option> { self.projection.as_ref().map(|p| { p.iter() diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index 0ed59758476a..240e3c82bbfc 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -24,6 +24,7 @@ //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. +pub mod decoder; pub mod display; pub mod file; pub mod file_compression_type; @@ -34,17 +35,23 @@ pub mod file_scan_config; pub mod file_sink_config; pub mod file_stream; pub mod memory; +pub mod schema_adapter; pub mod source; mod statistics; + #[cfg(test)] mod test_util; + pub mod url; pub mod write; use chrono::TimeZone; use datafusion_common::Result; use datafusion_common::{ScalarValue, Statistics}; -use futures::Stream; +use file_meta::FileMeta; +use futures::{Stream, StreamExt}; use object_store::{path::Path, ObjectMeta}; +use object_store::{GetOptions, GetRange, ObjectStore}; +use std::ops::Range; use std::pin::Pin; use std::sync::Arc; @@ -190,6 +197,110 @@ impl From for PartitionedFile { } } +/// Represents the possible outcomes of a range calculation. +/// +/// This enum is used to encapsulate the result of calculating the range of +/// bytes to read from an object (like a file) in an object store. +/// +/// Variants: +/// - `Range(Option>)`: +/// Represents a range of bytes to be read. It contains an `Option` wrapping a +/// `Range`. `None` signifies that the entire object should be read, +/// while `Some(range)` specifies the exact byte range to read. +/// - `TerminateEarly`: +/// Indicates that the range calculation determined no further action is +/// necessary, possibly because the calculated range is empty or invalid. +pub enum RangeCalculation { + Range(Option>), + TerminateEarly, +} + +/// Calculates an appropriate byte range for reading from an object based on the +/// provided metadata. +/// +/// This asynchronous function examines the `FileMeta` of an object in an object store +/// and determines the range of bytes to be read. The range calculation may adjust +/// the start and end points to align with meaningful data boundaries (like newlines). +/// +/// Returns a `Result` wrapping a `RangeCalculation`, which is either a calculated byte range or an indication to terminate early. +/// +/// Returns an `Error` if any part of the range calculation fails, such as issues in reading from the object store or invalid range boundaries. +pub async fn calculate_range( + file_meta: &FileMeta, + store: &Arc, + terminator: Option, +) -> Result { + let location = file_meta.location(); + let file_size = file_meta.object_meta.size; + let newline = terminator.unwrap_or(b'\n'); + + match file_meta.range { + None => Ok(RangeCalculation::Range(None)), + Some(FileRange { start, end }) => { + let (start, end) = (start as usize, end as usize); + + let start_delta = if start != 0 { + find_first_newline(store, location, start - 1, file_size, newline).await? + } else { + 0 + }; + + let end_delta = if end != file_size { + find_first_newline(store, location, end - 1, file_size, newline).await? + } else { + 0 + }; + + let range = start + start_delta..end + end_delta; + + if range.start == range.end { + return Ok(RangeCalculation::TerminateEarly); + } + + Ok(RangeCalculation::Range(Some(range))) + } + } +} + +/// Asynchronously finds the position of the first newline character in a specified byte range +/// within an object, such as a file, in an object store. +/// +/// This function scans the contents of the object starting from the specified `start` position +/// up to the `end` position, looking for the first occurrence of a newline character. +/// It returns the position of the first newline relative to the start of the range. +/// +/// Returns a `Result` wrapping a `usize` that represents the position of the first newline character found within the specified range. If no newline is found, it returns the length of the scanned data, effectively indicating the end of the range. +/// +/// The function returns an `Error` if any issues arise while reading from the object store or processing the data stream. +/// +async fn find_first_newline( + object_store: &Arc, + location: &Path, + start: usize, + end: usize, + newline: u8, +) -> Result { + let options = GetOptions { + range: Some(GetRange::Bounded(start..end)), + ..Default::default() + }; + + let result = object_store.get_opts(location, options).await?; + let mut result_stream = result.into_stream(); + + let mut index = 0; + + while let Some(chunk) = result_stream.next().await.transpose()? { + if let Some(position) = chunk.iter().position(|&byte| byte == newline) { + return Ok(index + position); + } + + index += chunk.len(); + } + + Ok(index) +} + #[cfg(test)] mod tests { use super::ListingTableUrl; diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/datasource/src/schema_adapter.rs similarity index 69% rename from datafusion/core/src/datasource/schema_adapter.rs rename to datafusion/datasource/src/schema_adapter.rs index 8076c114ad16..e3a4ea4918c1 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/datasource/src/schema_adapter.rs @@ -159,7 +159,7 @@ pub trait SchemaMapper: Debug + Send + Sync { /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{DataType, Field, Schema}; -/// # use datafusion::datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; +/// # use datafusion_datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; /// # use datafusion_common::record_batch; /// // Table has fields "a", "b" and "c" /// let table_schema = Schema::new(vec![ @@ -427,223 +427,3 @@ impl SchemaMapper for SchemaMapping { Ok(record_batch) } } - -#[cfg(test)] -mod tests { - use std::fs; - use std::sync::Arc; - - use crate::assert_batches_sorted_eq; - use arrow::array::{Int32Array, StringArray}; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use arrow::record_batch::RecordBatch; - use datafusion_datasource::file_scan_config::FileScanConfig; - use object_store::path::Path; - use object_store::ObjectMeta; - - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::ParquetSource; - use crate::datasource::schema_adapter::{ - DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, - }; - use crate::physical_plan::collect; - use crate::prelude::SessionContext; - - use datafusion_common::record_batch; - #[cfg(feature = "parquet")] - use parquet::arrow::ArrowWriter; - use tempfile::TempDir; - - #[tokio::test] - async fn can_override_schema_adapter() { - // Test shows that SchemaAdapter can add a column that doesn't existing in the - // record batches returned from parquet. This can be useful for schema evolution - // where older files may not have all columns. - let tmp_dir = TempDir::new().unwrap(); - let table_dir = tmp_dir.path().join("parquet_test"); - fs::DirBuilder::new().create(table_dir.as_path()).unwrap(); - let f1 = Field::new("id", DataType::Int32, true); - - let file_schema = Arc::new(Schema::new(vec![f1.clone()])); - let filename = "part.parquet".to_string(); - let path = table_dir.as_path().join(filename.clone()); - let file = fs::File::create(path.clone()).unwrap(); - let mut writer = ArrowWriter::try_new(file, file_schema.clone(), None).unwrap(); - - let ids = Arc::new(Int32Array::from(vec![1i32])); - let rec_batch = RecordBatch::try_new(file_schema.clone(), vec![ids]).unwrap(); - - writer.write(&rec_batch).unwrap(); - writer.close().unwrap(); - - let location = Path::parse(path.to_str().unwrap()).unwrap(); - let metadata = fs::metadata(path.as_path()).expect("Local file metadata"); - let meta = ObjectMeta { - location, - last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), - size: metadata.len() as usize, - e_tag: None, - version: None, - }; - - let partitioned_file = PartitionedFile { - object_meta: meta, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }; - - let f1 = Field::new("id", DataType::Int32, true); - let f2 = Field::new("extra_column", DataType::Utf8, true); - - let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); - let source = Arc::new( - ParquetSource::default() - .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})), - ); - let base_conf = - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema, source) - .with_file(partitioned_file); - - let parquet_exec = base_conf.build(); - - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); - let read = collect(parquet_exec, task_ctx).await.unwrap(); - - let expected = [ - "+----+--------------+", - "| id | extra_column |", - "+----+--------------+", - "| 1 | foo |", - "+----+--------------+", - ]; - - assert_batches_sorted_eq!(expected, &read); - } - - #[test] - fn default_schema_adapter() { - let table_schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]); - - // file has a subset of the table schema fields and different type - let file_schema = Schema::new(vec![ - Field::new("c", DataType::Float64, true), // not in table schema - Field::new("b", DataType::Float64, true), - ]); - - let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); - let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); - assert_eq!(indices, vec![1]); - - let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); - - let mapped_batch = mapper.map_batch(file_batch).unwrap(); - - // the mapped batch has the correct schema and the "b" column has been cast to Utf8 - let expected_batch = record_batch!( - ("a", Int32, vec![None, None]), // missing column filled with nulls - ("b", Utf8, vec!["1.0", "2.0"]) // b was cast to string and order was changed - ) - .unwrap(); - assert_eq!(mapped_batch, expected_batch); - } - - #[test] - fn default_schema_adapter_non_nullable_columns() { - let table_schema = Schema::new(vec![ - Field::new("a", DataType::Int32, false), // "a"" is declared non nullable - Field::new("b", DataType::Utf8, true), - ]); - let file_schema = Schema::new(vec![ - // since file doesn't have "a" it will be filled with nulls - Field::new("b", DataType::Float64, true), - ]); - - let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); - let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); - assert_eq!(indices, vec![0]); - - let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); - - // Mapping fails because it tries to fill in a non-nullable column with nulls - let err = mapper.map_batch(file_batch).unwrap_err().to_string(); - assert!(err.contains("Invalid argument error: Column 'a' is declared as non-nullable but contains null values"), "{err}"); - } - - #[derive(Debug)] - struct TestSchemaAdapterFactory; - - impl SchemaAdapterFactory for TestSchemaAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - _table_schema: SchemaRef, - ) -> Box { - Box::new(TestSchemaAdapter { - table_schema: projected_table_schema, - }) - } - } - - struct TestSchemaAdapter { - /// Schema for the table - table_schema: SchemaRef, - } - - impl SchemaAdapter for TestSchemaAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field = self.table_schema.field(index); - Some(file_schema.fields.find(field.name())?.0) - } - - fn map_schema( - &self, - file_schema: &Schema, - ) -> datafusion_common::Result<(Arc, Vec)> { - let mut projection = Vec::with_capacity(file_schema.fields().len()); - - for (file_idx, file_field) in file_schema.fields.iter().enumerate() { - if self.table_schema.fields().find(file_field.name()).is_some() { - projection.push(file_idx); - } - } - - Ok((Arc::new(TestSchemaMapping {}), projection)) - } - } - - #[derive(Debug)] - struct TestSchemaMapping {} - - impl SchemaMapper for TestSchemaMapping { - fn map_batch( - &self, - batch: RecordBatch, - ) -> datafusion_common::Result { - let f1 = Field::new("id", DataType::Int32, true); - let f2 = Field::new("extra_column", DataType::Utf8, true); - - let schema = Arc::new(Schema::new(vec![f1, f2])); - - let extra_column = Arc::new(StringArray::from(vec!["foo"])); - let mut new_columns = batch.columns().to_vec(); - new_columns.push(extra_column); - - Ok(RecordBatch::try_new(schema, new_columns).unwrap()) - } - - fn map_partial_batch( - &self, - batch: RecordBatch, - ) -> datafusion_common::Result { - self.map_batch(batch) - } - } -} diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs index ab025069bf76..9a9b98d5041b 100644 --- a/datafusion/datasource/src/test_util.rs +++ b/datafusion/datasource/src/test_util.rs @@ -15,21 +15,20 @@ // specific language governing permissions and limitations // under the License. +use crate::{ + file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, +}; + use std::sync::Arc; use arrow::datatypes::SchemaRef; -use datafusion_common::Statistics; +use datafusion_common::{Result, Statistics}; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use object_store::ObjectStore; -use crate::{ - file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, -}; -use datafusion_common::Result; - -/// Minimal [`FileSource`] implementation for use in tests. +/// Minimal [`crate::file::FileSource`] implementation for use in tests. #[derive(Clone, Default)] -pub struct MockSource { +pub(crate) struct MockSource { metrics: ExecutionPlanMetricsSet, projected_statistics: Option, } From bb76984edf2c825c60e8849b7122f2871d068ef6 Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Sat, 1 Mar 2025 20:36:42 +0000 Subject: [PATCH 2/7] Make avro non-default feature and fix some tests --- datafusion/catalog/src/session.rs | 3 ++ datafusion/core/Cargo.toml | 1 - .../core/src/datasource/listing/table.rs | 37 +++++++++---------- .../core/src/execution/session_state.rs | 6 ++- datafusion/core/tests/dataframe/mod.rs | 5 ++- 5 files changed, 29 insertions(+), 23 deletions(-) diff --git a/datafusion/catalog/src/session.rs b/datafusion/catalog/src/session.rs index 8915862dd028..9dd870e43568 100644 --- a/datafusion/catalog/src/session.rs +++ b/datafusion/catalog/src/session.rs @@ -129,6 +129,9 @@ pub trait Session: Send + Sync { self.table_options() .combine_with_session_config(self.config_options()) } + + /// Returns a mutable reference to [`TableOptions`] + fn table_options_mut(&mut self) -> &mut TableOptions; } /// Create a new task context instance from Session diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 477ac3e75310..af029b177607 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -60,7 +60,6 @@ default = [ "string_expressions", "unicode_expressions", "compression", - "avro", "parquet", "recursive_protection", ] diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 6fe3d08723bc..46e0ff2dec7d 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1184,8 +1184,6 @@ mod tests { assert_batches_eq, test::{columns, object_store::register_test_store}, }; - use datafusion_datasource_avro::AvroFormat; - use datafusion_physical_plan::collect; use arrow::compute::SortOptions; use arrow::record_batch::RecordBatch; @@ -1193,6 +1191,7 @@ mod tests { use datafusion_common::{assert_contains, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_physical_plan::collect; use datafusion_physical_plan::ExecutionPlanProperties; use tempfile::TempDir; @@ -1365,11 +1364,14 @@ mod tests { #[tokio::test] async fn read_empty_table() -> Result<()> { let ctx = SessionContext::new(); - let path = String::from("table/p1=v1/file.avro"); + let path = String::from("table/p1=v1/file.json"); register_test_store(&ctx, &[(&path, 100)]); - let opt = ListingOptions::new(Arc::new(AvroFormat)) - .with_file_extension(AvroFormat.get_ext()) + let format = JsonFormat::default(); + let ext = format.get_ext(); + + let opt = ListingOptions::new(Arc::new(format)) + .with_file_extension(ext) .with_table_partition_cols(vec![(String::from("p1"), DataType::Utf8)]) .with_target_partitions(4); @@ -1403,7 +1405,6 @@ mod tests { Ok(()) } - #[cfg(feature = "avro")] #[tokio::test] async fn test_assert_list_files_for_scan_grouping() -> Result<()> { // more expected partitions than files @@ -1480,9 +1481,9 @@ mod tests { // files that don't match the prefix or the default file extention assert_list_files_for_scan_grouping( &[ - "bucket/key-prefix/file0.avro", + "bucket/key-prefix/file0.json", "bucket/key-prefix/file1.parquet", - "bucket/other-prefix/roguefile.avro", + "bucket/other-prefix/roguefile.json", ], "test:///bucket/key-prefix/", 10, @@ -1493,7 +1494,6 @@ mod tests { Ok(()) } - #[cfg(feature = "avro")] #[tokio::test] async fn test_assert_list_files_for_multi_path() -> Result<()> { // more expected partitions than files @@ -1571,11 +1571,11 @@ mod tests { // files that don't match the prefix or the default file ext assert_list_files_for_multi_paths( &[ - "bucket/key1/file0.avro", + "bucket/key1/file0.json", "bucket/key1/file1.csv", - "bucket/key1/file2.avro", + "bucket/key1/file2.json", "bucket/key2/file3.csv", - "bucket/key2/file4.avro", + "bucket/key2/file4.json", "bucket/key3/file5.csv", ], &["test:///bucket/key1/", "test:///bucket/key3/"], @@ -1614,7 +1614,7 @@ mod tests { let ctx = SessionContext::new(); register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::>()); - let opt = ListingOptions::new(Arc::new(AvroFormat)) + let opt = ListingOptions::new(Arc::new(JsonFormat::default())) .with_file_extension_opt(file_ext) .with_target_partitions(target_partitions); @@ -1636,7 +1636,6 @@ mod tests { /// Check that the files listed by the table match the specified `output_partitioning` /// when the object store contains `files`. - #[cfg(feature = "avro")] async fn assert_list_files_for_multi_paths( files: &[&str], table_prefix: &[&str], @@ -1644,14 +1643,10 @@ mod tests { output_partitioning: usize, file_ext: Option<&str>, ) -> Result<()> { - use datafusion_datasource_avro::AvroFormat; - let ctx = SessionContext::new(); register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::>()); - let format = AvroFormat; - - let opt = ListingOptions::new(Arc::new(format)) + let opt = ListingOptions::new(Arc::new(JsonFormat::default())) .with_file_extension_opt(file_ext) .with_target_partitions(target_partitions); @@ -1710,6 +1705,7 @@ mod tests { Ok(()) } + #[cfg(feature = "parquet")] #[tokio::test] async fn test_insert_into_append_2_new_parquet_files_defaults() -> Result<()> { let mut config_map: HashMap = HashMap::new(); @@ -1728,6 +1724,7 @@ mod tests { Ok(()) } + #[cfg(feature = "parquet")] #[tokio::test] async fn test_insert_into_append_1_new_parquet_files_defaults() -> Result<()> { let mut config_map: HashMap = HashMap::new(); @@ -2014,6 +2011,7 @@ mod tests { ) .await?; } + #[cfg(feature = "parquet")] "parquet" => { session_ctx .register_parquet( @@ -2023,6 +2021,7 @@ mod tests { ) .await?; } + #[cfg(feature = "avro")] "avro" => { session_ctx .register_avro( diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index dc43af9fa5da..29e56ec2318f 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -262,6 +262,10 @@ impl Session for SessionState { fn table_options(&self) -> &TableOptions { self.table_options() } + + fn table_options_mut(&mut self) -> &mut TableOptions { + self.table_options_mut() + } } impl SessionState { @@ -846,7 +850,7 @@ impl SessionState { Session::default_table_options(self) } - /// Return mutable table options + /// Returns a mutable reference to [`TableOptions`] pub fn table_options_mut(&mut self) -> &mut TableOptions { &mut self.table_options } diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index b134ec54b13d..1875180d503b 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -54,7 +54,7 @@ use datafusion::execution::context::SessionContext; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::logical_expr::{ColumnarValue, Volatility}; use datafusion::prelude::{ - AvroReadOptions, CsvReadOptions, JoinType, NdJsonReadOptions, ParquetReadOptions, + CsvReadOptions, JoinType, NdJsonReadOptions, ParquetReadOptions, }; use datafusion::test_util::{ parquet_test_data, populate_csv_partitions, register_aggregate_csv, test_table, @@ -5263,6 +5263,7 @@ async fn register_non_csv_file() { ); } +#[cfg(feature = "avro")] #[tokio::test] async fn register_non_avro_file() { let ctx = SessionContext::new(); @@ -5270,7 +5271,7 @@ async fn register_non_avro_file() { .register_avro( "data", "tests/data/test_binary.parquet", - AvroReadOptions::default(), + datafusion::prelude::AvroReadOptions::default(), ) .await; assert_contains!( From 5ed105c74ccc1d0266f33cc311ee27f27d976fde Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Sat, 1 Mar 2025 20:47:30 +0000 Subject: [PATCH 3/7] . --- datafusion/catalog-listing/src/helpers.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/datafusion/catalog-listing/src/helpers.rs b/datafusion/catalog-listing/src/helpers.rs index f86bc0744816..dea5ac85f04f 100644 --- a/datafusion/catalog-listing/src/helpers.rs +++ b/datafusion/catalog-listing/src/helpers.rs @@ -1073,5 +1073,9 @@ mod tests { fn table_options(&self) -> &TableOptions { unimplemented!() } + + fn table_options_mut(&mut self) -> &mut TableOptions { + unimplemented!() + } } } From 022d10090a8e42b65944f2b56b491040a2159e2c Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Sat, 1 Mar 2025 21:13:12 +0000 Subject: [PATCH 4/7] feature flag avro --- datafusion/proto/Cargo.toml | 1 + datafusion/proto/src/logical_plan/mod.rs | 16 ++++++++++++--- datafusion/proto/src/physical_plan/mod.rs | 25 +++++++++++++++-------- 3 files changed, 31 insertions(+), 11 deletions(-) diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 00d4969182cf..39897cfcf2de 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -41,6 +41,7 @@ name = "datafusion_proto" default = ["parquet"] json = ["pbjson", "serde", "serde_json"] parquet = ["datafusion/parquet", "datafusion-common/parquet"] +avro = ["datafusion/avro", "datafusion-common/avro"] [dependencies] arrow = { workspace = true } diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 641dfe7b5fb8..f35c121236cb 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -35,6 +35,8 @@ use crate::{ use crate::protobuf::{proto_error, ToProtoError}; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::datasource::cte_worktable::CteWorkTable; +#[cfg(feature = "avro")] +use datafusion::datasource::file_format::avro::AvroFormat; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::file_format::{ @@ -43,8 +45,7 @@ use datafusion::datasource::file_format::{ use datafusion::{ datasource::{ file_format::{ - avro::AvroFormat, csv::CsvFormat, json::JsonFormat as OtherNdJsonFormat, - FileFormat, + csv::CsvFormat, json::JsonFormat as OtherNdJsonFormat, FileFormat, }, listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}, view::ViewTable, @@ -440,7 +441,15 @@ impl AsLogicalPlan for LogicalPlanNode { } Arc::new(json) } - FileFormatType::Avro(..) => Arc::new(AvroFormat), + #[cfg_attr(not(feature = "avro"), allow(unused_variables))] + FileFormatType::Avro(..) => { + #[cfg(feature = "avro")] + { + Arc::new(AvroFormat) + } + #[cfg(not(feature = "avro"))] + panic!("Unable to process avro file since `avro` feature is not enabled"); + } }; let table_paths = &scan @@ -1072,6 +1081,7 @@ impl AsLogicalPlan for LogicalPlanNode { })) } + #[cfg(feature = "avro")] if any.is::() { maybe_some_type = Some(FileFormatType::Avro(protobuf::AvroFormat {})) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index d0a31097b5cd..60972ac54ba7 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -29,9 +29,11 @@ use datafusion::datasource::file_format::file_compression_type::FileCompressionT use datafusion::datasource::file_format::json::JsonSink; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; +#[cfg(feature = "avro")] +use datafusion::datasource::physical_plan::AvroSource; #[cfg(feature = "parquet")] use datafusion::datasource::physical_plan::ParquetSource; -use datafusion::datasource::physical_plan::{AvroSource, CsvSource, FileScanConfig}; +use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig}; use datafusion::datasource::source::DataSourceExec; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; @@ -285,14 +287,20 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { #[cfg(not(feature = "parquet"))] panic!("Unable to process a Parquet PhysicalPlan when `parquet` feature is not enabled") } + #[cfg_attr(not(feature = "avro"), allow(unused_variables))] PhysicalPlanType::AvroScan(scan) => { - let conf = parse_protobuf_file_scan_config( - scan.base_conf.as_ref().unwrap(), - registry, - extension_codec, - Arc::new(AvroSource::new()), - )?; - Ok(conf.build()) + #[cfg(feature = "avro")] + { + let conf = parse_protobuf_file_scan_config( + scan.base_conf.as_ref().unwrap(), + registry, + extension_codec, + Arc::new(AvroSource::new()), + )?; + Ok(conf.build()) + } + #[cfg(not(feature = "avro"))] + panic!("Unable to process a Avro PhysicalPlan when `avro` feature is not enabled") } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { let input: Arc = into_physical_plan( @@ -1706,6 +1714,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { } } + #[cfg(feature = "avro")] if let Some(data_source_exec) = plan.downcast_ref::() { let data_source = data_source_exec.data_source(); if let Some(maybe_avro) = From 31c14a05c4b7dcbb7b5b8c92a1ca5427237bd993 Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Sat, 1 Mar 2025 21:19:18 +0000 Subject: [PATCH 5/7] no idea how this happened --- datafusion/proto/src/logical_plan/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index f35c121236cb..148856cd103c 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -445,7 +445,7 @@ impl AsLogicalPlan for LogicalPlanNode { FileFormatType::Avro(..) => { #[cfg(feature = "avro")] { - Arc::new(AvroFormat) + Arc::new(AvroFormat) } #[cfg(not(feature = "avro"))] panic!("Unable to process avro file since `avro` feature is not enabled"); From a49b3781363521e68e6b43e0a68b18f8aff07fad Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Mon, 3 Mar 2025 23:08:21 +0000 Subject: [PATCH 6/7] . --- datafusion/datasource/src/file_scan_config.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index d00552b19adf..8b5e8425e564 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -501,7 +501,6 @@ impl FileScanConfig { (schema, constraints, stats, output_ordering) } - #[allow(unused)] // Only used by avro pub fn projected_file_column_names(&self) -> Option> { self.projection.as_ref().map(|p| { p.iter() From dbce043cd25dd6112449d919499c9a82bad754e4 Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Tue, 4 Mar 2025 08:39:18 +0000 Subject: [PATCH 7/7] Fix test --- datafusion/core/src/datasource/listing/table.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 12e1b199249d..71e2d31ee958 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1658,11 +1658,11 @@ mod tests { // files that don't match the default file ext assert_list_files_for_exact_paths( &[ - "bucket/key1/file0.avro", + "bucket/key1/file0.json", "bucket/key1/file1.csv", - "bucket/key1/file2.avro", + "bucket/key1/file2.json", "bucket/key2/file3.csv", - "bucket/key2/file4.avro", + "bucket/key2/file4.json", "bucket/key3/file5.csv", ], 2, @@ -1780,7 +1780,7 @@ mod tests { let url = Url::parse("test://").unwrap(); ctx.register_object_store(&url, head_blocking_store.clone()); - let format = AvroFormat {}; + let format = JsonFormat::default(); let opt = ListingOptions::new(Arc::new(format)) .with_file_extension_opt(file_ext)