Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Add support for reading Arrow files #6337

Merged
merged 10 commits into from
May 16, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
96 changes: 96 additions & 0 deletions datafusion/core/src/datasource/file_format/arrow.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
// 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 Arrow format abstractions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you link to the format docs to make it clear this writer writes the IPC format https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format ?

Copy link
Contributor Author

@jonmmease jonmmease May 15, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done in f9053a3

//!
//! Works with files following the [Arrow IPC format](https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format)

use crate::datasource::file_format::FileFormat;
use crate::error::Result;
use crate::execution::context::SessionState;
use crate::physical_plan::file_format::{ArrowExec, FileScanConfig};
use crate::physical_plan::ExecutionPlan;
use arrow::ipc::reader::FileReader;
use arrow_schema::{Schema, SchemaRef};
use async_trait::async_trait;
use datafusion_common::Statistics;
use datafusion_physical_expr::PhysicalExpr;
use object_store::{GetResult, ObjectMeta, ObjectStore};
use std::any::Any;
use std::io::{Read, Seek};
use std::sync::Arc;

/// The default file extension of arrow files
pub const DEFAULT_ARROW_EXTENSION: &str = ".arrow";
/// Arrow `FileFormat` implementation.
#[derive(Default, Debug)]
pub struct ArrowFormat;

#[async_trait]
impl FileFormat for ArrowFormat {
fn as_any(&self) -> &dyn Any {
self
}

async fn infer_schema(
&self,
_state: &SessionState,
store: &Arc<dyn ObjectStore>,
objects: &[ObjectMeta],
) -> Result<SchemaRef> {
let mut schemas = vec![];
for object in objects {
let schema = match store.get(&object.location).await? {
GetResult::File(mut file, _) => read_arrow_schema_from_reader(&mut file)?,
r @ GetResult::Stream(_) => {
// TODO: Fetching entire file to get schema is potentially wasteful
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes I agree it is more than potentially 😆

I don't think we need to fix it for this PR, however. Maybe @tustvold has some ideas, and if not then I think we can just file a follow on ticket to track fetching only the parts that are needed

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Haha, yeah, I carried that comment over from the Avro reader as the situation is the same here. But maybe there's some way to look at the start of the stream and parse out the schema?

let data = r.bytes().await?;
let mut cursor = std::io::Cursor::new(&data);
read_arrow_schema_from_reader(&mut cursor)?
}
};
schemas.push(schema.as_ref().clone());
}
let merged_schema = Schema::try_merge(schemas)?;
Ok(Arc::new(merged_schema))
}

async fn infer_stats(
&self,
_state: &SessionState,
_store: &Arc<dyn ObjectStore>,
_table_schema: SchemaRef,
_object: &ObjectMeta,
) -> Result<Statistics> {
Ok(Statistics::default())
}

async fn create_physical_plan(
&self,
_state: &SessionState,
conf: FileScanConfig,
_filters: Option<&Arc<dyn PhysicalExpr>>,
) -> Result<Arc<dyn ExecutionPlan>> {
let exec = ArrowExec::new(conf);
Ok(Arc::new(exec))
}
}

fn read_arrow_schema_from_reader<R: Read + Seek>(reader: R) -> Result<SchemaRef> {
let reader = FileReader::try_new(reader, None)?;
Ok(reader.schema())
}
7 changes: 6 additions & 1 deletion datafusion/core/src/datasource/file_format/file_type.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ use datafusion_common::parsers::CompressionTypeVariant;
#[cfg(feature = "compression")]
use flate2::read::MultiGzDecoder;

use crate::datasource::file_format::arrow::DEFAULT_ARROW_EXTENSION;
use futures::stream::BoxStream;
use futures::StreamExt;
#[cfg(feature = "compression")]
Expand Down Expand Up @@ -211,6 +212,8 @@ impl FileCompressionType {
/// Readable file type
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum FileType {
/// Apache Arrow file
ARROW,
/// Apache Avro file
AVRO,
/// Apache Parquet file
Expand All @@ -224,6 +227,7 @@ pub enum FileType {
impl GetExt for FileType {
fn get_ext(&self) -> String {
match self {
FileType::ARROW => DEFAULT_ARROW_EXTENSION.to_owned(),
FileType::AVRO => DEFAULT_AVRO_EXTENSION.to_owned(),
FileType::PARQUET => DEFAULT_PARQUET_EXTENSION.to_owned(),
FileType::CSV => DEFAULT_CSV_EXTENSION.to_owned(),
Expand All @@ -238,6 +242,7 @@ impl FromStr for FileType {
fn from_str(s: &str) -> Result<Self> {
let s = s.to_uppercase();
match s.as_str() {
"ARROW" => Ok(FileType::ARROW),
"AVRO" => Ok(FileType::AVRO),
"PARQUET" => Ok(FileType::PARQUET),
"CSV" => Ok(FileType::CSV),
Expand All @@ -256,7 +261,7 @@ impl FileType {

match self {
FileType::JSON | FileType::CSV => Ok(format!("{}{}", ext, c.get_ext())),
FileType::PARQUET | FileType::AVRO => match c.variant {
FileType::PARQUET | FileType::AVRO | FileType::ARROW => match c.variant {
UNCOMPRESSED => Ok(ext),
_ => Err(DataFusionError::Internal(
"FileCompressionType can be specified for CSV/JSON FileType.".into(),
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/src/datasource/file_format/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
/// Default max records to scan to infer the schema
pub const DEFAULT_SCHEMA_INFER_MAX_RECORD: usize = 1000;

pub mod arrow;
pub mod avro;
pub mod csv;
pub mod file_type;
Expand Down
69 changes: 69 additions & 0 deletions datafusion/core/src/datasource/file_format/options.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ use arrow::datatypes::{DataType, Schema, SchemaRef};
use async_trait::async_trait;
use datafusion_common::DataFusionError;

use crate::datasource::file_format::arrow::{ArrowFormat, DEFAULT_ARROW_EXTENSION};
use crate::datasource::file_format::avro::DEFAULT_AVRO_EXTENSION;
use crate::datasource::file_format::csv::DEFAULT_CSV_EXTENSION;
use crate::datasource::file_format::file_type::FileCompressionType;
Expand Down Expand Up @@ -214,6 +215,52 @@ impl<'a> ParquetReadOptions<'a> {
}
}

/// Options that control the reading of ARROW files.
///
/// Note this structure is supplied when a datasource is created and
/// can not not vary from statement to statement. For settings that
/// can vary statement to statement see
/// [`ConfigOptions`](crate::config::ConfigOptions).
#[derive(Clone)]
pub struct ArrowReadOptions<'a> {
/// The data source schema.
pub schema: Option<&'a Schema>,

/// File extension; only files with this extension are selected for data input.
/// Defaults to `FileType::ARROW.get_ext().as_str()`.
pub file_extension: &'a str,

/// Partition Columns
pub table_partition_cols: Vec<(String, DataType)>,
}

impl<'a> Default for ArrowReadOptions<'a> {
fn default() -> Self {
Self {
schema: None,
file_extension: DEFAULT_ARROW_EXTENSION,
table_partition_cols: vec![],
}
}
}

impl<'a> ArrowReadOptions<'a> {
/// Specify table_partition_cols for partition pruning
pub fn table_partition_cols(
mut self,
table_partition_cols: Vec<(String, DataType)>,
) -> Self {
self.table_partition_cols = table_partition_cols;
self
}

/// Specify schema to use for AVRO read
pub fn schema(mut self, schema: &'a Schema) -> Self {
self.schema = Some(schema);
self
}
}

/// Options that control the reading of AVRO files.
///
/// Note this structure is supplied when a datasource is created and
Expand Down Expand Up @@ -484,3 +531,25 @@ impl ReadOptions<'_> for AvroReadOptions<'_> {
.await
}
}

#[async_trait]
impl ReadOptions<'_> for ArrowReadOptions<'_> {
fn to_listing_options(&self, config: &SessionConfig) -> ListingOptions {
let file_format = ArrowFormat::default();

ListingOptions::new(Arc::new(file_format))
.with_file_extension(self.file_extension)
.with_target_partitions(config.target_partitions())
.with_table_partition_cols(self.table_partition_cols.clone())
}

async fn get_resolved_schema(
&self,
config: &SessionConfig,
state: SessionState,
table_path: ListingTableUrl,
) -> Result<SchemaRef> {
self._get_resolved_schema(config, state, table_path, self.schema, false)
.await
}
}
5 changes: 3 additions & 2 deletions datafusion/core/src/datasource/listing/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,8 @@ use object_store::ObjectMeta;
use crate::datasource::file_format::file_type::{FileCompressionType, FileType};
use crate::datasource::{
file_format::{
avro::AvroFormat, csv::CsvFormat, json::JsonFormat, parquet::ParquetFormat,
FileFormat,
arrow::ArrowFormat, avro::AvroFormat, csv::CsvFormat, json::JsonFormat,
parquet::ParquetFormat, FileFormat,
},
get_statistics_with_limit,
listing::ListingTableUrl,
Expand Down Expand Up @@ -135,6 +135,7 @@ impl ListingTableConfig {
.map_err(|_| DataFusionError::Internal(err_msg))?;

let file_format: Arc<dyn FileFormat> = match file_type {
FileType::ARROW => Arc::new(ArrowFormat::default()),
FileType::AVRO => Arc::new(AvroFormat::default()),
FileType::CSV => Arc::new(
CsvFormat::default().with_file_compression_type(file_compression_type),
Expand Down
2 changes: 2 additions & 0 deletions datafusion/core/src/datasource/listing_table_factory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ use datafusion_common::DataFusionError;
use datafusion_expr::CreateExternalTable;

use crate::datasource::datasource::TableProviderFactory;
use crate::datasource::file_format::arrow::ArrowFormat;
use crate::datasource::file_format::avro::AvroFormat;
use crate::datasource::file_format::csv::CsvFormat;
use crate::datasource::file_format::file_type::{FileCompressionType, FileType};
Expand Down Expand Up @@ -81,6 +82,7 @@ impl TableProviderFactory for ListingTableFactory {
FileType::JSON => Arc::new(
JsonFormat::default().with_file_compression_type(file_compression_type),
),
FileType::ARROW => Arc::new(ArrowFormat::default()),
};

let (provided_schema, table_partition_cols) = if cmd.schema.fields().is_empty() {
Expand Down
37 changes: 37 additions & 0 deletions datafusion/core/src/execution/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,7 @@ use datafusion_sql::planner::object_name_to_table_reference;
use uuid::Uuid;

// backwards compatibility
use crate::execution::options::ArrowReadOptions;
use crate::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate;
pub use datafusion_execution::config::SessionConfig;
pub use datafusion_execution::TaskContext;
Expand Down Expand Up @@ -844,6 +845,20 @@ impl SessionContext {
self._read_type(table_paths, options).await
}

/// Creates a [`DataFrame`] for reading an Arrow data source.
///
/// For more control such as reading multiple files, you can use
/// [`read_table`](Self::read_table) with a [`ListingTable`].
///
/// For an example, see [`read_csv`](Self::read_csv)
pub async fn read_arrow<P: DataFilePaths>(
&self,
table_paths: P,
options: ArrowReadOptions<'_>,
) -> Result<DataFrame> {
self._read_type(table_paths, options).await
}

/// Creates an empty DataFrame.
pub fn read_empty(&self) -> Result<DataFrame> {
Ok(DataFrame::new(
Expand Down Expand Up @@ -1034,6 +1049,27 @@ impl SessionContext {
Ok(())
}

/// Registers an Arrow file as a table that can be referenced from
/// SQL statements executed against this context.
pub async fn register_arrow(
&self,
name: &str,
table_path: &str,
options: ArrowReadOptions<'_>,
) -> Result<()> {
let listing_options = options.to_listing_options(&self.copied_config());

self.register_listing_table(
name,
table_path,
listing_options,
options.schema.map(|s| Arc::new(s.to_owned())),
None,
)
.await?;
Ok(())
}

/// Registers a named catalog using a custom `CatalogProvider` so that
/// it can be referenced from SQL statements executed against this
/// context.
Expand Down Expand Up @@ -1360,6 +1396,7 @@ impl SessionState {
table_factories.insert("JSON".into(), Arc::new(ListingTableFactory::new()));
table_factories.insert("NDJSON".into(), Arc::new(ListingTableFactory::new()));
table_factories.insert("AVRO".into(), Arc::new(ListingTableFactory::new()));
table_factories.insert("ARROW".into(), Arc::new(ListingTableFactory::new()));

if config.create_default_catalog_and_schema() {
let default_catalog = MemoryCatalogProvider::new();
Expand Down
Loading