From 7e14f910b8bdda9f657a88c97512c62660c4808c Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 18 Aug 2021 00:25:39 +0200 Subject: [PATCH 01/58] Add avro as a datasource, file and table provider --- .gitmodules | 3 +- Cargo.toml | 1 + avro-converter/Cargo.toml | 24 ++ avro-converter/src/main.rs | 45 +++ datafusion-examples/examples/avro_sql.rs | 54 +++ datafusion/Cargo.toml | 1 + datafusion/src/avro/mod.rs | 295 ++++++++++++++ datafusion/src/datasource/avro.rs | 206 ++++++++++ datafusion/src/datasource/mod.rs | 1 + datafusion/src/error.rs | 12 + datafusion/src/execution/context.rs | 14 + datafusion/src/lib.rs | 1 + datafusion/src/physical_plan/avro.rs | 472 +++++++++++++++++++++++ datafusion/src/physical_plan/common.rs | 14 + datafusion/src/physical_plan/mod.rs | 1 + testing | 2 +- 16 files changed, 1144 insertions(+), 2 deletions(-) create mode 100644 avro-converter/Cargo.toml create mode 100644 avro-converter/src/main.rs create mode 100644 datafusion-examples/examples/avro_sql.rs create mode 100644 datafusion/src/avro/mod.rs create mode 100644 datafusion/src/datasource/avro.rs create mode 100644 datafusion/src/physical_plan/avro.rs diff --git a/.gitmodules b/.gitmodules index ec5d6208b8dd..52454695e343 100644 --- a/.gitmodules +++ b/.gitmodules @@ -3,4 +3,5 @@ url = https://github.com/apache/parquet-testing.git [submodule "testing"] path = testing - url = https://github.com/apache/arrow-testing + url = https://github.com/Igosuki/arrow-testing.git + branch = avro diff --git a/Cargo.toml b/Cargo.toml index d6da8c14cd96..a454800fd506 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,6 +26,7 @@ members = [ "ballista/rust/executor", "ballista/rust/scheduler", "ballista-examples", + "avro-converter" ] exclude = ["python"] diff --git a/avro-converter/Cargo.toml b/avro-converter/Cargo.toml new file mode 100644 index 000000000000..e128bda12230 --- /dev/null +++ b/avro-converter/Cargo.toml @@ -0,0 +1,24 @@ +[package] +name = "avro-converter" +version = "4.0.0-SNAPSHOT" +edition = "2018" +description = "DataFusion avro to parquet converter" +homepage = "https://github.com/apache/arrow-datafusion" +repository = "https://github.com/apache/arrow-datafusion" +authors = ["Apache Arrow "] +license = "Apache-2.0" +keywords = [ "arrow", "avro", "parquet" ] + +[[bin]] +path = "src/main.rs" +name = "avro-converter" + +[dependencies] +arrow-flight = { version = "5.0" } +datafusion = { path = "../datafusion" } +prost = "0.8" +tonic = "0.5" +tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync"] } +futures = "0.3" +num_cpus = "1.13.0" +structopt = { version = "0.3" } diff --git a/avro-converter/src/main.rs b/avro-converter/src/main.rs new file mode 100644 index 000000000000..b99e6bf3d973 --- /dev/null +++ b/avro-converter/src/main.rs @@ -0,0 +1,45 @@ +#[macro_use] +extern crate structopt; + +use datafusion::arrow::util::pretty; + +use crate::structopt::StructOpt; +use datafusion::error::Result; +use datafusion::prelude::*; +use std::path::PathBuf; + +fn parse_csv(src: &str) -> Vec { + src.split(',').map(|s| s.to_string()).collect() +} + +#[derive(StructOpt, Debug)] +#[structopt(name = "basic")] +struct ArrowConverterOptions { + #[structopt(short, long, parse(from_os_str))] + input: PathBuf, + #[structopt(short, long, parse(from_os_str))] + output: PathBuf, + #[structopt(short, long)] + partitions: Option, +} + +#[tokio::main] +async fn main() -> Result<()> { + // create local execution context + let mut ctx = ExecutionContext::new(); + + let opts: ArrowConverterOptions = ArrowConverterOptions::from_args(); + + // define the query using the DataFrame trait + let df = ctx + .read_parquet(opts.input.to_str().unwrap())? + .select_columns(&["id", "bool_col", "timestamp_col"])?; + + // execute the query + let results = df.collect().await?; + + // print the results + pretty::print_batches(&results)?; + + Ok(()) +} diff --git a/datafusion-examples/examples/avro_sql.rs b/datafusion-examples/examples/avro_sql.rs new file mode 100644 index 000000000000..50714ec45a64 --- /dev/null +++ b/datafusion-examples/examples/avro_sql.rs @@ -0,0 +1,54 @@ +// 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. + +use datafusion::arrow::util::pretty; + +use datafusion::error::Result; +use datafusion::physical_plan::avro::AvroReadOptions; +use datafusion::prelude::*; + +/// This example demonstrates executing a simple query against an Arrow data source (Parquet) and +/// fetching results +#[tokio::main] +async fn main() -> Result<()> { + // create local execution context + let mut ctx = ExecutionContext::new(); + + let testdata = datafusion::arrow::util::test_util::arrow_test_data(); + + // register parquet file with the execution context + ctx.register_avro( + "alltypes_plain", + &format!("{}/avro/alltypes_plain.avro", testdata), + AvroReadOptions::default(), + )?; + + let df = ctx.table("alltypes_plain").unwrap(); + println!("schema {:?}", df.schema()); + // execute the query + let df = ctx.sql( + "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ + FROM alltypes_plain \ + WHERE id > 1 AND tinyint_col < double_col", + )?; + let results = df.collect().await?; + + // print the results + pretty::print_batches(&results)?; + + Ok(()) +} diff --git a/datafusion/Cargo.toml b/datafusion/Cargo.toml index f30db0296565..061a72f5030f 100644 --- a/datafusion/Cargo.toml +++ b/datafusion/Cargo.toml @@ -69,6 +69,7 @@ regex = { version = "^1.4.3", optional = true } lazy_static = { version = "^1.4.0", optional = true } smallvec = { version = "1.6", features = ["union"] } rand = "0.8" +avro-rs = "0.13" [dev-dependencies] criterion = "0.3" diff --git a/datafusion/src/avro/mod.rs b/datafusion/src/avro/mod.rs new file mode 100644 index 000000000000..27e8bc63f20c --- /dev/null +++ b/datafusion/src/avro/mod.rs @@ -0,0 +1,295 @@ +// 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. + +//! This module contains utilities to manipulate avro metadata. + +use crate::arrow::datatypes::{DataType, IntervalUnit, Schema, TimeUnit}; +use crate::error::Result; +use arrow::datatypes::Field; +use avro_rs::schema::Name; +use avro_rs::types::Value; +use avro_rs::Schema as AvroSchema; +use std::collections::BTreeMap; +use std::convert::TryFrom; +use std::ptr::null; + +/// Converts an avro schema to an arrow schema +pub fn to_arrow_schema(avro_schema: &avro_rs::Schema) -> Result { + let mut schema_fields = vec![]; + match avro_schema { + AvroSchema::Record { fields, .. } => { + for field in fields { + schema_fields.push(schema_to_field_with_props( + &field.schema, + Some(&field.name), + false, + Some(&external_props(&field.schema)), + )?) + } + } + schema => schema_fields.push(schema_to_field(schema, Some(""), false)?), + } + + let schema = Schema::new(schema_fields); + Ok(schema) +} + +fn schema_to_field( + schema: &avro_rs::Schema, + name: Option<&str>, + nullable: bool, +) -> Result { + schema_to_field_with_props(schema, name, nullable, Some(&Default::default())) +} + +fn schema_to_field_with_props( + schema: &AvroSchema, + name: Option<&str>, + nullable: bool, + props: Option<&BTreeMap>, +) -> Result { + let mut nullable = nullable; + let field_type: DataType = match schema { + AvroSchema::Null => DataType::Null, + AvroSchema::Boolean => DataType::Boolean, + AvroSchema::Int => DataType::Int32, + AvroSchema::Long => DataType::Int64, + AvroSchema::Float => DataType::Float32, + AvroSchema::Double => DataType::Float64, + AvroSchema::Bytes => DataType::Binary, + AvroSchema::String => DataType::Utf8, + AvroSchema::Array(item_schema) => DataType::List(Box::new( + schema_to_field_with_props(item_schema, None, false, None)?, + )), + AvroSchema::Map(value_schema) => { + let value_field = + schema_to_field_with_props(value_schema, Some("value"), false, None)?; + DataType::Dictionary( + Box::new(DataType::Utf8), + Box::new(value_field.data_type().clone()), + ) + } + AvroSchema::Union(us) => { + nullable = us.find_schema(&Value::Null).is_some(); + let fields: Result> = us + .variants() + .into_iter() + .map(|s| schema_to_field_with_props(&s, None, nullable, None)) + .collect(); + DataType::Union(fields?) + } + AvroSchema::Record { name, fields, .. } => { + let fields: Result> = fields + .into_iter() + .map(|field| { + let mut props = BTreeMap::new(); + if let Some(doc) = &field.doc { + props.insert("doc".to_string(), doc.clone()); + } + /*if let Some(aliases) = fields.aliases { + props.insert("aliases", aliases); + }*/ + schema_to_field_with_props( + &field.schema, + Some(&format!("{}.{}", name.fullname(None), field.name)), + false, + Some(&props), + ) + }) + .collect(); + DataType::Struct(fields?) + } + AvroSchema::Enum { symbols, name, .. } => { + return Ok(Field::new_dict( + &name.fullname(None), + index_type(symbols.len()), + false, + 0, + false, + )) + } + AvroSchema::Fixed { size, .. } => DataType::FixedSizeBinary(*size as i32), + AvroSchema::Decimal { + precision, scale, .. + } => DataType::Decimal(*precision, *scale), + AvroSchema::Uuid => DataType::Utf8, + AvroSchema::Date => DataType::Date32, + AvroSchema::TimeMillis => DataType::Time32(TimeUnit::Millisecond), + AvroSchema::TimeMicros => DataType::Time64(TimeUnit::Microsecond), + AvroSchema::TimestampMillis => DataType::Timestamp(TimeUnit::Millisecond, None), + AvroSchema::TimestampMicros => DataType::Timestamp(TimeUnit::Microsecond, None), + AvroSchema::Duration => DataType::Duration(TimeUnit::Millisecond), + }; + + let data_type = field_type.clone(); + let name = name.unwrap_or_else(|| default_field_name(&data_type)); + + let mut field = Field::new(name, field_type, nullable); + field.set_metadata(props.cloned()); + Ok(field) +} + +fn default_field_name(dt: &DataType) -> &str { + match dt { + DataType::Null => "null", + DataType::Boolean => "bit", + DataType::Int8 => "tinyint", + DataType::Int16 => "smallint", + DataType::Int32 => "int", + DataType::Int64 => "bigint", + DataType::UInt8 => "uint1", + DataType::UInt16 => "uint2", + DataType::UInt32 => "uint4", + DataType::UInt64 => "uint8", + DataType::Float16 => "float2", + DataType::Float32 => "float4", + DataType::Float64 => "float8", + DataType::Date32 => "dateday", + DataType::Date64 => "datemilli", + DataType::Time32(tu) | DataType::Time64(tu) => match tu { + TimeUnit::Second => "timesec", + TimeUnit::Millisecond => "timemilli", + TimeUnit::Microsecond => "timemicro", + TimeUnit::Nanosecond => "timenano", + }, + DataType::Timestamp(tu, tz) => { + if tz.is_some() { + match tu { + TimeUnit::Second => "timestampsectz", + TimeUnit::Millisecond => "timestampmillitz", + TimeUnit::Microsecond => "timestampmicrotz", + TimeUnit::Nanosecond => "timestampnanotz", + } + } else { + match tu { + TimeUnit::Second => "timestampsec", + TimeUnit::Millisecond => "timestampmilli", + TimeUnit::Microsecond => "timestampmicro", + TimeUnit::Nanosecond => "timestampnano", + } + } + } + DataType::Duration(_) => "duration", + DataType::Interval(unit) => match unit { + IntervalUnit::YearMonth => "intervalyear", + IntervalUnit::DayTime => "intervalmonth", + }, + DataType::Binary => "varbinary", + DataType::FixedSizeBinary(_) => "fixedsizebinary", + DataType::LargeBinary => "largevarbinary", + DataType::Utf8 => "varchar", + DataType::LargeUtf8 => "largevarchar", + DataType::List(_) => "list", + DataType::FixedSizeList(_, _) => "fixed_size_list", + DataType::LargeList(_) => "largelist", + DataType::Struct(_) => "struct", + DataType::Union(_) => "union", + DataType::Dictionary(_, _) => "map", + DataType::Decimal(_, _) => "decimal", + } +} + +fn index_type(len: usize) -> DataType { + if len <= usize::from(u8::MAX) { + DataType::Int8 + } else if len <= usize::from(u16::MAX) { + DataType::Int16 + } else if usize::try_from(u32::MAX).map(|i| len < i).unwrap_or(false) { + DataType::Int32 + } else { + DataType::Int64 + } +} + +fn external_props(schema: &AvroSchema) -> BTreeMap { + let mut props = BTreeMap::new(); + match &schema { + AvroSchema::Record { + doc: Some(ref doc), .. + } + | AvroSchema::Enum { + doc: Some(ref doc), .. + } => { + props.insert("doc".to_string(), doc.clone()); + } + _ => {} + } + match &schema { + AvroSchema::Record { + name: + Name { + aliases: Some(aliases), + namespace, + .. + }, + .. + } + | AvroSchema::Enum { + name: + Name { + aliases: Some(aliases), + namespace, + .. + }, + .. + } + | AvroSchema::Fixed { + name: + Name { + aliases: Some(aliases), + namespace, + .. + }, + .. + } => { + let aliases: Vec = aliases + .into_iter() + .map(|alias| fullname(alias, namespace.as_ref(), None)) + .collect(); + props.insert("aliases".to_string(), format!("[{}]", aliases.join(","))); + } + _ => {} + } + props +} + +fn get_metadata( + _schema: AvroSchema, + props: BTreeMap, +) -> BTreeMap { + let mut metadata: BTreeMap = Default::default(); + metadata.extend(props); + return metadata; +} + +/// Returns the fully qualified name for a field +pub fn fullname( + name: &str, + namespace: Option<&String>, + default_namespace: Option<&str>, +) -> String { + if name.contains('.') { + name.to_string() + } else { + let namespace = namespace.as_ref().map(|s| s.as_ref()).or(default_namespace); + + match namespace { + Some(ref namespace) => format!("{}.{}", namespace, name), + None => name.to_string(), + } + } +} diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs new file mode 100644 index 000000000000..8efe55590e69 --- /dev/null +++ b/datafusion/src/datasource/avro.rs @@ -0,0 +1,206 @@ +// 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. + +//! Line-delimited Avro data source +//! +//! This data source allows Line-delimited Avro records or files to be used as input for queries. +//! + +use std::{ + any::Any, + io::{Read, Seek}, + sync::{Arc, Mutex}, +}; + +use crate::{ + datasource::{Source, TableProvider}, + error::{DataFusionError, Result}, + physical_plan::{common, ExecutionPlan}, +}; +use arrow::datatypes::SchemaRef; + +use super::datasource::Statistics; +use crate::physical_plan::avro::{AvroExec, AvroReadOptions}; + +trait SeekRead: Read + Seek {} + +impl SeekRead for T {} + +/// Represents a line-delimited JSON file with a provided schema +pub struct AvroFile { + source: Source>, + schema: SchemaRef, + file_extension: String, + statistics: Statistics, +} + +impl AvroFile { + /// Attempt to initialize a `AvroFile` from a path. The schema can be inferred automatically. + pub fn try_new(path: &str, options: AvroReadOptions) -> Result { + let schema = if let Some(schema) = options.schema { + schema.clone() + } else { + let filenames = + common::build_checked_file_list(path, options.file_extension)?; + Arc::new(AvroExec::try_infer_schema(&filenames)?) + }; + + Ok(Self { + source: Source::Path(path.to_string()), + schema, + file_extension: options.file_extension.to_string(), + statistics: Statistics::default(), + }) + } + + /// Attempt to initialize a `AvroFile` from a reader. The schema MUST be provided in options + pub fn try_new_from_reader( + reader: R, + options: AvroReadOptions, + ) -> Result { + let schema = match options.schema { + Some(s) => s.clone(), + None => { + return Err(DataFusionError::Execution( + "Schema must be provided to CsvRead".to_string(), + )); + } + }; + Ok(Self { + source: Source::Reader(Mutex::new(Some(Box::new(reader)))), + schema, + statistics: Statistics::default(), + file_extension: String::new(), + }) + } + + /// Attempt to initialize an AvroFile from a reader impls Seek. The schema can be inferred automatically. + pub fn try_new_from_reader_infer_schema( + mut reader: R, + options: AvroReadOptions, + ) -> Result { + let schema = { + if let Some(schema) = options.schema { + schema + } else { + Arc::new(AvroExec::infer_avro_schema_from_reader(&mut reader)?) + } + }; + + Ok(Self { + source: Source::Reader(Mutex::new(Some(Box::new(reader)))), + schema, + statistics: Statistics::default(), + file_extension: String::new(), + }) + } + + /// Get the path for Avro file(s) represented by this AvroFile instance + pub fn path(&self) -> &str { + match &self.source { + Source::Reader(_) => "", + Source::Path(path) => path, + } + } + + /// Get the file extension for the Avro file(s) represented by this AvroFile instance + pub fn file_extension(&self) -> &str { + &self.file_extension + } +} + +impl TableProvider for AvroFile { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn scan( + &self, + projection: &Option>, + batch_size: usize, + _filters: &[crate::logical_plan::Expr], + limit: Option, + ) -> Result> { + let opts = AvroReadOptions { + schema: Some(self.schema.clone()), + file_extension: self.file_extension.as_str(), + }; + let batch_size = limit + .map(|l| std::cmp::min(l, batch_size)) + .unwrap_or(batch_size); + + let exec = match &self.source { + Source::Reader(maybe_reader) => { + if let Some(rdr) = maybe_reader.lock().unwrap().take() { + AvroExec::try_new_from_reader( + rdr, + opts, + projection.clone(), + batch_size, + limit, + )? + } else { + return Err(DataFusionError::Execution( + "You can only read once if the data comes from a reader" + .to_string(), + )); + } + } + Source::Path(p) => { + AvroExec::try_new(p, opts, projection.clone(), batch_size, limit)? + } + }; + Ok(Arc::new(exec)) + } + + fn statistics(&self) -> Statistics { + self.statistics.clone() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::prelude::*; + const TEST_DATA_BASE: &str = "tests/jsons"; + + #[tokio::test] + async fn csv_file_from_reader() -> Result<()> { + let mut ctx = ExecutionContext::new(); + let path = format!("{}/2.json", TEST_DATA_BASE); + ctx.register_table( + "ndjson", + Arc::new(AvroFile::try_new(&path, Default::default())?), + )?; + let df = ctx.sql("select sum(a) from ndjson")?; + let batches = df.collect().await?; + assert_eq!( + batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0), + 100000000000011 + ); + Ok(()) + } +} diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index 53ba5177a2fc..cfa90036b163 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -17,6 +17,7 @@ //! DataFusion data sources +pub mod avro; pub mod csv; pub mod datasource; pub mod empty; diff --git a/datafusion/src/error.rs b/datafusion/src/error.rs index 903faeabf695..3f079859e34d 100644 --- a/datafusion/src/error.rs +++ b/datafusion/src/error.rs @@ -23,6 +23,7 @@ use std::io; use std::result; use arrow::error::ArrowError; +use avro_rs::Error as AvroError; use parquet::errors::ParquetError; use sqlparser::parser::ParserError; @@ -37,6 +38,8 @@ pub enum DataFusionError { ArrowError(ArrowError), /// Wraps an error from the Parquet crate ParquetError(ParquetError), + /// Wraps an error from the Avro crate + AvroError(AvroError), /// Error associated to I/O operations and associated traits. IoError(io::Error), /// Error returned when SQL is syntactically incorrect. @@ -83,6 +86,12 @@ impl From for DataFusionError { } } +impl From for DataFusionError { + fn from(e: AvroError) -> Self { + DataFusionError::AvroError(e) + } +} + impl From for DataFusionError { fn from(e: ParserError) -> Self { DataFusionError::SQL(e) @@ -96,6 +105,9 @@ impl Display for DataFusionError { DataFusionError::ParquetError(ref desc) => { write!(f, "Parquet error: {}", desc) } + DataFusionError::AvroError(ref desc) => { + write!(f, "Avro error: {}", desc) + } DataFusionError::IoError(ref desc) => write!(f, "IO error: {}", desc), DataFusionError::SQL(ref desc) => { write!(f, "SQL error: {:?}", desc) diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index 82947aaee1ba..b6926d46a228 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -67,6 +67,8 @@ use crate::physical_optimizer::coalesce_batches::CoalesceBatches; use crate::physical_optimizer::merge_exec::AddCoalescePartitionsExec; use crate::physical_optimizer::repartition::Repartition; +use crate::datasource::avro::AvroFile; +use crate::physical_plan::avro::AvroReadOptions; use crate::physical_plan::csv::CsvReadOptions; use crate::physical_plan::planner::DefaultPhysicalPlanner; use crate::physical_plan::udf::ScalarUDF; @@ -334,6 +336,18 @@ impl ExecutionContext { Ok(()) } + /// Registers an Avro data source so that it can be referenced from SQL statements + /// executed against this context. + pub fn register_avro( + &mut self, + name: &str, + filename: &str, + options: AvroReadOptions, + ) -> Result<()> { + self.register_table(name, Arc::new(AvroFile::try_new(filename, options)?))?; + Ok(()) + } + /// Registers a named catalog using a custom `CatalogProvider` so that /// it can be referenced from SQL statements executed against this /// context. diff --git a/datafusion/src/lib.rs b/datafusion/src/lib.rs index eac9b5f5a78a..0e95e8f6df34 100644 --- a/datafusion/src/lib.rs +++ b/datafusion/src/lib.rs @@ -212,6 +212,7 @@ extern crate sqlparser; +pub mod avro; pub mod catalog; pub mod dataframe; pub mod datasource; diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs new file mode 100644 index 000000000000..9d5235a2405d --- /dev/null +++ b/datafusion/src/physical_plan/avro.rs @@ -0,0 +1,472 @@ +// 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 async_trait::async_trait; +use futures::Stream; + +use super::{common, source::Source, ExecutionPlan, Partitioning, RecordBatchStream}; +use crate::avro; +use crate::error::{DataFusionError, Result}; +use arrow::{ + datatypes::{Schema, SchemaRef}, + error::Result as ArrowResult, + json, + record_batch::RecordBatch, +}; +use std::fs::File; +use std::{any::Any, io::Seek}; +use std::{ + io::Read, + pin::Pin, + sync::{Arc, Mutex}, + task::{Context, Poll}, +}; + +/// Line-delimited JSON read options +#[derive(Clone)] +pub struct AvroReadOptions<'a> { + /// The data source schema. + pub schema: Option, + + /// File extension; only files with this extension are selected for data input. + /// Defaults to ".avro". + pub file_extension: &'a str, +} + +impl<'a> Default for AvroReadOptions<'a> { + fn default() -> Self { + Self { + schema: None, + file_extension: ".avro", + } + } +} + +trait SeekRead: Read + Seek {} + +impl SeekRead for T {} +/// Execution plan for scanning NdJson data source +#[derive(Debug)] +pub struct AvroExec { + source: Source>, + schema: SchemaRef, + projection: Option>, + file_extension: String, + batch_size: usize, + limit: Option, +} + +impl AvroExec { + /// Create a new execution plan for reading from a path + pub fn try_new( + path: &str, + options: AvroReadOptions, + projection: Option>, + batch_size: usize, + limit: Option, + ) -> Result { + let file_extension = options.file_extension.to_string(); + + let filenames = common::build_file_list(path, &file_extension)?; + + if filenames.is_empty() { + return Err(DataFusionError::Execution(format!( + "No files found at {path} with file extension {file_extension}", + path = path, + file_extension = file_extension.as_str() + ))); + } + + let schema = match options.schema { + Some(s) => s, + None => Arc::new(AvroExec::try_infer_schema(filenames.as_slice())?), + }; + + let projected_schema = match &projection { + None => schema.clone(), + Some(p) => Arc::new(Schema::new( + p.iter().map(|i| schema.field(*i).clone()).collect(), + )), + }; + + Ok(Self { + source: Source::PartitionedFiles { + path: path.to_string(), + filenames, + }, + schema: projected_schema.clone(), + file_extension, + projection, + batch_size, + limit, + }) + } + /// Create a new execution plan for reading from a reader + pub fn try_new_from_reader( + reader: impl Read + Seek + Send + Sync + 'static, + options: AvroReadOptions, + projection: Option>, + batch_size: usize, + limit: Option, + ) -> Result { + let schema = match options.schema { + Some(s) => s, + None => { + return Err(DataFusionError::Execution( + "The schema must be provided in options when reading from a reader" + .to_string(), + )); + } + }; + + let projected_schema = match &projection { + None => schema.clone(), + Some(p) => Arc::new(Schema::new( + p.iter().map(|i| schema.field(*i).clone()).collect(), + )), + }; + + Ok(Self { + source: Source::Reader(Mutex::new(Some(Box::new(reader)))), + schema: projected_schema.clone(), + file_extension: String::new(), + projection, + batch_size, + limit, + }) + } + + /// Path to directory containing partitioned CSV files with the same schema + pub fn path(&self) -> &str { + self.source.path() + } + + /// The individual files under path + pub fn filenames(&self) -> &[String] { + self.source.filenames() + } + + /// File extension + pub fn file_extension(&self) -> &str { + &self.file_extension + } + + /// Get the schema of the CSV file + pub fn file_schema(&self) -> SchemaRef { + self.schema.clone() + } + + /// Optional projection for which columns to load + pub fn projection(&self) -> Option<&Vec> { + self.projection.as_ref() + } + + /// Batch size + pub fn batch_size(&self) -> usize { + self.batch_size + } + + /// Limit + pub fn limit(&self) -> Option { + self.limit + } + + /// Infer schema for given Avro dataset + pub fn try_infer_schema(filenames: &[String]) -> Result { + let mut schemas = Vec::new(); + for filename in filenames { + let mut file = File::open(filename)?; + let schema = Self::infer_avro_schema_from_reader(&mut file)?; + schemas.push(schema); + } + + Ok(Schema::try_merge(schemas)?) + } + + /// Infer Avro schema given a reader + pub fn infer_avro_schema_from_reader( + reader: &mut R, + ) -> Result { + let avro_reader = avro_rs::Reader::new(reader)?; + let schema = avro_reader.writer_schema(); + avro::to_arrow_schema(schema) + } +} + +#[async_trait] +impl ExecutionPlan for AvroExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(match &self.source { + Source::PartitionedFiles { filenames, .. } => filenames.len(), + Source::Reader(_) => 1, + }) + } + + fn children(&self) -> Vec> { + Vec::new() + } + + fn with_new_children( + &self, + children: Vec>, + ) -> Result> { + if !children.is_empty() { + Err(DataFusionError::Internal(format!( + "Children cannot be replaced in {:?}", + self + ))) + } else if let Source::PartitionedFiles { filenames, path } = &self.source { + Ok(Arc::new(Self { + source: Source::PartitionedFiles { + filenames: filenames.clone(), + path: path.clone(), + }, + schema: self.schema.clone(), + projection: self.projection.clone(), + batch_size: self.batch_size, + limit: self.limit, + file_extension: self.file_extension.clone(), + })) + } else { + Err(DataFusionError::Internal( + "AvroExec with reader source cannot be used with `with_new_children`" + .to_string(), + )) + } + } + + async fn execute( + &self, + partition: usize, + ) -> Result { + let mut builder = json::ReaderBuilder::new() + .with_schema(self.schema.clone()) + .with_batch_size(self.batch_size); + if let Some(proj) = &self.projection { + builder = builder.with_projection( + proj.iter() + .map(|col_idx| self.schema.field(*col_idx).name()) + .cloned() + .collect(), + ); + } + match &self.source { + Source::PartitionedFiles { filenames, .. } => { + let file = File::open(&filenames[partition])?; + + Ok(Box::pin(AvroStream::new(builder.build(file)?, self.limit))) + } + Source::Reader(rdr) => { + if partition != 0 { + Err(DataFusionError::Internal( + "Only partition 0 is valid when CSV comes from a reader" + .to_string(), + )) + } else if let Some(rdr) = rdr.lock().unwrap().take() { + Ok(Box::pin(AvroStream::new(builder.build(rdr)?, self.limit))) + } else { + Err(DataFusionError::Execution( + "Error reading CSV: Data can only be read a single time when the source is a reader" + .to_string(), + )) + } + } + } + } +} + +struct AvroStream { + reader: json::Reader, + remain: Option, +} + +impl AvroStream { + fn new(reader: json::Reader, limit: Option) -> Self { + Self { + reader, + remain: limit, + } + } +} + +impl Stream for AvroStream { + type Item = ArrowResult; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + if let Some(remain) = self.remain.as_mut() { + if *remain < 1 { + return Poll::Ready(None); + } + } + + Poll::Ready(match self.reader.next() { + Ok(Some(item)) => { + if let Some(remain) = self.remain.as_mut() { + if *remain >= item.num_rows() { + *remain -= item.num_rows(); + Some(Ok(item)) + } else { + let len = *remain; + *remain = 0; + Some(Ok(RecordBatch::try_new( + item.schema(), + item.columns() + .iter() + .map(|column| column.slice(0, len)) + .collect(), + )?)) + } + } else { + Some(Ok(item)) + } + } + Ok(None) => None, + Err(err) => Some(Err(err)), + }) + } +} + +impl RecordBatchStream for AvroStream { + fn schema(&self) -> SchemaRef { + self.reader.schema() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use futures::StreamExt; + + const TEST_DATA_BASE: &str = "tests/jsons"; + + #[tokio::test] + async fn avro_exec_file_without_projection() -> Result<()> { + use arrow::datatypes::DataType; + let path = format!("{}/1.json", TEST_DATA_BASE); + let exec = AvroExec::try_new(&path, Default::default(), None, 1024, Some(3))?; + let inferred_schema = exec.schema(); + assert_eq!(inferred_schema.fields().len(), 4); + + // a,b,c,d should be inferred + inferred_schema.field_with_name("a").unwrap(); + inferred_schema.field_with_name("b").unwrap(); + inferred_schema.field_with_name("c").unwrap(); + inferred_schema.field_with_name("d").unwrap(); + + assert_eq!( + inferred_schema.field_with_name("a").unwrap().data_type(), + &DataType::Int64 + ); + assert!(matches!( + inferred_schema.field_with_name("b").unwrap().data_type(), + DataType::List(_) + )); + assert_eq!( + inferred_schema.field_with_name("d").unwrap().data_type(), + &DataType::Utf8 + ); + + let mut it = exec.execute(0).await?; + let batch = it.next().await.unwrap()?; + + assert_eq!(batch.num_rows(), 3); + let values = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(values.value(0), 1); + assert_eq!(values.value(1), -10); + assert_eq!(values.value(2), 2); + + Ok(()) + } + + #[tokio::test] + async fn avro_exec_file_projection() -> Result<()> { + let path = format!("{}/1.json", TEST_DATA_BASE); + let exec = + AvroExec::try_new(&path, Default::default(), Some(vec![0, 2]), 1024, None)?; + let inferred_schema = exec.schema(); + assert_eq!(inferred_schema.fields().len(), 2); + + inferred_schema.field_with_name("a").unwrap(); + inferred_schema.field_with_name("b").unwrap_err(); + inferred_schema.field_with_name("c").unwrap(); + inferred_schema.field_with_name("d").unwrap_err(); + + let mut it = exec.execute(0).await?; + let batch = it.next().await.unwrap()?; + + assert_eq!(batch.num_rows(), 4); + let values = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(values.value(0), 1); + assert_eq!(values.value(1), -10); + assert_eq!(values.value(2), 2); + Ok(()) + } + + #[tokio::test] + async fn avro_exec_from_reader() -> Result<()> { + let content = r#"{"a":"aaa", "b":[2.0, 1.3, -6.1], "c":[false, true], "d":"4"} +{"a":"bbb", "b":[2.0, 1.3, -6.1], "c":[true, true], "d":"4"}"#; + let cur = std::io::Cursor::new(content); + let mut bufrdr = std::io::BufReader::new(cur); + let schema = + arrow::json::reader::infer_json_schema_from_seekable(&mut bufrdr, None)?; + let exec = AvroExec::try_new_from_reader( + bufrdr, + AvroReadOptions { + schema: Some(Arc::new(schema)), + ..Default::default() + }, + None, + 1024, + Some(1), + )?; + + let mut it = exec.execute(0).await?; + let batch = it.next().await.unwrap()?; + + assert_eq!(batch.num_rows(), 1); + + let values = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(values.value(0), "aaa"); + + Ok(()) + } +} diff --git a/datafusion/src/physical_plan/common.rs b/datafusion/src/physical_plan/common.rs index d0b7a07f3b79..3be9e7245eb7 100644 --- a/datafusion/src/physical_plan/common.rs +++ b/datafusion/src/physical_plan/common.rs @@ -107,6 +107,20 @@ pub(crate) fn combine_batches( } } +/// Recursively builds a list of files in a directory with a given extension +pub fn build_checked_file_list(dir: &str, ext: &str) -> Result> { + let mut filenames: Vec = Vec::new(); + build_file_list_recurse(dir, &mut filenames, ext)?; + if filenames.is_empty() { + return Err(DataFusionError::Plan(format!( + "No files found at {path} with file extension {file_extension}", + path = dir, + file_extension = ext + ))); + } + Ok(filenames) +} + /// Recursively builds a list of files in a directory with a given extension pub fn build_file_list(dir: &str, ext: &str) -> Result> { let mut filenames: Vec = Vec::new(); diff --git a/datafusion/src/physical_plan/mod.rs b/datafusion/src/physical_plan/mod.rs index af868871abb8..3701e908f971 100644 --- a/datafusion/src/physical_plan/mod.rs +++ b/datafusion/src/physical_plan/mod.rs @@ -603,6 +603,7 @@ pub trait Accumulator: Send + Sync + Debug { pub mod aggregates; pub mod analyze; pub mod array_expressions; +pub mod avro; pub mod coalesce_batches; pub mod coalesce_partitions; pub mod common; diff --git a/testing b/testing index b658b087767b..22d990e195f9 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit b658b087767b041b2081766814655b4dd5a9a439 +Subproject commit 22d990e195f9bfda0311c29dfccf9f1fa5cecae9 From 0f940ab037e6cb0f861064d955b68d3d07d9c21c Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 18 Aug 2021 01:17:35 +0200 Subject: [PATCH 02/58] wip --- datafusion-examples/examples/avro_sql.rs | 2 +- datafusion/src/avro/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion-examples/examples/avro_sql.rs b/datafusion-examples/examples/avro_sql.rs index 50714ec45a64..042d2b4ca218 100644 --- a/datafusion-examples/examples/avro_sql.rs +++ b/datafusion-examples/examples/avro_sql.rs @@ -43,7 +43,7 @@ async fn main() -> Result<()> { let df = ctx.sql( "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ FROM alltypes_plain \ - WHERE id > 1 AND tinyint_col < double_col", + WHERE id.int > 1 AND tinyint_col < double_col", )?; let results = df.collect().await?; diff --git a/datafusion/src/avro/mod.rs b/datafusion/src/avro/mod.rs index 27e8bc63f20c..67a6e4bbaa66 100644 --- a/datafusion/src/avro/mod.rs +++ b/datafusion/src/avro/mod.rs @@ -25,7 +25,6 @@ use avro_rs::types::Value; use avro_rs::Schema as AvroSchema; use std::collections::BTreeMap; use std::convert::TryFrom; -use std::ptr::null; /// Converts an avro schema to an arrow schema pub fn to_arrow_schema(avro_schema: &avro_rs::Schema) -> Result { @@ -267,6 +266,7 @@ fn external_props(schema: &AvroSchema) -> BTreeMap { props } +#[allow(dead_code)] fn get_metadata( _schema: AvroSchema, props: BTreeMap, From 2ca7ccbe25a81023cbf38fca13e41f7738e6c716 Mon Sep 17 00:00:00 2001 From: "Jorge C. Leitao" Date: Sat, 26 Jun 2021 19:44:45 +0000 Subject: [PATCH 03/58] Added support composite identifiers for struct type. --- datafusion/src/lib.rs | 1 + datafusion/src/logical_plan/expr.rs | 40 +++++++ datafusion/src/optimizer/utils.rs | 6 ++ .../physical_plan/expressions/get_field.rs | 100 ++++++++++++++++++ .../src/physical_plan/expressions/mod.rs | 2 + datafusion/src/physical_plan/planner.rs | 8 ++ datafusion/src/sql/planner.rs | 35 +++--- datafusion/src/sql/utils.rs | 4 + datafusion/src/utils.rs | 43 ++++++++ datafusion/tests/sql.rs | 33 ++++-- 10 files changed, 248 insertions(+), 24 deletions(-) create mode 100644 datafusion/src/physical_plan/expressions/get_field.rs create mode 100644 datafusion/src/utils.rs diff --git a/datafusion/src/lib.rs b/datafusion/src/lib.rs index 0e95e8f6df34..42895564ecd6 100644 --- a/datafusion/src/lib.rs +++ b/datafusion/src/lib.rs @@ -225,6 +225,7 @@ pub mod physical_plan; pub mod prelude; pub mod scalar; pub mod sql; +mod utils; pub mod variable; // re-export dependencies from arrow-rs to minimise version maintenance for crate users diff --git a/datafusion/src/logical_plan/expr.rs b/datafusion/src/logical_plan/expr.rs index ec017d0765f9..5e2feb33be0f 100644 --- a/datafusion/src/logical_plan/expr.rs +++ b/datafusion/src/logical_plan/expr.rs @@ -25,6 +25,7 @@ use crate::physical_plan::{ aggregates, expressions::binary_operator_data_type, functions, udf::ScalarUDF, window_functions, }; +use crate::utils::get_field; use crate::{physical_plan::udaf::AggregateUDF, scalar::ScalarValue}; use aggregates::{AccumulatorFunctionImplementation, StateTypeFunction}; use arrow::{compute::can_cast_types, datatypes::DataType}; @@ -244,6 +245,13 @@ pub enum Expr { IsNull(Box), /// arithmetic negation of an expression, the operand must be of a signed numeric data type Negative(Box), + /// Returns the field of a [`StructArray`] by name + GetField { + /// the expression to take the field from + expr: Box, + /// The name of the field to take + name: String, + }, /// Whether an expression is between a given range. Between { /// The value to compare @@ -432,6 +440,10 @@ impl Expr { Expr::Wildcard => Err(DataFusionError::Internal( "Wildcard expressions are not valid in a logical query plan".to_owned(), )), + Expr::GetField { ref expr, name } => { + let data_type = expr.get_type(schema)?; + get_field(&data_type, name).map(|x| x.data_type().clone()) + } } } @@ -487,6 +499,10 @@ impl Expr { Expr::Wildcard => Err(DataFusionError::Internal( "Wildcard expressions are not valid in a logical query plan".to_owned(), )), + Expr::GetField { ref expr, name } => { + let data_type = expr.get_type(input_schema)?; + get_field(&data_type, name).map(|x| x.is_nullable()) + } } } @@ -627,6 +643,14 @@ impl Expr { Expr::IsNotNull(Box::new(self)) } + /// Returns the values of the field `name` from an expression returning a `Struct` + pub fn get_field>(self, name: I) -> Expr { + Expr::GetField { + expr: Box::new(self), + name: name.into(), + } + } + /// Create a sort expression from an existing expression. /// /// ``` @@ -762,6 +786,7 @@ impl Expr { .try_fold(visitor, |visitor, arg| arg.accept(visitor)) } Expr::Wildcard => Ok(visitor), + Expr::GetField { ref expr, .. } => expr.accept(visitor), }?; visitor.post_visit(self) @@ -919,6 +944,10 @@ impl Expr { negated, }, Expr::Wildcard => Expr::Wildcard, + Expr::GetField { expr, name } => Expr::GetField { + expr: rewrite_boxed(expr, rewriter)?, + name, + }, }; // now rewrite this expression itself @@ -1680,6 +1709,7 @@ impl fmt::Debug for Expr { } } Expr::Wildcard => write!(f, "*"), + Expr::GetField { ref expr, name } => write!(f, "({:?}).{}", expr, name), } } } @@ -1756,6 +1786,10 @@ fn create_name(e: &Expr, input_schema: &DFSchema) -> Result { let expr = create_name(expr, input_schema)?; Ok(format!("{} IS NOT NULL", expr)) } + Expr::GetField { expr, name } => { + let expr = create_name(expr, input_schema)?; + Ok(format!("{}.{}", expr, name)) + } Expr::ScalarFunction { fun, args, .. } => { create_function_name(&fun.to_string(), false, args, input_schema) } @@ -1877,6 +1911,12 @@ mod tests { ); } + #[test] + fn display_get_field() { + let col_null = col("col1").get_field("name"); + assert_eq!(format!("{:?}", col_null), "(#col1).name"); + } + #[derive(Default)] struct RecordingRewriter { v: Vec, diff --git a/datafusion/src/optimizer/utils.rs b/datafusion/src/optimizer/utils.rs index 435daefb88eb..f7f4cb640138 100644 --- a/datafusion/src/optimizer/utils.rs +++ b/datafusion/src/optimizer/utils.rs @@ -79,6 +79,7 @@ impl ExpressionVisitor for ColumnNameVisitor<'_> { Expr::AggregateUDF { .. } => {} Expr::InList { .. } => {} Expr::Wildcard => {} + Expr::GetField { .. } => {} } Ok(Recursion::Continue(self)) } @@ -305,6 +306,7 @@ pub fn expr_sub_expressions(expr: &Expr) -> Result> { Expr::Wildcard { .. } => Err(DataFusionError::Internal( "Wildcard expressions are not valid in a logical query plan".to_owned(), )), + Expr::GetField { expr, .. } => Ok(vec![expr.as_ref().to_owned()]), } } @@ -320,6 +322,10 @@ pub fn rewrite_expression(expr: &Expr, expressions: &[Expr]) -> Result { }), Expr::IsNull(_) => Ok(Expr::IsNull(Box::new(expressions[0].clone()))), Expr::IsNotNull(_) => Ok(Expr::IsNotNull(Box::new(expressions[0].clone()))), + Expr::GetField { expr: _, name } => Ok(Expr::GetField { + expr: Box::new(expressions[0].clone()), + name: name.clone(), + }), Expr::ScalarFunction { fun, .. } => Ok(Expr::ScalarFunction { fun: fun.clone(), args: expressions.to_vec(), diff --git a/datafusion/src/physical_plan/expressions/get_field.rs b/datafusion/src/physical_plan/expressions/get_field.rs new file mode 100644 index 000000000000..da56a29f0777 --- /dev/null +++ b/datafusion/src/physical_plan/expressions/get_field.rs @@ -0,0 +1,100 @@ +// 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. + +//! get field of a struct array + +use std::{any::Any, sync::Arc}; + +use arrow::{ + array::StructArray, + datatypes::{DataType, Schema}, + record_batch::RecordBatch, +}; + +use crate::{ + error::DataFusionError, + error::Result, + physical_plan::{ColumnarValue, PhysicalExpr}, + utils::get_field as get_data_type_field, +}; + +/// expression to get a field of a struct array. +#[derive(Debug)] +pub struct GetFieldExpr { + arg: Arc, + name: String, +} + +impl GetFieldExpr { + /// Create new get field expression + pub fn new(arg: Arc, name: String) -> Self { + Self { arg, name } + } + + /// Get the input expression + pub fn arg(&self) -> &Arc { + &self.arg + } +} + +impl std::fmt::Display for GetFieldExpr { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "({}).{}", self.arg, self.name) + } +} + +impl PhysicalExpr for GetFieldExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> Result { + let data_type = self.arg.data_type(input_schema)?; + get_data_type_field(&data_type, &self.name).map(|f| f.data_type().clone()) + } + + fn nullable(&self, input_schema: &Schema) -> Result { + let data_type = self.arg.data_type(input_schema)?; + get_data_type_field(&data_type, &self.name).map(|f| f.is_nullable()) + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + let arg = self.arg.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => Ok(ColumnarValue::Array( + array + .as_any() + .downcast_ref::() + .unwrap() + .column_by_name(&self.name) + .unwrap() + .clone(), + )), + ColumnarValue::Scalar(_) => Err(DataFusionError::NotImplemented( + "field is not yet implemented for scalar values".to_string(), + )), + } + } +} + +/// Create an `.field` expression +pub fn get_field( + arg: Arc, + name: String, +) -> Result> { + Ok(Arc::new(GetFieldExpr::new(arg, name))) +} diff --git a/datafusion/src/physical_plan/expressions/mod.rs b/datafusion/src/physical_plan/expressions/mod.rs index 5a5a1189af05..d599c332d9d9 100644 --- a/datafusion/src/physical_plan/expressions/mod.rs +++ b/datafusion/src/physical_plan/expressions/mod.rs @@ -33,6 +33,7 @@ mod cast; mod coercion; mod column; mod count; +mod get_field; mod in_list; mod is_not_null; mod is_null; @@ -62,6 +63,7 @@ pub use cast::{ }; pub use column::{col, Column}; pub use count::Count; +pub use get_field::{get_field, GetFieldExpr}; pub use in_list::{in_list, InListExpr}; pub use is_not_null::{is_not_null, IsNotNullExpr}; pub use is_null::{is_null, IsNullExpr}; diff --git a/datafusion/src/physical_plan/planner.rs b/datafusion/src/physical_plan/planner.rs index 0ff595817e7c..0b994bb1d3c9 100644 --- a/datafusion/src/physical_plan/planner.rs +++ b/datafusion/src/physical_plan/planner.rs @@ -138,6 +138,10 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { let expr = create_physical_name(expr, false)?; Ok(format!("{} IS NOT NULL", expr)) } + Expr::GetField { expr, name } => { + let expr = physical_name(expr, input_schema)?; + Ok(format!("{}.{}", expr, name)) + } Expr::ScalarFunction { fun, args, .. } => { create_function_physical_name(&fun.to_string(), false, args) } @@ -960,6 +964,10 @@ impl DefaultPhysicalPlanner { Expr::IsNotNull(expr) => expressions::is_not_null( self.create_physical_expr(expr, input_dfschema, input_schema, ctx_state)?, ), + Expr::GetField { expr, name } => expressions::get_field( + self.create_physical_expr(expr, input_dfschema, input_schema, ctx_state)?, + name.clone(), + ), Expr::ScalarFunction { fun, args } => { let physical_args = args .iter() diff --git a/datafusion/src/sql/planner.rs b/datafusion/src/sql/planner.rs index e613ff385b39..233fd5af25ef 100644 --- a/datafusion/src/sql/planner.rs +++ b/datafusion/src/sql/planner.rs @@ -81,6 +81,22 @@ pub struct SqlToRel<'a, S: ContextProvider> { schema_provider: &'a S, } +fn plan_compound(mut identifiers: Vec) -> Expr { + if &identifiers[0][0..1] == "@" { + Expr::ScalarVariable(identifiers) + } else if identifiers.len() == 2 { + // "table.column" + let name = identifiers.pop().unwrap(); + let relation = Some(identifiers.pop().unwrap()); + Expr::Column(Column { relation, name }) + } else { + // "table.column.field..." + let name = identifiers.pop().unwrap(); + let expr = Box::new(plan_compound(identifiers)); + Expr::GetField { expr, name } + } +} + impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// Create a new query planner pub fn new(schema_provider: &'a S) -> Self { @@ -1093,23 +1109,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } SQLExpr::CompoundIdentifier(ids) => { - let mut var_names = vec![]; - for id in ids { - var_names.push(id.value.clone()); - } - if &var_names[0][0..1] == "@" { - Ok(Expr::ScalarVariable(var_names)) - } else if var_names.len() == 2 { - // table.column identifier - let name = var_names.pop().unwrap(); - let relation = Some(var_names.pop().unwrap()); - Ok(Expr::Column(Column { relation, name })) - } else { - Err(DataFusionError::NotImplemented(format!( - "Unsupported compound identifier '{:?}'", - var_names, - ))) - } + let var_names = ids.iter().map(|x| x.value.clone()).collect::>(); + Ok(plan_compound(var_names)) } SQLExpr::Wildcard => Ok(Expr::Wildcard), diff --git a/datafusion/src/sql/utils.rs b/datafusion/src/sql/utils.rs index 41bcd205800d..cbd173f312ed 100644 --- a/datafusion/src/sql/utils.rs +++ b/datafusion/src/sql/utils.rs @@ -368,6 +368,10 @@ where Ok(expr.clone()) } Expr::Wildcard => Ok(Expr::Wildcard), + Expr::GetField { expr, name } => Ok(Expr::GetField { + expr: Box::new(clone_with_replacement(expr.as_ref(), replacement_fn)?), + name: name.clone(), + }), }, } } diff --git a/datafusion/src/utils.rs b/datafusion/src/utils.rs new file mode 100644 index 000000000000..587cb18e5856 --- /dev/null +++ b/datafusion/src/utils.rs @@ -0,0 +1,43 @@ +// 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. + +use arrow::datatypes::{DataType, Field}; + +use crate::error::{DataFusionError, Result}; + +/// Returns the first field named `name` from the fields of a [`DataType::Struct`]. +/// # Error +/// Errors iff +/// * the `data_type` is not a Struct or, +/// * there is no field named `name` +pub fn get_field<'a>(data_type: &'a DataType, name: &str) -> Result<&'a Field> { + if let DataType::Struct(fields) = data_type { + let maybe_field = fields.iter().find(|x| x.name() == name); + if let Some(field) = maybe_field { + Ok(field) + } else { + Err(DataFusionError::Plan(format!( + "The `Struct` has no field named \"{}\"", + name + ))) + } + } else { + Err(DataFusionError::Plan( + "The expression to get a field is only valid for `Struct`".to_string(), + )) + } +} diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index eaf988fba0cf..3f22ad1d0390 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -27,14 +27,8 @@ use chrono::Duration; extern crate arrow; extern crate datafusion; -use arrow::{array::*, datatypes::TimeUnit}; -use arrow::{datatypes::Int32Type, datatypes::Int64Type, record_batch::RecordBatch}; use arrow::{ - datatypes::{ - ArrowNativeType, ArrowPrimitiveType, ArrowTimestampType, DataType, Field, Schema, - SchemaRef, TimestampMicrosecondType, TimestampMillisecondType, - TimestampNanosecondType, TimestampSecondType, - }, + array::*, datatypes::*, record_batch::RecordBatch, util::display::array_value_to_string, }; @@ -3346,6 +3340,31 @@ async fn query_is_not_null() -> Result<()> { Ok(()) } +#[tokio::test] +async fn query_get_field() -> Result<()> { + let inner_field = Field::new("inner", DataType::Float64, true); + let field = Field::new("c1", DataType::Struct(vec![inner_field.clone()]), true); + let schema = Arc::new(Schema::new(vec![field])); + + let array = Arc::new(Float64Array::from(vec![Some(1.1), None])) as ArrayRef; + + let data = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(StructArray::from(vec![(inner_field, array)]))], + )?; + + let table = MemTable::try_new(schema, vec![vec![data]])?; + + let mut ctx = ExecutionContext::new(); + ctx.register_table("test", Arc::new(table))?; + let sql = "SELECT test.c1.inner FROM test"; + let actual = execute(&mut ctx, sql).await; + let expected = vec![vec!["1.1"], vec!["NULL"]]; + + assert_eq!(expected, actual); + Ok(()) +} + #[tokio::test] async fn query_count_distinct() -> Result<()> { let schema = Arc::new(Schema::new(vec![Field::new("c1", DataType::Int32, true)])); From fcbf43bf667542eccd6666bcb1c6986599e0b0f6 Mon Sep 17 00:00:00 2001 From: "Jorge C. Leitao" Date: Sun, 27 Jun 2021 04:09:03 +0000 Subject: [PATCH 04/58] Fixed build. --- ballista/rust/core/src/serde/logical_plan/to_proto.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index 10bc63e4807b..953d59b32353 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -1284,7 +1284,7 @@ impl TryInto for &Expr { Expr::Wildcard => Ok(protobuf::LogicalExprNode { expr_type: Some(protobuf::logical_expr_node::ExprType::Wildcard(true)), }), - Expr::TryCast { .. } => unimplemented!(), + _ => unimplemented!(), } } } From bae2f0054f9aed5404d0b5b1abc60db6d0a9d2b5 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 18 Aug 2021 01:40:20 +0200 Subject: [PATCH 05/58] cheat and add unions to valid composite column types --- datafusion/src/utils.rs | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/datafusion/src/utils.rs b/datafusion/src/utils.rs index 587cb18e5856..0d51e95091b9 100644 --- a/datafusion/src/utils.rs +++ b/datafusion/src/utils.rs @@ -25,19 +25,21 @@ use crate::error::{DataFusionError, Result}; /// * the `data_type` is not a Struct or, /// * there is no field named `name` pub fn get_field<'a>(data_type: &'a DataType, name: &str) -> Result<&'a Field> { - if let DataType::Struct(fields) = data_type { - let maybe_field = fields.iter().find(|x| x.name() == name); - if let Some(field) = maybe_field { - Ok(field) - } else { - Err(DataFusionError::Plan(format!( - "The `Struct` has no field named \"{}\"", - name - ))) + match data_type { + DataType::Struct(fields) | DataType::Union(fields) => { + let maybe_field = fields.iter().find(|x| x.name() == name); + if let Some(field) = maybe_field { + Ok(field) + } else { + Err(DataFusionError::Plan(format!( + "The `Struct` has no field named \"{}\"", + name + ))) + } } - } else { - Err(DataFusionError::Plan( - "The expression to get a field is only valid for `Struct`".to_string(), - )) + _ => Err(DataFusionError::Plan( + "The expression to get a field is only valid for `Struct` or 'Union'" + .to_string(), + )), } } From a352dad85e07f6c8adb24dc369e9b3e70be420c8 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 18 Aug 2021 10:46:42 +0200 Subject: [PATCH 06/58] Implement the AvroArrayReader --- datafusion-examples/examples/avro_sql.rs | 3 +- datafusion/Cargo.toml | 1 + datafusion/src/avro/arrow_array_reader.rs | 898 ++++++++++++++++++++++ datafusion/src/avro/mod.rs | 38 +- datafusion/src/avro/reader.rs | 160 ++++ datafusion/src/datasource/avro.rs | 257 ++++++- datafusion/src/logical_plan/dfschema.rs | 9 +- datafusion/src/physical_plan/avro.rs | 345 ++++++--- 8 files changed, 1583 insertions(+), 128 deletions(-) create mode 100644 datafusion/src/avro/arrow_array_reader.rs create mode 100644 datafusion/src/avro/reader.rs diff --git a/datafusion-examples/examples/avro_sql.rs b/datafusion-examples/examples/avro_sql.rs index 042d2b4ca218..793228224345 100644 --- a/datafusion-examples/examples/avro_sql.rs +++ b/datafusion-examples/examples/avro_sql.rs @@ -38,12 +38,11 @@ async fn main() -> Result<()> { )?; let df = ctx.table("alltypes_plain").unwrap(); - println!("schema {:?}", df.schema()); // execute the query let df = ctx.sql( "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ FROM alltypes_plain \ - WHERE id.int > 1 AND tinyint_col < double_col", + WHERE id > 1 AND tinyint_col < double_col", )?; let results = df.collect().await?; diff --git a/datafusion/Cargo.toml b/datafusion/Cargo.toml index 061a72f5030f..ca89185fc8be 100644 --- a/datafusion/Cargo.toml +++ b/datafusion/Cargo.toml @@ -70,6 +70,7 @@ lazy_static = { version = "^1.4.0", optional = true } smallvec = { version = "1.6", features = ["union"] } rand = "0.8" avro-rs = "0.13" +num-traits = "0.2" [dev-dependencies] criterion = "0.3" diff --git a/datafusion/src/avro/arrow_array_reader.rs b/datafusion/src/avro/arrow_array_reader.rs new file mode 100644 index 000000000000..f43daa2fa645 --- /dev/null +++ b/datafusion/src/avro/arrow_array_reader.rs @@ -0,0 +1,898 @@ +use crate::arrow::array::{ + make_array, Array, ArrayBuilder, ArrayData, ArrayDataBuilder, ArrayRef, + BooleanBuilder, LargeStringArray, ListBuilder, NullArray, OffsetSizeTrait, + PrimitiveArray, PrimitiveBuilder, StringArray, StringBuilder, + StringDictionaryBuilder, +}; +use crate::arrow::buffer::{Buffer, MutableBuffer}; +use crate::arrow::datatypes::{ + ArrowDictionaryKeyType, ArrowNumericType, ArrowPrimitiveType, DataType, Date32Type, + Date64Type, Field, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, + Int8Type, Schema, Time32MillisecondType, Time32SecondType, Time64MicrosecondType, + Time64NanosecondType, TimeUnit, TimestampMicrosecondType, TimestampMillisecondType, + 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 arrow::array::GenericListArray; +use arrow::datatypes::SchemaRef; +use arrow::error::ArrowError::SchemaError; +use arrow::error::Result as ArrowResult; +use avro_rs::schema::Schema as AvroSchema; +use avro_rs::types::Value; +use avro_rs::Reader as AvroReader; +use std::collections::HashMap; +use std::io::Read; +use std::sync::Arc; + +type RecordSlice<'a> = &'a [Vec<(String, Value)>]; + +pub struct AvroArrowArrayReader<'a, R: Read> { + reader: AvroReader<'a, R>, + schema: SchemaRef, + projection: Option>, + schema_lookup: HashMap, +} + +impl<'a, R: Read> AvroArrowArrayReader<'a, R> { + pub fn try_new( + reader: AvroReader<'a, R>, + schema: SchemaRef, + projection: Option>, + ) -> Result { + let writer_schema = reader.writer_schema().clone(); + let schema_lookup = Self::schema_lookup(writer_schema)?; + Ok(Self { + reader, + schema, + projection, + schema_lookup, + }) + } + + pub fn schema_lookup(schema: AvroSchema) -> Result> { + match schema { + AvroSchema::Record { + lookup: ref schema_lookup, + .. + } => Ok(schema_lookup.clone()), + _ => Err(DataFusionError::ArrowError(SchemaError( + "expected avro schema to be a record".to_string(), + ))), + } + } + + /// Read the next batch of records + #[allow(clippy::should_implement_trait)] + pub fn next_batch(&mut self, batch_size: usize) -> ArrowResult> { + let mut rows = Vec::with_capacity(batch_size); + for value in self.reader.by_ref().take(batch_size) { + let v = value.map_err(|e| { + ArrowError::ParseError(format!("Failed to parse avro value: {:?}", e)) + })?; + match v { + Value::Record(v) => { + rows.push(v); + } + other => { + return Err(ArrowError::ParseError(format!( + "Row needs to be of type object, got: {:?}", + other + ))) + } + } + } + /*if rows.is_empty() { + // reached end of file + return Ok(None); + }*/ + let rows = &rows[..]; + let projection = self.projection.clone().unwrap_or_else(Vec::new); + let arrays = self.build_struct_array(rows, self.schema.fields(), &projection); + let projected_fields: Vec = if projection.is_empty() { + self.schema.fields().to_vec() + } else { + projection + .iter() + .map(|name| self.schema.column_with_name(name)) + .flatten() + .map(|(_, field)| field.clone()) + .collect() + }; + let projected_schema = Arc::new(Schema::new(projected_fields)); + arrays.and_then(|arr| { + RecordBatch::try_new(projected_schema, arr) + .map(Some) + .map_err(|e| e.into()) + }) + } + + fn build_boolean_array( + &self, + rows: RecordSlice, + col_name: &str, + ) -> ArrowResult { + let mut builder = BooleanBuilder::new(rows.len()); + for row in rows { + if let Some(value) = self.field_lookup(col_name, row) { + if let Value::Boolean(boolean) = value { + builder.append_value(boolean)? + } else { + builder.append_null()?; + } + } else { + builder.append_null()?; + } + } + Ok(Arc::new(builder.finish())) + } + + #[allow(clippy::unnecessary_wraps)] + fn build_primitive_array( + &self, + rows: RecordSlice, + col_name: &str, + ) -> ArrowResult + where + T: ArrowNumericType, + T::Native: num_traits::cast::NumCast, + { + Ok(Arc::new( + rows.iter() + .map(|row| { + self.field_lookup(col_name, row) + .and_then(|value| value.resolve(&AvroSchema::Double).ok()) + .and_then(|v| match v { + Value::Double(f) => Some(f), + _ => None, + }) + .and_then(num_traits::cast::cast) + }) + .collect::>(), + )) + } + + #[inline(always)] + #[allow(clippy::unnecessary_wraps)] + fn build_string_dictionary_builder( + &self, + row_len: usize, + ) -> ArrowResult> + where + T: ArrowPrimitiveType + ArrowDictionaryKeyType, + { + let key_builder = PrimitiveBuilder::::new(row_len); + let values_builder = StringBuilder::new(row_len * 5); + Ok(StringDictionaryBuilder::new(key_builder, values_builder)) + } + + fn build_wrapped_list_array( + &self, + rows: RecordSlice, + col_name: &str, + key_type: &DataType, + ) -> ArrowResult { + match *key_type { + DataType::Int8 => { + let dtype = DataType::Dictionary( + Box::new(DataType::Int8), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + DataType::Int16 => { + let dtype = DataType::Dictionary( + Box::new(DataType::Int16), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + DataType::Int32 => { + let dtype = DataType::Dictionary( + Box::new(DataType::Int32), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + DataType::Int64 => { + let dtype = DataType::Dictionary( + Box::new(DataType::Int64), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + DataType::UInt8 => { + let dtype = DataType::Dictionary( + Box::new(DataType::UInt8), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + DataType::UInt16 => { + let dtype = DataType::Dictionary( + Box::new(DataType::UInt16), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + DataType::UInt32 => { + let dtype = DataType::Dictionary( + Box::new(DataType::UInt32), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + DataType::UInt64 => { + let dtype = DataType::Dictionary( + Box::new(DataType::UInt64), + Box::new(DataType::Utf8), + ); + self.list_array_string_array_builder::(&dtype, col_name, rows) + } + ref e => Err(SchemaError(format!( + "Data type is currently not supported for dictionaries in list : {:?}", + e + ))), + } + } + + #[inline(always)] + fn list_array_string_array_builder( + &self, + data_type: &DataType, + col_name: &str, + rows: RecordSlice, + ) -> ArrowResult + where + DICT_TY: ArrowPrimitiveType + ArrowDictionaryKeyType, + { + let mut builder: Box = match data_type { + DataType::Utf8 => { + let values_builder = StringBuilder::new(rows.len() * 5); + Box::new(ListBuilder::new(values_builder)) + } + DataType::Dictionary(_, _) => { + let values_builder = + self.build_string_dictionary_builder::(rows.len() * 5)?; + Box::new(ListBuilder::new(values_builder)) + } + e => { + return Err(SchemaError(format!( + "Nested list data builder type is not supported: {:?}", + e + ))) + } + }; + + for row in rows { + if let Some(value) = self.field_lookup(col_name, row) { + // value can be an array or a scalar + let vals: Vec> = if let Value::String(v) = value { + vec![Some(v.to_string())] + } else if let Value::Array(n) = value { + n.into_iter() + .map(|v| { + if let Some(v) = self.as_string(v.clone()) { + Some(v) + } else if matches!( + v, + Value::Array(_) | Value::Record(_) | Value::Null + ) { + // implicitly drop nested values + // TODO support deep-nesting + None + } else { + None + } + }) + .collect() + } else if let Value::Null = value { + vec![None] + } else if !matches!(value, Value::Record(_)) { + vec![self.as_string(value)] + } else { + return Err(SchemaError(format!( + "Only scalars are currently supported in Avro arrays", + ))); + }; + + // TODO: ARROW-10335: APIs of dictionary arrays and others are different. Unify + // them. + match data_type { + DataType::Utf8 => { + let builder = builder + .as_any_mut() + .downcast_mut::>() + .ok_or_else(||ArrowError::JsonError( + "Cast failed for ListBuilder during nested data parsing".to_string(), + ))?; + for val in vals { + if let Some(v) = val { + builder.values().append_value(&v)? + } else { + builder.values().append_null()? + }; + } + + // Append to the list + builder.append(true)?; + } + DataType::Dictionary(_, _) => { + let builder = builder.as_any_mut().downcast_mut::>>().ok_or_else(||ArrowError::JsonError( + "Cast failed for ListBuilder during nested data parsing".to_string(), + ))?; + for val in vals { + if let Some(v) = val { + let _ = builder.values().append(&v)?; + } else { + builder.values().append_null()? + }; + } + + // Append to the list + builder.append(true)?; + } + e => { + return Err(SchemaError(format!( + "Nested list data builder type is not supported: {:?}", + e + ))) + } + } + } + } + + Ok(builder.finish() as ArrayRef) + } + + #[inline(always)] + fn build_dictionary_array( + &self, + rows: RecordSlice, + col_name: &str, + ) -> ArrowResult + where + T::Native: num_traits::cast::NumCast, + T: ArrowPrimitiveType + ArrowDictionaryKeyType, + { + let mut builder: StringDictionaryBuilder = + self.build_string_dictionary_builder(rows.len())?; + for row in rows { + if let Some(value) = self.field_lookup(col_name, row) { + if let Some(str_v) = self.as_string(value) { + builder.append(str_v).map(drop)? + } else { + builder.append_null()? + } + } else { + builder.append_null()? + } + } + Ok(Arc::new(builder.finish()) as ArrayRef) + } + + #[inline(always)] + fn build_string_dictionary_array( + &self, + rows: RecordSlice, + col_name: &str, + key_type: &DataType, + value_type: &DataType, + ) -> ArrowResult { + if let DataType::Utf8 = *value_type { + match *key_type { + DataType::Int8 => self.build_dictionary_array::(rows, col_name), + DataType::Int16 => { + self.build_dictionary_array::(rows, col_name) + } + DataType::Int32 => { + self.build_dictionary_array::(rows, col_name) + } + DataType::Int64 => { + self.build_dictionary_array::(rows, col_name) + } + DataType::UInt8 => { + self.build_dictionary_array::(rows, col_name) + } + DataType::UInt16 => { + self.build_dictionary_array::(rows, col_name) + } + DataType::UInt32 => { + self.build_dictionary_array::(rows, col_name) + } + DataType::UInt64 => { + self.build_dictionary_array::(rows, col_name) + } + _ => Err(ArrowError::SchemaError( + "unsupported dictionary key type".to_string(), + )), + } + } else { + Err(ArrowError::SchemaError( + "dictionary types other than UTF-8 not yet supported".to_string(), + )) + } + } + + /// Build a nested GenericListArray from a list of unnested `Value`s + fn build_nested_list_array( + &self, + rows: &[Value], + list_field: &Field, + ) -> ArrowResult { + // build list offsets + let mut cur_offset = OffsetSize::zero(); + let list_len = rows.len(); + let num_list_bytes = bit_util::ceil(list_len, 8); + let mut offsets = Vec::with_capacity(list_len + 1); + let mut list_nulls = MutableBuffer::from_len_zeroed(num_list_bytes); + let list_nulls = list_nulls.as_slice_mut(); + offsets.push(cur_offset); + rows.iter().enumerate().for_each(|(i, v)| { + if let Value::Array(a) = v { + cur_offset += OffsetSize::from_usize(a.len()).unwrap(); + bit_util::set_bit(list_nulls, i); + } else if let Value::Null = v { + // value is null, not incremented + } else { + cur_offset += OffsetSize::one(); + } + offsets.push(cur_offset); + }); + let valid_len = cur_offset.to_usize().unwrap(); + let array_data = match list_field.data_type() { + DataType::Null => NullArray::new(valid_len).data().clone(), + DataType::Boolean => { + let num_bytes = bit_util::ceil(valid_len, 8); + let mut bool_values = MutableBuffer::from_len_zeroed(num_bytes); + let mut bool_nulls = + MutableBuffer::new(num_bytes).with_bitset(num_bytes, true); + let mut curr_index = 0; + rows.iter().for_each(|v| { + if let Value::Array(vs) = v { + vs.iter().for_each(|value| { + if let Value::Boolean(child) = value { + // if valid boolean, append value + if *child { + bit_util::set_bit( + bool_values.as_slice_mut(), + curr_index, + ); + } + } else { + // null slot + bit_util::unset_bit( + bool_nulls.as_slice_mut(), + curr_index, + ); + } + curr_index += 1; + }); + } + }); + ArrayData::builder(list_field.data_type().clone()) + .len(valid_len) + .add_buffer(bool_values.into()) + .null_bit_buffer(bool_nulls.into()) + .build() + } + DataType::Int8 => self.read_primitive_list_values::(rows), + DataType::Int16 => self.read_primitive_list_values::(rows), + DataType::Int32 => self.read_primitive_list_values::(rows), + DataType::Int64 => self.read_primitive_list_values::(rows), + DataType::UInt8 => self.read_primitive_list_values::(rows), + DataType::UInt16 => self.read_primitive_list_values::(rows), + DataType::UInt32 => self.read_primitive_list_values::(rows), + DataType::UInt64 => self.read_primitive_list_values::(rows), + DataType::Float16 => { + return Err(ArrowError::JsonError("Float16 not supported".to_string())) + } + DataType::Float32 => self.read_primitive_list_values::(rows), + DataType::Float64 => self.read_primitive_list_values::(rows), + DataType::Timestamp(_, _) + | DataType::Date32 + | DataType::Date64 + | DataType::Time32(_) + | DataType::Time64(_) => { + return Err(ArrowError::JsonError( + "Temporal types are not yet supported, see ARROW-4803".to_string(), + )) + } + DataType::Utf8 => flatten_string_values(rows) + .into_iter() + .collect::() + .data() + .clone(), + DataType::LargeUtf8 => flatten_string_values(rows) + .into_iter() + .collect::() + .data() + .clone(), + DataType::List(field) => { + let child = + self.build_nested_list_array::(&flatten_values(rows), field)?; + child.data().clone() + } + DataType::LargeList(field) => { + let child = + self.build_nested_list_array::(&flatten_values(rows), field)?; + child.data().clone() + } + DataType::Struct(fields) => { + // extract list values, with non-lists converted to Value::Null + let array_item_count = rows + .iter() + .map(|row| match row { + Value::Array(values) => values.len(), + _ => 1, + }) + .sum(); + let num_bytes = bit_util::ceil(array_item_count, 8); + let mut null_buffer = MutableBuffer::from_len_zeroed(num_bytes); + let mut struct_index = 0; + let rows: Vec> = rows + .iter() + .map(|row| { + if let Value::Array(values) = row { + values.iter().for_each(|_| { + bit_util::set_bit( + null_buffer.as_slice_mut(), + struct_index, + ); + struct_index += 1; + }); + values + .iter() + .map(|v| ("".to_string(), v.clone())) + .collect::>() + } else { + struct_index += 1; + vec![("null".to_string(), Value::Null)] + } + }) + .collect(); + let arrays = + self.build_struct_array(rows.as_slice(), fields.as_slice(), &[])?; + let data_type = DataType::Struct(fields.clone()); + let buf = null_buffer.into(); + ArrayDataBuilder::new(data_type) + .len(rows.len()) + .null_bit_buffer(buf) + .child_data(arrays.into_iter().map(|a| a.data().clone()).collect()) + .build() + } + datatype => { + return Err(ArrowError::JsonError(format!( + "Nested list of {:?} not supported", + datatype + ))); + } + }; + // build list + let list_data = ArrayData::builder(DataType::List(Box::new(list_field.clone()))) + .len(list_len) + .add_buffer(Buffer::from_slice_ref(&offsets)) + .add_child_data(array_data) + .null_bit_buffer(list_nulls.into()) + .build(); + Ok(Arc::new(GenericListArray::::from(list_data))) + } + + /// Builds the child values of a `StructArray`, falling short of constructing the StructArray. + /// The function does not construct the StructArray as some callers would want the child arrays. + /// + /// *Note*: The function is recursive, and will read nested structs. + /// + /// If `projection` is not empty, then all values are returned. The first level of projection + /// occurs at the `RecordBatch` level. No further projection currently occurs, but would be + /// useful if plucking values from a struct, e.g. getting `a.b.c.e` from `a.b.c.{d, e}`. + fn build_struct_array( + &self, + rows: RecordSlice, + struct_fields: &[Field], + projection: &[String], + ) -> ArrowResult> { + let arrays: ArrowResult> = struct_fields + .iter() + .filter(|field| projection.is_empty() || projection.contains(field.name())) + .map(|field| { + match field.data_type() { + DataType::Null => { + Ok(Arc::new(NullArray::new(rows.len())) as ArrayRef) + } + DataType::Boolean => self.build_boolean_array(rows, field.name()), + DataType::Float64 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::Float32 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::Int64 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::Int32 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::Int16 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::Int8 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::UInt64 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::UInt32 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::UInt16 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::UInt8 => { + self.build_primitive_array::(rows, field.name()) + } + // TODO: this is incomplete + DataType::Timestamp(unit, _) => match unit { + TimeUnit::Second => self + .build_primitive_array::( + rows, + field.name(), + ), + TimeUnit::Microsecond => self + .build_primitive_array::( + rows, + field.name(), + ), + TimeUnit::Millisecond => self + .build_primitive_array::( + rows, + field.name(), + ), + TimeUnit::Nanosecond => self + .build_primitive_array::( + rows, + field.name(), + ), + }, + DataType::Date64 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::Date32 => { + self.build_primitive_array::(rows, field.name()) + } + DataType::Time64(unit) => match unit { + TimeUnit::Microsecond => self + .build_primitive_array::( + rows, + field.name(), + ), + TimeUnit::Nanosecond => self + .build_primitive_array::( + rows, + field.name(), + ), + t => Err(ArrowError::SchemaError(format!( + "TimeUnit {:?} not supported with Time64", + t + ))), + }, + DataType::Time32(unit) => match unit { + TimeUnit::Second => self + .build_primitive_array::( + rows, + field.name(), + ), + TimeUnit::Millisecond => self + .build_primitive_array::( + rows, + field.name(), + ), + t => Err(ArrowError::SchemaError(format!( + "TimeUnit {:?} not supported with Time32", + t + ))), + }, + DataType::Utf8 => Ok(Arc::new( + rows.iter() + .map(|row| { + let maybe_value = self.field_lookup(field.name(), row); + maybe_value.and_then(|value| self.as_string(value)) + }) + .collect::(), + ) as ArrayRef), + DataType::List(ref list_field) => { + match list_field.data_type() { + DataType::Dictionary(ref key_ty, _) => { + self.build_wrapped_list_array(rows, field.name(), key_ty) + } + _ => { + // extract rows by name + let extracted_rows = rows + .iter() + .map(|row| { + self.field_lookup(field.name(), row) + .unwrap_or(Value::Null) + }) + .collect::>(); + self.build_nested_list_array::( + extracted_rows.as_slice(), + list_field, + ) + } + } + } + DataType::Dictionary(ref key_ty, ref val_ty) => self + .build_string_dictionary_array( + rows, + field.name(), + key_ty, + val_ty, + ), + DataType::Struct(fields) => { + let len = rows.len(); + let num_bytes = bit_util::ceil(len, 8); + let mut null_buffer = MutableBuffer::from_len_zeroed(num_bytes); + let struct_rows = rows + .iter() + .enumerate() + .map(|(i, row)| (i, self.field_lookup(field.name(), row))) + .map(|(i, v)| match v { + // we want the field as an object, if it's not, we treat as null + Some(Value::Record(ref value)) => { + bit_util::set_bit(null_buffer.as_slice_mut(), i); + value.clone() + } + _ => vec![], + }) + .collect::>>(); + let arrays = + self.build_struct_array(struct_rows.as_slice(), fields, &[])?; + // construct a struct array's data in order to set null buffer + let data_type = DataType::Struct(fields.clone()); + let data = ArrayDataBuilder::new(data_type) + .len(len) + .null_bit_buffer(null_buffer.into()) + .child_data( + arrays.into_iter().map(|a| a.data().clone()).collect(), + ) + .build(); + Ok(make_array(data)) + } + _ => Err(ArrowError::SchemaError(format!( + "type {:?} not supported", + field.data_type() + ))), + } + }) + .collect(); + arrays + } + + /// Read the primitive list's values into ArrayData + fn read_primitive_list_values(&self, rows: &[Value]) -> ArrayData + where + T: ArrowPrimitiveType + ArrowNumericType, + T::Native: num_traits::cast::NumCast, + { + let values = rows + .iter() + .flat_map(|row| { + // read values from list + if let Value::Array(values) = row { + values + .iter() + .map(|value| { + let v: Option = + value_as_f64(value).and_then(num_traits::cast::cast); + v + }) + .collect::>>() + } else if let Some(f) = value_as_f64(row) { + // handle the scalar number case + let v: Option = num_traits::cast::cast(f); + v.map(|v| vec![Some(v)]).unwrap_or_default() + } else { + vec![] + } + }) + .collect::>>(); + let array = values.iter().collect::>(); + array.data().clone() + } + + fn field_lookup(&self, name: &str, row: &[(String, Value)]) -> Option { + self.schema_lookup + .get(name) + .and_then(|i| row.get(*i)) + .map(|o| o.1.clone()) + } + + fn as_string(&self, v: Value) -> Option { + match v { + Value::String(s) => Ok(Value::String(s)), + Value::Bytes(bytes) => Ok(Value::String( + String::from_utf8(bytes) + .map_err(avro_rs::Error::ConvertToUtf8) + .ok()?, + )), + other => Err(avro_rs::Error::GetString(other.into())), + } + .ok() + .and_then(|v| match v { + Value::String(s) => Some(s.clone()), + _ => None, + }) + } +} + +/// Flattens a list of Avro values, by flattening lists, and treating all other values as +/// single-value lists. +/// This is used to read into nested lists (list of list, list of struct) and non-dictionary lists. +#[inline] +fn flatten_values(values: &[Value]) -> Vec { + values + .iter() + .flat_map(|row| { + if let Value::Array(values) = row { + values.clone() + } else if let Value::Null = row { + vec![Value::Null] + } else { + // we interpret a scalar as a single-value list to minimise data loss + vec![row.clone()] + } + }) + .collect() +} + +/// Flattens a list into string values, dropping Value::Null in the process. +/// This is useful for interpreting any Avro array as string, dropping nulls. +/// See `value_as_string`. +#[inline] +fn flatten_string_values(values: &[Value]) -> Vec> { + values + .iter() + .flat_map(|row| { + if let Value::Array(values) = row { + values + .iter() + .map(value_as_string) + .collect::>>() + } else if let Value::Null = row { + vec![] + } else { + vec![value_as_string(row)] + } + }) + .collect::>>() +} + +/// Reads an Avro value as a string, regardless of its type. +/// This is useful if the expected datatype is a string, in which case we preserve +/// all the values regardless of they type. +#[inline(always)] +fn value_as_string(value: &Value) -> Option { + match value { + Value::Null => None, + Value::String(string) => Some(string.clone()), + _ => None, + } +} + +fn value_as_f64(value: &Value) -> Option { + match value { + Value::Int(n) => Ok(Value::Double(f64::from(*n))), + Value::Long(n) => Ok(Value::Double(*n as f64)), + Value::Float(x) => Ok(Value::Double(f64::from(*x))), + Value::Double(x) => Ok(Value::Double(*x)), + other => Err(avro_rs::Error::GetDouble(other.into())), + } + .ok() + .and_then(|v| match v { + Value::Double(f) => Some(f), + _ => None, + }) +} diff --git a/datafusion/src/avro/mod.rs b/datafusion/src/avro/mod.rs index 67a6e4bbaa66..79b795123e88 100644 --- a/datafusion/src/avro/mod.rs +++ b/datafusion/src/avro/mod.rs @@ -17,8 +17,11 @@ //! This module contains utilities to manipulate avro metadata. +mod arrow_array_reader; +mod reader; + use crate::arrow::datatypes::{DataType, IntervalUnit, Schema, TimeUnit}; -use crate::error::Result; +use crate::error::{DataFusionError, Result}; use arrow::datatypes::Field; use avro_rs::schema::Name; use avro_rs::types::Value; @@ -26,6 +29,8 @@ use avro_rs::Schema as AvroSchema; use std::collections::BTreeMap; use std::convert::TryFrom; +pub use reader::{infer_avro_schema_from_reader, Reader, ReaderBuilder}; + /// Converts an avro schema to an arrow schema pub fn to_arrow_schema(avro_schema: &avro_rs::Schema) -> Result { let mut schema_fields = vec![]; @@ -83,13 +88,30 @@ fn schema_to_field_with_props( ) } AvroSchema::Union(us) => { - nullable = us.find_schema(&Value::Null).is_some(); - let fields: Result> = us - .variants() - .into_iter() - .map(|s| schema_to_field_with_props(&s, None, nullable, None)) - .collect(); - DataType::Union(fields?) + // If there are only two variants and one of them is null, set the other type as the field data type + let has_nullable = us.find_schema(&Value::Null).is_some(); + let sub_schemas = us.variants(); + if has_nullable && sub_schemas.len() == 2 { + nullable = true; + if let Some(schema) = sub_schemas + .iter() + .find(|&schema| !matches!(schema, AvroSchema::Null)) + { + schema_to_field_with_props(&schema, None, has_nullable, None)? + .data_type() + .clone() + } else { + return Err(DataFusionError::AvroError( + avro_rs::Error::GetUnionDuplicate, + )); + } + } else { + let fields = sub_schemas + .into_iter() + .map(|s| schema_to_field_with_props(&s, None, has_nullable, None)) + .collect::>>()?; + DataType::Union(fields) + } } AvroSchema::Record { name, fields, .. } => { let fields: Result> = fields diff --git a/datafusion/src/avro/reader.rs b/datafusion/src/avro/reader.rs new file mode 100644 index 000000000000..4ed1d4020ab4 --- /dev/null +++ b/datafusion/src/avro/reader.rs @@ -0,0 +1,160 @@ +use crate::arrow::datatypes::{Schema, SchemaRef}; +use crate::arrow::record_batch::RecordBatch; +use crate::avro::arrow_array_reader::AvroArrowArrayReader; +use crate::error::Result; +use arrow::error::Result as ArrowResult; +use avro_rs::Reader as AvroReader; +use std::io::{BufReader, Read, Seek}; +use std::sync::Arc; + +/// Avro file reader builder +#[derive(Debug)] +pub struct ReaderBuilder { + /// Optional schema for the JSON file + /// + /// If the schema is not supplied, the reader will try to infer the schema + /// based on the JSON structure. + schema: Option, + /// Batch size (number of records to load each time) + /// + /// The default batch size when using the `ReaderBuilder` is 1024 records + batch_size: usize, + /// Optional projection for which columns to load (zero-based column indices) + projection: Option>, +} + +impl Default for ReaderBuilder { + fn default() -> Self { + Self { + schema: None, + batch_size: 1024, + projection: None, + } + } +} + +impl ReaderBuilder { + /// Create a new builder for configuring JSON parsing options. + /// + /// To convert a builder into a reader, call `Reader::from_builder` + /// + /// # Example + /// + /// ``` + /// extern crate avro_rs; + /// + /// use std::fs::File; + /// + /// fn example() -> avro_rs::Reader { + /// let file = File::open("test/data/basic.avro").unwrap(); + /// + /// // create a builder, inferring the schema with the first 100 records + /// let builder = crate::datafusion::avro::ReaderBuilder::new().infer_schema(Some(100)); + /// + /// let reader = builder.build::(file).unwrap(); + /// + /// reader + /// } + /// ``` + pub fn new() -> Self { + Self::default() + } + + /// Set the Avro file's schema + pub fn with_schema(mut self, schema: SchemaRef) -> Self { + self.schema = Some(schema); + self + } + + /// Set the Avro reader to infer the schema of the file + pub fn infer_schema(mut self) -> Self { + // remove any schema that is set + self.schema = None; + self + } + + /// Set the batch size (number of records to load at one time) + pub fn with_batch_size(mut self, batch_size: usize) -> Self { + self.batch_size = batch_size; + self + } + + /// Set the reader's column projection + pub fn with_projection(mut self, projection: Vec) -> Self { + self.projection = Some(projection); + self + } + + /// Create a new `Reader` from the `ReaderBuilder` + pub fn build<'a, R>(self, source: R) -> Result>> + where + R: Read + Seek, + { + let mut buf_reader = BufReader::new(source); + + // check if schema should be inferred + let schema = match self.schema { + Some(schema) => schema, + None => Arc::new(infer_avro_schema_from_reader(&mut buf_reader)?), + }; + + Reader::try_new(buf_reader, schema, self.batch_size, self.projection) + } +} + +/// Avro file record reader +pub struct Reader<'a, R: Read> { + array_reader: AvroArrowArrayReader<'a, R>, + schema: SchemaRef, + batch_size: usize, +} + +impl<'a, R: Read> Reader<'a, R> { + /// Create a new Avro Reader from any value that implements the `Read` trait. + /// + /// If reading a `File`, you can customise the Reader, such as to enable schema + /// inference, use `ReaderBuilder`. + pub fn try_new( + reader: R, + schema: SchemaRef, + batch_size: usize, + projection: Option>, + ) -> Result { + Ok(Self { + array_reader: AvroArrowArrayReader::try_new( + AvroReader::new(reader)?, + schema.clone(), + projection, + )?, + schema, + batch_size, + }) + } + + /// Returns the schema of the reader, useful for getting the schema without reading + /// record batches + pub fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + /// Read the next batch of records + #[allow(clippy::should_implement_trait)] + pub fn next(&mut self) -> ArrowResult> { + self.array_reader.next_batch(self.batch_size) + } +} + +impl<'a, R: Read> Iterator for Reader<'a, R> { + type Item = ArrowResult; + + fn next(&mut self) -> Option { + self.next().transpose() + } +} + +/// Infer Avro schema given a reader +pub fn infer_avro_schema_from_reader(reader: &mut R) -> Result { + let avro_reader = avro_rs::Reader::new(reader)?; + let schema = avro_reader.writer_schema(); + super::to_arrow_schema(schema) +} diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index 8efe55590e69..9eeb269f96ce 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -40,7 +40,7 @@ trait SeekRead: Read + Seek {} impl SeekRead for T {} -/// Represents a line-delimited JSON file with a provided schema +/// Represents a line-delimited Avro file with a provided schema pub struct AvroFile { source: Source>, schema: SchemaRef, @@ -97,7 +97,7 @@ impl AvroFile { if let Some(schema) = options.schema { schema } else { - Arc::new(AvroExec::infer_avro_schema_from_reader(&mut reader)?) + Arc::new(crate::avro::infer_avro_schema_from_reader(&mut reader)?) } }; @@ -179,28 +179,245 @@ impl TableProvider for AvroFile { #[cfg(test)] mod tests { use super::*; - use crate::prelude::*; - const TEST_DATA_BASE: &str = "tests/jsons"; + use crate::datasource::avro::AvroFile; + use crate::datasource::TableProvider; + use arrow::array::{ + BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, + TimestampNanosecondArray, + }; + use arrow::record_batch::RecordBatch; + use futures::StreamExt; + + fn load_table(name: &str) -> Result> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/{}", testdata, name); + let table = AvroFile::try_new(&filename, AvroReadOptions::default())?; + Ok(Arc::new(table)) + } + + #[tokio::test] + async fn read_small_batches() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = None; + let exec = table.scan(&projection, 2, &[], None)?; + let stream = exec.execute(0).await?; + + let _ = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; + + // test metadata + assert_eq!(table.statistics().num_rows, Some(8)); + assert_eq!(table.statistics().total_byte_size, Some(671)); + + Ok(()) + } + + #[tokio::test] + async fn read_alltypes_plain_avro() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + + let x: Vec = table + .schema() + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect(); + let y = x.join("\n"); + assert_eq!( + "id: Int32\n\ + bool_col: Boolean\n\ + tinyint_col: Int32\n\ + smallint_col: Int32\n\ + int_col: Int32\n\ + bigint_col: Int64\n\ + float_col: Float32\n\ + double_col: Float64\n\ + date_string_col: Binary\n\ + string_col: Binary\n\ + timestamp_col: Timestamp(Nanosecond, None)", + y + ); + + let projection = None; + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(11, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + Ok(()) + } + + #[tokio::test] + async fn read_bool_alltypes_plain_avro() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![1]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[true, false, true, false, true, false, true, false]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[tokio::test] + async fn read_i32_alltypes_plain_avro() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![0]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); + + Ok(()) + } #[tokio::test] - async fn csv_file_from_reader() -> Result<()> { - let mut ctx = ExecutionContext::new(); - let path = format!("{}/2.json", TEST_DATA_BASE); - ctx.register_table( - "ndjson", - Arc::new(AvroFile::try_new(&path, Default::default())?), - )?; - let df = ctx.sql("select sum(a) from ndjson")?; - let batches = df.collect().await?; + async fn read_i96_alltypes_plain_avro() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![10]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); + + Ok(()) + } + + #[tokio::test] + async fn read_f32_alltypes_plain_avro() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![6]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + assert_eq!( - batches[0] - .column(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0), - 100000000000011 + "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", + format!("{:?}", values) ); + Ok(()) } + + #[tokio::test] + async fn read_f64_alltypes_plain_avro() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![7]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[tokio::test] + async fn read_binary_alltypes_plain_avro() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![9]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec<&str> = vec![]; + for i in 0..batch.num_rows() { + values.push(std::str::from_utf8(array.value(i)).unwrap()); + } + + assert_eq!( + "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", + format!("{:?}", values) + ); + + Ok(()) + } + + async fn get_first_batch( + table: Arc, + projection: &Option>, + ) -> Result { + let exec = table.scan(projection, 1024, &[], None)?; + let mut it = exec.execute(0).await?; + it.next() + .await + .expect("should have received at least one batch") + .map_err(|e| e.into()) + } } diff --git a/datafusion/src/logical_plan/dfschema.rs b/datafusion/src/logical_plan/dfschema.rs index c067b5f963ee..d2918421e4a0 100644 --- a/datafusion/src/logical_plan/dfschema.rs +++ b/datafusion/src/logical_plan/dfschema.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use crate::error::{DataFusionError, Result}; use crate::logical_plan::Column; +use crate::utils::get_field; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use std::fmt::{Display, Formatter}; @@ -140,6 +141,7 @@ impl DFSchema { return Ok(i); } } + println!("{}", name); Err(DataFusionError::Plan(format!( "No field named '{}'. Valid fields are {}.", name, @@ -160,14 +162,17 @@ impl DFSchema { // field to lookup is qualified. // current field is qualified and not shared between relations, compare both // qualifer and name. - (Some(q), Some(field_q)) => q == field_q && field.name() == name, + (Some(q), Some(field_q)) => { + (q == field_q && field.name() == name) + || (q == field.name() + && get_field(field.field.data_type(), name).is_ok()) + } // field to lookup is qualified but current field is unqualified. (Some(_), None) => false, // field to lookup is unqualified, no need to compare qualifier (None, Some(_)) | (None, None) => field.name() == name, }) .map(|(idx, _)| idx); - match matches.next() { None => Err(DataFusionError::Plan(format!( "No field named '{}.{}'. Valid fields are {}.", diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index 9d5235a2405d..d6e18317ecad 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -15,17 +15,16 @@ // specific language governing permissions and limitations // under the License. -//! Execution plan for reading line-delimited JSON files +//! Execution plan for reading line-delimited Avro files use async_trait::async_trait; use futures::Stream; use super::{common, source::Source, ExecutionPlan, Partitioning, RecordBatchStream}; -use crate::avro; +use crate::avro::infer_avro_schema_from_reader; use crate::error::{DataFusionError, Result}; use arrow::{ datatypes::{Schema, SchemaRef}, error::Result as ArrowResult, - json, record_batch::RecordBatch, }; use std::fs::File; @@ -37,7 +36,7 @@ use std::{ task::{Context, Poll}, }; -/// Line-delimited JSON read options +/// Line-delimited Avro read options #[derive(Clone)] pub struct AvroReadOptions<'a> { /// The data source schema. @@ -60,12 +59,13 @@ impl<'a> Default for AvroReadOptions<'a> { trait SeekRead: Read + Seek {} impl SeekRead for T {} -/// Execution plan for scanning NdJson data source +/// Execution plan for scanning Avro data source #[derive(Debug)] pub struct AvroExec { source: Source>, schema: SchemaRef, projection: Option>, + projected_schema: SchemaRef, file_extension: String, batch_size: usize, limit: Option, @@ -109,7 +109,8 @@ impl AvroExec { path: path.to_string(), filenames, }, - schema: projected_schema.clone(), + schema, + projected_schema, file_extension, projection, batch_size, @@ -143,9 +144,10 @@ impl AvroExec { Ok(Self { source: Source::Reader(Mutex::new(Some(Box::new(reader)))), - schema: projected_schema.clone(), + schema, file_extension: String::new(), projection, + projected_schema, batch_size, limit, }) @@ -191,21 +193,12 @@ impl AvroExec { let mut schemas = Vec::new(); for filename in filenames { let mut file = File::open(filename)?; - let schema = Self::infer_avro_schema_from_reader(&mut file)?; + let schema = infer_avro_schema_from_reader(&mut file)?; schemas.push(schema); } Ok(Schema::try_merge(schemas)?) } - - /// Infer Avro schema given a reader - pub fn infer_avro_schema_from_reader( - reader: &mut R, - ) -> Result { - let avro_reader = avro_rs::Reader::new(reader)?; - let schema = avro_reader.writer_schema(); - avro::to_arrow_schema(schema) - } } #[async_trait] @@ -215,7 +208,7 @@ impl ExecutionPlan for AvroExec { } fn schema(&self) -> SchemaRef { - self.schema.clone() + self.projected_schema.clone() } fn output_partitioning(&self) -> Partitioning { @@ -246,6 +239,7 @@ impl ExecutionPlan for AvroExec { }, schema: self.schema.clone(), projection: self.projection.clone(), + projected_schema: self.projected_schema.clone(), batch_size: self.batch_size, limit: self.limit, file_extension: self.file_extension.clone(), @@ -262,7 +256,7 @@ impl ExecutionPlan for AvroExec { &self, partition: usize, ) -> Result { - let mut builder = json::ReaderBuilder::new() + let mut builder = crate::avro::ReaderBuilder::new() .with_schema(self.schema.clone()) .with_batch_size(self.batch_size); if let Some(proj) = &self.projection { @@ -298,13 +292,13 @@ impl ExecutionPlan for AvroExec { } } -struct AvroStream { - reader: json::Reader, +struct AvroStream<'a, R: Read> { + reader: crate::avro::Reader<'a, R>, remain: Option, } -impl AvroStream { - fn new(reader: json::Reader, limit: Option) -> Self { +impl<'a, R: Read> AvroStream<'a, R> { + fn new(reader: crate::avro::Reader<'a, R>, limit: Option) -> Self { Self { reader, remain: limit, @@ -312,7 +306,7 @@ impl AvroStream { } } -impl Stream for AvroStream { +impl Stream for AvroStream<'_, R> { type Item = ArrowResult; fn poll_next( @@ -352,7 +346,7 @@ impl Stream for AvroStream { } } -impl RecordBatchStream for AvroStream { +impl RecordBatchStream for AvroStream<'_, R> { fn schema(&self) -> SchemaRef { self.reader.schema() } @@ -361,112 +355,271 @@ impl RecordBatchStream for AvroStream { #[cfg(test)] mod tests { use super::*; + use crate::datasource::avro::AvroFile; + use crate::datasource::TableProvider; + use crate::logical_plan::combine_filters; + use arrow::array::{ + BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, + TimestampNanosecondArray, + }; + use arrow::record_batch::RecordBatch; use futures::StreamExt; - const TEST_DATA_BASE: &str = "tests/jsons"; + fn load_table(name: &str) -> Result> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/{}", testdata, name); + let table = AvroFile::try_new(&filename, AvroReadOptions::default())?; + Ok(Arc::new(table)) + } #[tokio::test] - async fn avro_exec_file_without_projection() -> Result<()> { - use arrow::datatypes::DataType; - let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = AvroExec::try_new(&path, Default::default(), None, 1024, Some(3))?; - let inferred_schema = exec.schema(); - assert_eq!(inferred_schema.fields().len(), 4); - - // a,b,c,d should be inferred - inferred_schema.field_with_name("a").unwrap(); - inferred_schema.field_with_name("b").unwrap(); - inferred_schema.field_with_name("c").unwrap(); - inferred_schema.field_with_name("d").unwrap(); + async fn read_small_batches() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = None; + let exec = table.scan(&projection, 2, &[], None)?; + let stream = exec.execute(0).await?; + + let _ = stream + .map(|batch| { + let batch = batch.unwrap(); + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + }) + .fold(0, |acc, _| async move { acc + 1i32 }) + .await; + + // test metadata + assert_eq!(table.statistics().num_rows, Some(8)); + assert_eq!(table.statistics().total_byte_size, Some(671)); + + Ok(()) + } + #[tokio::test] + async fn read_alltypes_plain_parquet() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + + let x: Vec = table + .schema() + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect(); + let y = x.join("\n"); assert_eq!( - inferred_schema.field_with_name("a").unwrap().data_type(), - &DataType::Int64 + "id: Int32\n\ + bool_col: Boolean\n\ + tinyint_col: Int32\n\ + smallint_col: Int32\n\ + int_col: Int32\n\ + bigint_col: Int64\n\ + float_col: Float32\n\ + double_col: Float64\n\ + date_string_col: Binary\n\ + string_col: Binary\n\ + timestamp_col: Timestamp(Microsecond, None)", + y ); - assert!(matches!( - inferred_schema.field_with_name("b").unwrap().data_type(), - DataType::List(_) - )); + + let projection = None; + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(11, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + Ok(()) + } + + #[tokio::test] + async fn read_bool_alltypes_plain_parquet() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![1]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + assert_eq!( - inferred_schema.field_with_name("d").unwrap().data_type(), - &DataType::Utf8 + "[true, false, true, false, true, false, true, false]", + format!("{:?}", values) ); - let mut it = exec.execute(0).await?; - let batch = it.next().await.unwrap()?; + Ok(()) + } + + #[tokio::test] + async fn read_i32_alltypes_plain_parquet() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![0]); + let batch = get_first_batch(table, &projection).await?; - assert_eq!(batch.num_rows(), 3); - let values = batch + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch .column(0) .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); - assert_eq!(values.value(0), 1); - assert_eq!(values.value(1), -10); - assert_eq!(values.value(2), 2); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); Ok(()) } #[tokio::test] - async fn avro_exec_file_projection() -> Result<()> { - let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = - AvroExec::try_new(&path, Default::default(), Some(vec![0, 2]), 1024, None)?; - let inferred_schema = exec.schema(); - assert_eq!(inferred_schema.fields().len(), 2); - - inferred_schema.field_with_name("a").unwrap(); - inferred_schema.field_with_name("b").unwrap_err(); - inferred_schema.field_with_name("c").unwrap(); - inferred_schema.field_with_name("d").unwrap_err(); + async fn read_i96_alltypes_plain_parquet() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![10]); + let batch = get_first_batch(table, &projection).await?; - let mut it = exec.execute(0).await?; - let batch = it.next().await.unwrap()?; + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); - assert_eq!(batch.num_rows(), 4); - let values = batch + let array = batch .column(0) .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); - assert_eq!(values.value(0), 1); - assert_eq!(values.value(1), -10); - assert_eq!(values.value(2), 2); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); + Ok(()) } #[tokio::test] - async fn avro_exec_from_reader() -> Result<()> { - let content = r#"{"a":"aaa", "b":[2.0, 1.3, -6.1], "c":[false, true], "d":"4"} -{"a":"bbb", "b":[2.0, 1.3, -6.1], "c":[true, true], "d":"4"}"#; - let cur = std::io::Cursor::new(content); - let mut bufrdr = std::io::BufReader::new(cur); - let schema = - arrow::json::reader::infer_json_schema_from_seekable(&mut bufrdr, None)?; - let exec = AvroExec::try_new_from_reader( - bufrdr, - AvroReadOptions { - schema: Some(Arc::new(schema)), - ..Default::default() - }, - None, - 1024, - Some(1), - )?; + async fn read_f32_alltypes_plain_parquet() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![6]); + let batch = get_first_batch(table, &projection).await?; - let mut it = exec.execute(0).await?; - let batch = it.next().await.unwrap()?; + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[tokio::test] + async fn read_f64_alltypes_plain_parquet() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![7]); + let batch = get_first_batch(table, &projection).await?; + + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..batch.num_rows() { + values.push(array.value(i)); + } + + assert_eq!( + "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", + format!("{:?}", values) + ); + + Ok(()) + } + + #[tokio::test] + async fn read_binary_alltypes_plain_parquet() -> Result<()> { + let table = load_table("alltypes_plain.avro")?; + let projection = Some(vec![9]); + let batch = get_first_batch(table, &projection).await?; - assert_eq!(batch.num_rows(), 1); + assert_eq!(1, batch.num_columns()); + assert_eq!(8, batch.num_rows()); - let values = batch + let array = batch .column(0) .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); - assert_eq!(values.value(0), "aaa"); + let mut values: Vec<&str> = vec![]; + for i in 0..batch.num_rows() { + values.push(std::str::from_utf8(array.value(i)).unwrap()); + } + + assert_eq!( + "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", + format!("{:?}", values) + ); Ok(()) } + + async fn get_first_batch( + table: Arc, + projection: &Option>, + ) -> Result { + let exec = table.scan(projection, 1024, &[], None)?; + let mut it = exec.execute(0).await?; + it.next() + .await + .expect("should have received at least one batch") + .map_err(|e| e.into()) + } + + #[test] + fn combine_zero_filters() { + let result = combine_filters(&[]); + assert_eq!(result, None); + } + + #[test] + fn combine_one_filter() { + use crate::logical_plan::{binary_expr, col, lit, Operator}; + let filter = binary_expr(col("c1"), Operator::Lt, lit(1)); + let result = combine_filters(&[filter.clone()]); + assert_eq!(result, Some(filter)); + } + + #[test] + fn combine_multiple_filters() { + use crate::logical_plan::{and, binary_expr, col, lit, Operator}; + let filter1 = binary_expr(col("c1"), Operator::Lt, lit(1)); + let filter2 = binary_expr(col("c2"), Operator::Lt, lit(2)); + let filter3 = binary_expr(col("c3"), Operator::Lt, lit(3)); + let result = + combine_filters(&[filter1.clone(), filter2.clone(), filter3.clone()]); + assert_eq!(result, Some(and(and(filter1, filter2), filter3))); + } } From d42cdd18042e3fd3e67829f9a78bc94f6f363bda Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 18 Aug 2021 12:29:34 +0200 Subject: [PATCH 07/58] Add binary types --- datafusion-examples/Cargo.toml | 1 + datafusion-examples/examples/avro_sql.rs | 8 ++- datafusion/Cargo.toml | 2 +- datafusion/src/avro/arrow_array_reader.rs | 64 +++++++++++++++++++---- datafusion/src/avro/reader.rs | 1 - 5 files changed, 59 insertions(+), 17 deletions(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 9b859c6238f8..f98121b0ece8 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -36,3 +36,4 @@ tonic = "0.5" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync"] } futures = "0.3" num_cpus = "1.13.0" +avro-rs = { version = "0.13", features = ["snappy"] } diff --git a/datafusion-examples/examples/avro_sql.rs b/datafusion-examples/examples/avro_sql.rs index 793228224345..98ac3c605eb8 100644 --- a/datafusion-examples/examples/avro_sql.rs +++ b/datafusion-examples/examples/avro_sql.rs @@ -20,6 +20,7 @@ use datafusion::arrow::util::pretty; use datafusion::error::Result; use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::prelude::*; +use std::fs::File; /// This example demonstrates executing a simple query against an Arrow data source (Parquet) and /// fetching results @@ -31,11 +32,8 @@ async fn main() -> Result<()> { let testdata = datafusion::arrow::util::test_util::arrow_test_data(); // register parquet file with the execution context - ctx.register_avro( - "alltypes_plain", - &format!("{}/avro/alltypes_plain.avro", testdata), - AvroReadOptions::default(), - )?; + let avro_file = &format!("{}/avro/alltypes_plain.avro", testdata); + ctx.register_avro("alltypes_plain", avro_file, AvroReadOptions::default())?; let df = ctx.table("alltypes_plain").unwrap(); // execute the query diff --git a/datafusion/Cargo.toml b/datafusion/Cargo.toml index ca89185fc8be..fee3f08746d9 100644 --- a/datafusion/Cargo.toml +++ b/datafusion/Cargo.toml @@ -69,7 +69,7 @@ regex = { version = "^1.4.3", optional = true } lazy_static = { version = "^1.4.0", optional = true } smallvec = { version = "1.6", features = ["union"] } rand = "0.8" -avro-rs = "0.13" +avro-rs = { version = "0.13", features = ["snappy"] } num-traits = "0.2" [dev-dependencies] diff --git a/datafusion/src/avro/arrow_array_reader.rs b/datafusion/src/avro/arrow_array_reader.rs index f43daa2fa645..f5aa7d7874b0 100644 --- a/datafusion/src/avro/arrow_array_reader.rs +++ b/datafusion/src/avro/arrow_array_reader.rs @@ -17,13 +17,13 @@ use crate::arrow::error::ArrowError; use crate::arrow::record_batch::RecordBatch; use crate::arrow::util::bit_util; use crate::error::{DataFusionError, Result}; -use arrow::array::GenericListArray; +use arrow::array::{BinaryArray, GenericListArray}; use arrow::datatypes::SchemaRef; use arrow::error::ArrowError::SchemaError; use arrow::error::Result as ArrowResult; use avro_rs::schema::Schema as AvroSchema; use avro_rs::types::Value; -use avro_rs::Reader as AvroReader; +use avro_rs::{AvroResult, Reader as AvroReader}; use std::collections::HashMap; use std::io::Read; use std::sync::Arc; @@ -85,10 +85,10 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } } } - /*if rows.is_empty() { + if rows.is_empty() { // reached end of file return Ok(None); - }*/ + } let rows = &rows[..]; let projection = self.projection.clone().unwrap_or_else(Vec::new); let arrays = self.build_struct_array(rows, self.schema.fields(), &projection); @@ -240,14 +240,14 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } #[inline(always)] - fn list_array_string_array_builder( + fn list_array_string_array_builder( &self, data_type: &DataType, col_name: &str, rows: RecordSlice, ) -> ArrowResult where - DICT_TY: ArrowPrimitiveType + ArrowDictionaryKeyType, + D: ArrowPrimitiveType + ArrowDictionaryKeyType, { let mut builder: Box = match data_type { DataType::Utf8 => { @@ -256,7 +256,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } DataType::Dictionary(_, _) => { let values_builder = - self.build_string_dictionary_builder::(rows.len() * 5)?; + self.build_string_dictionary_builder::(rows.len() * 5)?; Box::new(ListBuilder::new(values_builder)) } e => { @@ -321,7 +321,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { builder.append(true)?; } DataType::Dictionary(_, _) => { - let builder = builder.as_any_mut().downcast_mut::>>().ok_or_else(||ArrowError::JsonError( + let builder = builder.as_any_mut().downcast_mut::>>().ok_or_else(||ArrowError::JsonError( "Cast failed for ListBuilder during nested data parsing".to_string(), ))?; for val in vals { @@ -695,14 +695,24 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { t ))), }, - DataType::Utf8 => Ok(Arc::new( + DataType::Utf8 | DataType::LargeUtf8 => Ok(Arc::new( rows.iter() .map(|row| { let maybe_value = self.field_lookup(field.name(), row); maybe_value.and_then(|value| self.as_string(value)) }) .collect::(), - ) as ArrayRef), + ) + as ArrayRef), + DataType::Binary | DataType::LargeBinary => Ok(Arc::new( + rows.iter() + .map(|row| { + let maybe_value = self.field_lookup(field.name(), row); + maybe_value.and_then(|value| self.as_bytes(value)) + }) + .collect::(), + ) + as ArrayRef), DataType::List(ref list_field) => { match list_field.data_type() { DataType::Dictionary(ref key_ty, _) => { @@ -811,6 +821,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } fn as_string(&self, v: Value) -> Option { + let v = if let Value::Union(b) = v { *b } else { v }; match v { Value::String(s) => Ok(Value::String(s)), Value::Bytes(bytes) => Ok(Value::String( @@ -826,6 +837,27 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { _ => None, }) } + + fn as_bytes(&self, v: Value) -> Option> { + let v = if let Value::Union(b) = v { *b } else { v }; + match v { + Value::Bytes(bytes) => Ok(Value::Bytes(bytes)), + Value::String(s) => Ok(Value::Bytes(s.into_bytes())), + Value::Array(items) => Ok(Value::Bytes( + items + .into_iter() + .map(try_u8) + .collect::, _>>() + .ok()?, + )), + other => Err(avro_rs::Error::GetBytes(other.into())), + } + .ok() + .and_then(|v| match v { + Value::Bytes(s) => Some(s.clone()), + _ => None, + }) + } } /// Flattens a list of Avro values, by flattening lists, and treating all other values as @@ -896,3 +928,15 @@ fn value_as_f64(value: &Value) -> Option { _ => None, }) } + +fn try_u8(v: Value) -> AvroResult { + println!("{:?}", v); + let int = v.resolve(&AvroSchema::Int)?; + if let Value::Int(n) = int { + if n >= 0 && n <= i32::from(u8::MAX) { + return Ok(n as u8); + } + } + + Err(avro_rs::Error::GetU8(int.into())) +} diff --git a/datafusion/src/avro/reader.rs b/datafusion/src/avro/reader.rs index 4ed1d4020ab4..81e29d6df923 100644 --- a/datafusion/src/avro/reader.rs +++ b/datafusion/src/avro/reader.rs @@ -97,7 +97,6 @@ impl ReaderBuilder { Some(schema) => schema, None => Arc::new(infer_avro_schema_from_reader(&mut buf_reader)?), }; - Reader::try_new(buf_reader, schema, self.batch_size, self.projection) } } From cbdc9c72e6a10bf1e201f99bd1875a856979b833 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 18 Aug 2021 14:00:38 +0200 Subject: [PATCH 08/58] Enable Avro as a FileType --- Cargo.toml | 1 - ballista/rust/core/proto/ballista.proto | 1 + ballista/rust/core/src/serde/logical_plan/from_proto.rs | 1 + ballista/rust/core/src/serde/logical_plan/to_proto.rs | 1 + datafusion/src/sql/parser.rs | 5 ++++- datafusion/src/sql/planner.rs | 1 + 6 files changed, 8 insertions(+), 2 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index a454800fd506..d6da8c14cd96 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,7 +26,6 @@ members = [ "ballista/rust/executor", "ballista/rust/scheduler", "ballista-examples", - "avro-converter" ] exclude = ["python"] diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index dd9978f5c26d..601868fdd7b2 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -340,6 +340,7 @@ enum FileType{ NdJson = 0; Parquet = 1; CSV = 2; + Avro = 3; } message AnalyzeNode { diff --git a/ballista/rust/core/src/serde/logical_plan/from_proto.rs b/ballista/rust/core/src/serde/logical_plan/from_proto.rs index 38de341ed01d..fa4f44e915e9 100644 --- a/ballista/rust/core/src/serde/logical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/from_proto.rs @@ -1209,6 +1209,7 @@ impl Into for protobuf::FileType { protobuf::FileType::NdJson => FileType::NdJson, protobuf::FileType::Parquet => FileType::Parquet, protobuf::FileType::Csv => FileType::CSV, + protobuf::FileType::Avro => FileType::Avro, } } } diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index 953d59b32353..bfe8e0f26615 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -974,6 +974,7 @@ impl TryInto for &LogicalPlan { FileType::NdJson => protobuf::FileType::NdJson, FileType::Parquet => protobuf::FileType::Parquet, FileType::CSV => protobuf::FileType::Csv, + FileType::Avro => protobuf::FileType::Avro, }; Ok(protobuf::LogicalPlanNode { diff --git a/datafusion/src/sql/parser.rs b/datafusion/src/sql/parser.rs index bb2f9e6bbb24..ac303f649b93 100644 --- a/datafusion/src/sql/parser.rs +++ b/datafusion/src/sql/parser.rs @@ -43,6 +43,8 @@ pub enum FileType { Parquet, /// Comma separated values CSV, + /// Avro binary records + Avro, } impl FromStr for FileType { @@ -53,8 +55,9 @@ impl FromStr for FileType { "PARQUET" => Ok(Self::Parquet), "NDJSON" => Ok(Self::NdJson), "CSV" => Ok(Self::CSV), + "AVRO" => Ok(Self::Avro), other => Err(ParserError::ParserError(format!( - "expect one of PARQUET, NDJSON, or CSV, found: {}", + "expect one of PARQUET, AVRO, NDJSON, or CSV, found: {}", other ))), } diff --git a/datafusion/src/sql/planner.rs b/datafusion/src/sql/planner.rs index 233fd5af25ef..ccb5c8f903be 100644 --- a/datafusion/src/sql/planner.rs +++ b/datafusion/src/sql/planner.rs @@ -228,6 +228,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } FileType::NdJson => {} + FileType::Avro => {} }; let schema = self.build_schema(columns)?; From 7d6078ec57903f5321836a76c0440fd699b27426 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 18 Aug 2021 14:07:13 +0200 Subject: [PATCH 09/58] Enable registering an avro table in the sql parsing --- ballista/rust/core/src/serde/logical_plan/from_proto.rs | 1 + ballista/rust/core/src/serde/logical_plan/mod.rs | 8 ++++++-- datafusion/src/execution/context.rs | 5 +++++ datafusion/src/sql/parser.rs | 2 +- 4 files changed, 13 insertions(+), 3 deletions(-) diff --git a/ballista/rust/core/src/serde/logical_plan/from_proto.rs b/ballista/rust/core/src/serde/logical_plan/from_proto.rs index fa4f44e915e9..1fad748204c5 100644 --- a/ballista/rust/core/src/serde/logical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/from_proto.rs @@ -1193,6 +1193,7 @@ impl TryFrom for protobuf::FileType { _x if _x == FileType::NdJson as i32 => Ok(FileType::NdJson), _x if _x == FileType::Parquet as i32 => Ok(FileType::Parquet), _x if _x == FileType::Csv as i32 => Ok(FileType::Csv), + _x if _x == FileType::Avro as i32 => Ok(FileType::Avro), invalid => Err(BallistaError::General(format!( "Attempted to convert invalid i32 to protobuf::Filetype: {}", invalid diff --git a/ballista/rust/core/src/serde/logical_plan/mod.rs b/ballista/rust/core/src/serde/logical_plan/mod.rs index dbaac1de7b57..ada3c85de674 100644 --- a/ballista/rust/core/src/serde/logical_plan/mod.rs +++ b/ballista/rust/core/src/serde/logical_plan/mod.rs @@ -643,8 +643,12 @@ mod roundtrip_tests { let df_schema_ref = schema.to_dfschema_ref()?; - let filetypes: [FileType; 3] = - [FileType::NdJson, FileType::Parquet, FileType::CSV]; + let filetypes: [FileType; 4] = [ + FileType::NdJson, + FileType::Parquet, + FileType::CSV, + FileType::Avro, + ]; for file in filetypes.iter() { let create_table_node = LogicalPlan::CreateExternalTable { diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index b6926d46a228..f9c4b5a0f24f 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -199,6 +199,11 @@ impl ExecutionContext { let plan = LogicalPlanBuilder::empty(false).build()?; Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan))) } + FileType::Avro => { + self.register_avro(name, location, AvroReadOptions::default())?; + let plan = LogicalPlanBuilder::empty(false).build()?; + Ok(Arc::new(DataFrameImpl::new(self.state.clone(), &plan))) + } _ => Err(DataFusionError::NotImplemented(format!( "Unsupported file type {:?}.", file_type diff --git a/datafusion/src/sql/parser.rs b/datafusion/src/sql/parser.rs index ac303f649b93..527c3cddd9b3 100644 --- a/datafusion/src/sql/parser.rs +++ b/datafusion/src/sql/parser.rs @@ -396,7 +396,7 @@ mod tests { // Error cases: Invalid type let sql = "CREATE EXTERNAL TABLE t(c1 int) STORED AS UNKNOWN_TYPE LOCATION 'foo.csv'"; - expect_parse_error(sql, "expect one of PARQUET, NDJSON, or CSV"); + expect_parse_error(sql, "expect one of PARQUET, AVRO, NDJSON, or CSV"); Ok(()) } From 44cfa87bdab5e0cace4f71bd6d08b722dd252ce2 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 19 Aug 2021 22:03:17 +0200 Subject: [PATCH 10/58] Change package name for datafusion/avro --- datafusion/src/{avro => avro_to_arrow}/arrow_array_reader.rs | 0 datafusion/src/{avro => avro_to_arrow}/mod.rs | 0 datafusion/src/{avro => avro_to_arrow}/reader.rs | 4 ++-- 3 files changed, 2 insertions(+), 2 deletions(-) rename datafusion/src/{avro => avro_to_arrow}/arrow_array_reader.rs (100%) rename datafusion/src/{avro => avro_to_arrow}/mod.rs (100%) rename datafusion/src/{avro => avro_to_arrow}/reader.rs (96%) diff --git a/datafusion/src/avro/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs similarity index 100% rename from datafusion/src/avro/arrow_array_reader.rs rename to datafusion/src/avro_to_arrow/arrow_array_reader.rs diff --git a/datafusion/src/avro/mod.rs b/datafusion/src/avro_to_arrow/mod.rs similarity index 100% rename from datafusion/src/avro/mod.rs rename to datafusion/src/avro_to_arrow/mod.rs diff --git a/datafusion/src/avro/reader.rs b/datafusion/src/avro_to_arrow/reader.rs similarity index 96% rename from datafusion/src/avro/reader.rs rename to datafusion/src/avro_to_arrow/reader.rs index 81e29d6df923..21994778dc6b 100644 --- a/datafusion/src/avro/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -1,6 +1,6 @@ use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::arrow::record_batch::RecordBatch; -use crate::avro::arrow_array_reader::AvroArrowArrayReader; +use crate::avro_to_arrow::arrow_array_reader::AvroArrowArrayReader; use crate::error::Result; use arrow::error::Result as ArrowResult; use avro_rs::Reader as AvroReader; @@ -49,7 +49,7 @@ impl ReaderBuilder { /// let file = File::open("test/data/basic.avro").unwrap(); /// /// // create a builder, inferring the schema with the first 100 records - /// let builder = crate::datafusion::avro::ReaderBuilder::new().infer_schema(Some(100)); + /// let builder = crate::datafusion::avro_to_arrow::ReaderBuilder::new().infer_schema(Some(100)); /// /// let reader = builder.build::(file).unwrap(); /// From 6ef92d87334f792fe80593aa4f9ffa6857e4f9eb Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Fri, 20 Aug 2021 05:14:42 +0200 Subject: [PATCH 11/58] Implement Avro datasource tests and fix avro_rs::Value resolution to Arrow types --- datafusion-examples/examples/avro_sql.rs | 2 - .../src/avro_to_arrow/arrow_array_reader.rs | 180 +++++++++--------- datafusion/src/datasource/avro.rs | 39 ++-- datafusion/src/lib.rs | 2 +- datafusion/src/physical_plan/avro.rs | 9 +- 5 files changed, 118 insertions(+), 114 deletions(-) diff --git a/datafusion-examples/examples/avro_sql.rs b/datafusion-examples/examples/avro_sql.rs index 98ac3c605eb8..aaa6cc6381ee 100644 --- a/datafusion-examples/examples/avro_sql.rs +++ b/datafusion-examples/examples/avro_sql.rs @@ -20,7 +20,6 @@ use datafusion::arrow::util::pretty; use datafusion::error::Result; use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::prelude::*; -use std::fs::File; /// This example demonstrates executing a simple query against an Arrow data source (Parquet) and /// fetching results @@ -35,7 +34,6 @@ async fn main() -> Result<()> { let avro_file = &format!("{}/avro/alltypes_plain.avro", testdata); ctx.register_avro("alltypes_plain", avro_file, AvroReadOptions::default())?; - let df = ctx.table("alltypes_plain").unwrap(); // execute the query let df = ctx.sql( "SELECT int_col, double_col, CAST(date_string_col as VARCHAR) \ diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index f5aa7d7874b0..6f0e412fcfb2 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -22,8 +22,10 @@ use arrow::datatypes::SchemaRef; use arrow::error::ArrowError::SchemaError; use arrow::error::Result as ArrowResult; use avro_rs::schema::Schema as AvroSchema; +use avro_rs::schema::SchemaKind; use avro_rs::types::Value; use avro_rs::{AvroResult, Reader as AvroReader}; +use num_traits::NumCast; use std::collections::HashMap; use std::io::Read; use std::sync::Arc; @@ -118,7 +120,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let mut builder = BooleanBuilder::new(rows.len()); for row in rows { if let Some(value) = self.field_lookup(col_name, row) { - if let Value::Boolean(boolean) = value { + if let Some(boolean) = resolve_boolean(&value) { builder.append_value(boolean)? } else { builder.append_null()?; @@ -131,7 +133,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } #[allow(clippy::unnecessary_wraps)] - fn build_primitive_array( + fn build_primitive_array( &self, rows: RecordSlice, col_name: &str, @@ -144,12 +146,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { rows.iter() .map(|row| { self.field_lookup(col_name, row) - .and_then(|value| value.resolve(&AvroSchema::Double).ok()) - .and_then(|v| match v { - Value::Double(f) => Some(f), - _ => None, - }) - .and_then(num_traits::cast::cast) + .and_then(|value| resolve_item::(&value)) }) .collect::>(), )) @@ -275,7 +272,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } else if let Value::Array(n) = value { n.into_iter() .map(|v| { - if let Some(v) = self.as_string(v.clone()) { + if let Some(v) = resolve_string(&v) { Some(v) } else if matches!( v, @@ -292,7 +289,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } else if let Value::Null = value { vec![None] } else if !matches!(value, Value::Record(_)) { - vec![self.as_string(value)] + vec![resolve_string(&value)] } else { return Err(SchemaError(format!( "Only scalars are currently supported in Avro arrays", @@ -362,7 +359,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { self.build_string_dictionary_builder(rows.len())?; for row in rows { if let Some(value) = self.field_lookup(col_name, row) { - if let Some(str_v) = self.as_string(value) { + if let Some(str_v) = resolve_string(&value) { builder.append(str_v).map(drop)? } else { builder.append_null()? @@ -699,7 +696,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { rows.iter() .map(|row| { let maybe_value = self.field_lookup(field.name(), row); - maybe_value.and_then(|value| self.as_string(value)) + maybe_value.and_then(|value| resolve_string(&value)) }) .collect::(), ) @@ -708,7 +705,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { rows.iter() .map(|row| { let maybe_value = self.field_lookup(field.name(), row); - maybe_value.and_then(|value| self.as_bytes(value)) + maybe_value.and_then(|value| resolve_bytes(value)) }) .collect::(), ) @@ -794,16 +791,10 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { if let Value::Array(values) = row { values .iter() - .map(|value| { - let v: Option = - value_as_f64(value).and_then(num_traits::cast::cast); - v - }) + .map(resolve_item::) .collect::>>() - } else if let Some(f) = value_as_f64(row) { - // handle the scalar number case - let v: Option = num_traits::cast::cast(f); - v.map(|v| vec![Some(v)]).unwrap_or_default() + } else if let Some(f) = resolve_item::(row) { + vec![Some(f)] } else { vec![] } @@ -819,45 +810,6 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { .and_then(|i| row.get(*i)) .map(|o| o.1.clone()) } - - fn as_string(&self, v: Value) -> Option { - let v = if let Value::Union(b) = v { *b } else { v }; - match v { - Value::String(s) => Ok(Value::String(s)), - Value::Bytes(bytes) => Ok(Value::String( - String::from_utf8(bytes) - .map_err(avro_rs::Error::ConvertToUtf8) - .ok()?, - )), - other => Err(avro_rs::Error::GetString(other.into())), - } - .ok() - .and_then(|v| match v { - Value::String(s) => Some(s.clone()), - _ => None, - }) - } - - fn as_bytes(&self, v: Value) -> Option> { - let v = if let Value::Union(b) = v { *b } else { v }; - match v { - Value::Bytes(bytes) => Ok(Value::Bytes(bytes)), - Value::String(s) => Ok(Value::Bytes(s.into_bytes())), - Value::Array(items) => Ok(Value::Bytes( - items - .into_iter() - .map(try_u8) - .collect::, _>>() - .ok()?, - )), - other => Err(avro_rs::Error::GetBytes(other.into())), - } - .ok() - .and_then(|v| match v { - Value::Bytes(s) => Some(s.clone()), - _ => None, - }) - } } /// Flattens a list of Avro values, by flattening lists, and treating all other values as @@ -891,12 +843,12 @@ fn flatten_string_values(values: &[Value]) -> Vec> { if let Value::Array(values) = row { values .iter() - .map(value_as_string) + .map(resolve_string) .collect::>>() } else if let Value::Null = row { vec![] } else { - vec![value_as_string(row)] + vec![resolve_string(row)] } }) .collect::>>() @@ -905,38 +857,96 @@ fn flatten_string_values(values: &[Value]) -> Vec> { /// Reads an Avro value as a string, regardless of its type. /// This is useful if the expected datatype is a string, in which case we preserve /// all the values regardless of they type. -#[inline(always)] -fn value_as_string(value: &Value) -> Option { - match value { - Value::Null => None, - Value::String(string) => Some(string.clone()), - _ => None, +fn resolve_string(v: &Value) -> Option { + let v = if let Value::Union(b) = v { b } else { v }; + match v { + Value::String(s) => Ok(s.clone()), + Value::Bytes(bytes) => Ok(String::from_utf8(bytes.to_vec()) + .map_err(avro_rs::Error::ConvertToUtf8) + .ok()?), + other => Err(avro_rs::Error::GetString(other.into())), } + .ok() } -fn value_as_f64(value: &Value) -> Option { - match value { - Value::Int(n) => Ok(Value::Double(f64::from(*n))), - Value::Long(n) => Ok(Value::Double(*n as f64)), - Value::Float(x) => Ok(Value::Double(f64::from(*x))), - Value::Double(x) => Ok(Value::Double(*x)), - other => Err(avro_rs::Error::GetDouble(other.into())), +fn resolve_u8(v: Value) -> AvroResult { + let int = v.resolve(&AvroSchema::Int)?; + if let Value::Int(n) = int { + if n >= 0 && n <= std::convert::From::from(u8::MAX) { + return Ok(n as u8); + } + } + + Err(avro_rs::Error::GetU8(int.into())) +} + +fn resolve_bytes(v: Value) -> Option> { + let v = if let Value::Union(b) = v { *b } else { v }; + match v { + Value::Bytes(bytes) => Ok(Value::Bytes(bytes)), + Value::String(s) => Ok(Value::Bytes(s.into_bytes())), + Value::Array(items) => Ok(Value::Bytes( + items + .into_iter() + .map(resolve_u8) + .collect::, _>>() + .ok()?, + )), + other => Err(avro_rs::Error::GetBytes(other.into())), } .ok() .and_then(|v| match v { - Value::Double(f) => Some(f), + Value::Bytes(s) => Some(s.clone()), _ => None, }) } -fn try_u8(v: Value) -> AvroResult { - println!("{:?}", v); - let int = v.resolve(&AvroSchema::Int)?; - if let Value::Int(n) = int { - if n >= 0 && n <= i32::from(u8::MAX) { - return Ok(n as u8); - } +fn resolve_boolean(value: &Value) -> Option { + let v = if let Value::Union(b) = value { + b + } else { + value + }; + match v { + Value::Boolean(boolean) => Some(*boolean), + _ => None, } +} - Err(avro_rs::Error::GetU8(int.into())) +trait Resolver: ArrowPrimitiveType { + fn resolve(value: &Value) -> Option; +} + +fn resolve_item(value: &Value) -> Option { + T::resolve(value) +} + +impl Resolver for N +where + N: ArrowNumericType, + N::Native: num_traits::cast::NumCast, +{ + fn resolve(value: &Value) -> Option { + let value = if SchemaKind::from(value) == SchemaKind::Union { + // Pull out the Union, and attempt to resolve against it. + let v = match value { + Value::Union(b) => b, + _ => unreachable!(), + }; + v + } else { + value + }; + match value { + Value::Int(i) | Value::TimeMillis(i) | Value::Date(i) => NumCast::from(*i), + Value::Long(l) + | Value::TimeMicros(l) + | Value::TimestampMillis(l) + | Value::TimestampMicros(l) => NumCast::from(*l), + Value::Float(f) => NumCast::from(*f), + Value::Double(f) => NumCast::from(*f), + Value::Duration(_d) => unimplemented!(), // shenanigans type + _ => unreachable!(), + } + } } diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index 9eeb269f96ce..362a51607d5b 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -26,15 +26,16 @@ use std::{ sync::{Arc, Mutex}, }; +use arrow::datatypes::SchemaRef; + +use crate::physical_plan::avro::{AvroExec, AvroReadOptions}; use crate::{ datasource::{Source, TableProvider}, error::{DataFusionError, Result}, physical_plan::{common, ExecutionPlan}, }; -use arrow::datatypes::SchemaRef; use super::datasource::Statistics; -use crate::physical_plan::avro::{AvroExec, AvroReadOptions}; trait SeekRead: Read + Seek {} @@ -97,7 +98,9 @@ impl AvroFile { if let Some(schema) = options.schema { schema } else { - Arc::new(crate::avro::infer_avro_schema_from_reader(&mut reader)?) + Arc::new(crate::avro_to_arrow::infer_avro_schema_from_reader( + &mut reader, + )?) } }; @@ -178,22 +181,14 @@ impl TableProvider for AvroFile { #[cfg(test)] mod tests { - use super::*; - use crate::datasource::avro::AvroFile; - use crate::datasource::TableProvider; use arrow::array::{ BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, - TimestampNanosecondArray, + TimestampMicrosecondArray, }; use arrow::record_batch::RecordBatch; use futures::StreamExt; - fn load_table(name: &str) -> Result> { - let testdata = crate::test_util::arrow_test_data(); - let filename = format!("{}/avro/{}", testdata, name); - let table = AvroFile::try_new(&filename, AvroReadOptions::default())?; - Ok(Arc::new(table)) - } + use super::*; #[tokio::test] async fn read_small_batches() -> Result<()> { @@ -211,10 +206,6 @@ mod tests { .fold(0, |acc, _| async move { acc + 1i32 }) .await; - // test metadata - assert_eq!(table.statistics().num_rows, Some(8)); - assert_eq!(table.statistics().total_byte_size, Some(671)); - Ok(()) } @@ -240,7 +231,7 @@ mod tests { double_col: Float64\n\ date_string_col: Binary\n\ string_col: Binary\n\ - timestamp_col: Timestamp(Nanosecond, None)", + timestamp_col: Timestamp(Microsecond, None)", y ); @@ -309,21 +300,20 @@ mod tests { let table = load_table("alltypes_plain.avro")?; let projection = Some(vec![10]); let batch = get_first_batch(table, &projection).await?; - assert_eq!(1, batch.num_columns()); assert_eq!(8, batch.num_rows()); let array = batch .column(0) .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); let mut values: Vec = vec![]; for i in 0..batch.num_rows() { values.push(array.value(i)); } - assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); + assert_eq!("[1235865600000000, 1235865660000000, 1238544000000000, 1238544060000000, 1233446400000000, 1233446460000000, 1230768000000000, 1230768060000000]", format!("{:?}", values)); Ok(()) } @@ -409,6 +399,13 @@ mod tests { Ok(()) } + fn load_table(name: &str) -> Result> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/{}", testdata, name); + let table = AvroFile::try_new(&filename, AvroReadOptions::default())?; + Ok(Arc::new(table)) + } + async fn get_first_batch( table: Arc, projection: &Option>, diff --git a/datafusion/src/lib.rs b/datafusion/src/lib.rs index 42895564ecd6..2452146ccb93 100644 --- a/datafusion/src/lib.rs +++ b/datafusion/src/lib.rs @@ -212,7 +212,7 @@ extern crate sqlparser; -pub mod avro; +pub mod avro_to_arrow; pub mod catalog; pub mod dataframe; pub mod datasource; diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index d6e18317ecad..70e98d1429f8 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -20,7 +20,7 @@ use async_trait::async_trait; use futures::Stream; use super::{common, source::Source, ExecutionPlan, Partitioning, RecordBatchStream}; -use crate::avro::infer_avro_schema_from_reader; +use crate::avro_to_arrow::infer_avro_schema_from_reader; use crate::error::{DataFusionError, Result}; use arrow::{ datatypes::{Schema, SchemaRef}, @@ -256,7 +256,7 @@ impl ExecutionPlan for AvroExec { &self, partition: usize, ) -> Result { - let mut builder = crate::avro::ReaderBuilder::new() + let mut builder = crate::avro_to_arrow::ReaderBuilder::new() .with_schema(self.schema.clone()) .with_batch_size(self.batch_size); if let Some(proj) = &self.projection { @@ -293,12 +293,12 @@ impl ExecutionPlan for AvroExec { } struct AvroStream<'a, R: Read> { - reader: crate::avro::Reader<'a, R>, + reader: crate::avro_to_arrow::Reader<'a, R>, remain: Option, } impl<'a, R: Read> AvroStream<'a, R> { - fn new(reader: crate::avro::Reader<'a, R>, limit: Option) -> Self { + fn new(reader: crate::avro_to_arrow::Reader<'a, R>, limit: Option) -> Self { Self { reader, remain: limit, @@ -489,7 +489,6 @@ mod tests { assert_eq!(1, batch.num_columns()); assert_eq!(8, batch.num_rows()); - let array = batch .column(0) .as_any() From 01cbb1852c0258dc93e5970c061a147355f1dedf Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Fri, 20 Aug 2021 05:25:48 +0200 Subject: [PATCH 12/58] Test for AvroExec::try_from_path --- datafusion/src/datasource/avro.rs | 2 +- datafusion/src/physical_plan/avro.rs | 283 +++------------------------ 2 files changed, 28 insertions(+), 257 deletions(-) diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index 362a51607d5b..c84712dd0663 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -168,7 +168,7 @@ impl TableProvider for AvroFile { } } Source::Path(p) => { - AvroExec::try_new(p, opts, projection.clone(), batch_size, limit)? + AvroExec::try_from_path(p, opts, projection.clone(), batch_size, limit)? } }; Ok(Arc::new(exec)) diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index 70e98d1429f8..5c3654ddd79e 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -73,7 +73,7 @@ pub struct AvroExec { impl AvroExec { /// Create a new execution plan for reading from a path - pub fn try_new( + pub fn try_from_path( path: &str, options: AvroReadOptions, projection: Option>, @@ -168,7 +168,7 @@ impl AvroExec { &self.file_extension } - /// Get the schema of the CSV file + /// Get the schema of the avro file pub fn file_schema(&self) -> SchemaRef { self.schema.clone() } @@ -355,270 +355,41 @@ impl RecordBatchStream for AvroStream<'_, R> { #[cfg(test)] mod tests { use super::*; - use crate::datasource::avro::AvroFile; - use crate::datasource::TableProvider; - use crate::logical_plan::combine_filters; - use arrow::array::{ - BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, - TimestampNanosecondArray, - }; - use arrow::record_batch::RecordBatch; use futures::StreamExt; - fn load_table(name: &str) -> Result> { - let testdata = crate::test_util::arrow_test_data(); - let filename = format!("{}/avro/{}", testdata, name); - let table = AvroFile::try_new(&filename, AvroReadOptions::default())?; - Ok(Arc::new(table)) - } - - #[tokio::test] - async fn read_small_batches() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = None; - let exec = table.scan(&projection, 2, &[], None)?; - let stream = exec.execute(0).await?; - - let _ = stream - .map(|batch| { - let batch = batch.unwrap(); - assert_eq!(11, batch.num_columns()); - assert_eq!(2, batch.num_rows()); - }) - .fold(0, |acc, _| async move { acc + 1i32 }) - .await; - - // test metadata - assert_eq!(table.statistics().num_rows, Some(8)); - assert_eq!(table.statistics().total_byte_size, Some(671)); - - Ok(()) - } - - #[tokio::test] - async fn read_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - - let x: Vec = table - .schema() - .fields() - .iter() - .map(|f| format!("{}: {:?}", f.name(), f.data_type())) - .collect(); - let y = x.join("\n"); - assert_eq!( - "id: Int32\n\ - bool_col: Boolean\n\ - tinyint_col: Int32\n\ - smallint_col: Int32\n\ - int_col: Int32\n\ - bigint_col: Int64\n\ - float_col: Float32\n\ - double_col: Float64\n\ - date_string_col: Binary\n\ - string_col: Binary\n\ - timestamp_col: Timestamp(Microsecond, None)", - y - ); - - let projection = None; - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(11, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - Ok(()) - } - - #[tokio::test] - async fn read_bool_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![1]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[true, false, true, false, true, false, true, false]", - format!("{:?}", values) - ); - - Ok(()) - } - - #[tokio::test] - async fn read_i32_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![0]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); - - Ok(()) - } - #[tokio::test] - async fn read_i96_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![10]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); - - Ok(()) - } - - #[tokio::test] - async fn read_f32_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![6]); - let batch = get_first_batch(table, &projection).await?; + async fn test() -> Result<()> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/alltypes_plain.avro", testdata); + let parquet_exec = AvroExec::try_from_path( + &filename, + AvroReadOptions::default(), + Some(vec![0, 1, 2]), + 1024, + None, + )?; + assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); + + let mut results = parquet_exec.execute(0).await?; + let batch = results.next().await.unwrap()?; - assert_eq!(1, batch.num_columns()); assert_eq!(8, batch.num_rows()); + assert_eq!(3, batch.num_columns()); - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", - format!("{:?}", values) - ); + let schema = batch.schema(); + let field_names: Vec<&str> = + schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!(vec!["id", "bool_col", "tinyint_col"], field_names); - Ok(()) - } + let batch = results.next().await; + assert!(batch.is_none()); - #[tokio::test] - async fn read_f64_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![7]); - let batch = get_first_batch(table, &projection).await?; + let batch = results.next().await; + assert!(batch.is_none()); - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec = vec![]; - for i in 0..batch.num_rows() { - values.push(array.value(i)); - } - - assert_eq!( - "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", - format!("{:?}", values) - ); + let batch = results.next().await; + assert!(batch.is_none()); Ok(()) } - - #[tokio::test] - async fn read_binary_alltypes_plain_parquet() -> Result<()> { - let table = load_table("alltypes_plain.avro")?; - let projection = Some(vec![9]); - let batch = get_first_batch(table, &projection).await?; - - assert_eq!(1, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - - let array = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - let mut values: Vec<&str> = vec![]; - for i in 0..batch.num_rows() { - values.push(std::str::from_utf8(array.value(i)).unwrap()); - } - - assert_eq!( - "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", - format!("{:?}", values) - ); - - Ok(()) - } - - async fn get_first_batch( - table: Arc, - projection: &Option>, - ) -> Result { - let exec = table.scan(projection, 1024, &[], None)?; - let mut it = exec.execute(0).await?; - it.next() - .await - .expect("should have received at least one batch") - .map_err(|e| e.into()) - } - - #[test] - fn combine_zero_filters() { - let result = combine_filters(&[]); - assert_eq!(result, None); - } - - #[test] - fn combine_one_filter() { - use crate::logical_plan::{binary_expr, col, lit, Operator}; - let filter = binary_expr(col("c1"), Operator::Lt, lit(1)); - let result = combine_filters(&[filter.clone()]); - assert_eq!(result, Some(filter)); - } - - #[test] - fn combine_multiple_filters() { - use crate::logical_plan::{and, binary_expr, col, lit, Operator}; - let filter1 = binary_expr(col("c1"), Operator::Lt, lit(1)); - let filter2 = binary_expr(col("c2"), Operator::Lt, lit(2)); - let filter3 = binary_expr(col("c3"), Operator::Lt, lit(3)); - let result = - combine_filters(&[filter1.clone(), filter2.clone(), filter3.clone()]); - assert_eq!(result, Some(and(and(filter1, filter2), filter3))); - } } From 537577aa1e479bc035887a3d7713ac00aeb97e72 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Fri, 20 Aug 2021 06:05:30 +0200 Subject: [PATCH 13/58] external table avro test --- datafusion/src/sql/parser.rs | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/datafusion/src/sql/parser.rs b/datafusion/src/sql/parser.rs index 527c3cddd9b3..8e70c7c15883 100644 --- a/datafusion/src/sql/parser.rs +++ b/datafusion/src/sql/parser.rs @@ -393,6 +393,17 @@ mod tests { }); expect_parse_ok(sql, expected)?; + // positive case: it is ok for parquet files not to have columns specified + let sql = "CREATE EXTERNAL TABLE t STORED AS AVRO LOCATION 'foo.avro'"; + let expected = Statement::CreateExternalTable(CreateExternalTable { + name: "t".into(), + columns: vec![], + file_type: FileType::Avro, + has_header: false, + location: "foo.avro".into(), + }); + expect_parse_ok(sql, expected)?; + // Error cases: Invalid type let sql = "CREATE EXTERNAL TABLE t(c1 int) STORED AS UNKNOWN_TYPE LOCATION 'foo.csv'"; From 75566d180bf888cd89b041601a8f1cf82a766ec7 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Fri, 20 Aug 2021 06:34:52 +0200 Subject: [PATCH 14/58] Basic schema conversion tests --- datafusion/src/avro_to_arrow/mod.rs | 148 +++++++++++++++++++++++++++- 1 file changed, 145 insertions(+), 3 deletions(-) diff --git a/datafusion/src/avro_to_arrow/mod.rs b/datafusion/src/avro_to_arrow/mod.rs index 79b795123e88..15a8b41474ff 100644 --- a/datafusion/src/avro_to_arrow/mod.rs +++ b/datafusion/src/avro_to_arrow/mod.rs @@ -279,7 +279,7 @@ fn external_props(schema: &AvroSchema) -> BTreeMap { } => { let aliases: Vec = aliases .into_iter() - .map(|alias| fullname(alias, namespace.as_ref(), None)) + .map(|alias| aliased(alias, namespace.as_deref(), None)) .collect(); props.insert("aliases".to_string(), format!("[{}]", aliases.join(","))); } @@ -299,9 +299,9 @@ fn get_metadata( } /// Returns the fully qualified name for a field -pub fn fullname( +pub fn aliased( name: &str, - namespace: Option<&String>, + namespace: Option<&str>, default_namespace: Option<&str>, ) -> String { if name.contains('.') { @@ -315,3 +315,145 @@ pub fn fullname( } } } + +#[cfg(test)] +mod test { + use crate::arrow::datatypes::DataType::{Binary, Float32, Float64, Timestamp, Utf8}; + use crate::arrow::datatypes::TimeUnit::Microsecond; + use crate::arrow::datatypes::{Field, Schema}; + use crate::avro_to_arrow::{aliased, external_props, to_arrow_schema}; + use arrow::datatypes::DataType::{Boolean, Int32, Int64}; + use avro_rs::schema::Name; + use avro_rs::Schema as AvroSchema; + + #[test] + fn test_alias() { + assert_eq!(aliased("foo.bar", None, None), "foo.bar"); + assert_eq!(aliased("bar", Some("foo"), None), "foo.bar"); + assert_eq!(aliased("bar", Some("foo"), Some("cat")), "foo.bar"); + assert_eq!(aliased("bar", None, Some("cat")), "cat.bar"); + } + + #[test] + fn test_external_props() { + let record_schema = AvroSchema::Record { + name: Name { + name: "record".to_string(), + namespace: None, + aliases: Some(vec!["fooalias".to_string(), "baralias".to_string()]), + }, + doc: Some("record documentation".to_string()), + fields: vec![], + lookup: Default::default(), + }; + let props = external_props(&record_schema); + assert_eq!(props.get("doc"), Some(&"record documentation".to_string())); + assert_eq!( + props.get("aliases"), + Some(&"[fooalias,baralias]".to_string()) + ); + let enum_schema = AvroSchema::Enum { + name: Name { + name: "enum".to_string(), + namespace: None, + aliases: Some(vec!["fooenum".to_string(), "barenum".to_string()]), + }, + doc: Some("enum documentation".to_string()), + symbols: vec![], + }; + let props = external_props(&enum_schema); + assert_eq!(props.get("doc"), Some(&"enum documentation".to_string())); + assert_eq!(props.get("aliases"), Some(&"[fooenum,barenum]".to_string())); + let fixed_schema = AvroSchema::Fixed { + name: Name { + name: "fixed".to_string(), + namespace: None, + aliases: Some(vec!["foofixed".to_string(), "barfixed".to_string()]), + }, + size: 1, + }; + let props = external_props(&fixed_schema); + assert_eq!( + props.get("aliases"), + Some(&"[foofixed,barfixed]".to_string()) + ); + } + + #[test] + fn test_invalid_avro_schema() {} + + #[test] + fn test_plain_types_schema() { + let schema = AvroSchema::parse_str( + r#" + { + "type" : "record", + "name" : "topLevelRecord", + "fields" : [ { + "name" : "id", + "type" : [ "int", "null" ] + }, { + "name" : "bool_col", + "type" : [ "boolean", "null" ] + }, { + "name" : "tinyint_col", + "type" : [ "int", "null" ] + }, { + "name" : "smallint_col", + "type" : [ "int", "null" ] + }, { + "name" : "int_col", + "type" : [ "int", "null" ] + }, { + "name" : "bigint_col", + "type" : [ "long", "null" ] + }, { + "name" : "float_col", + "type" : [ "float", "null" ] + }, { + "name" : "double_col", + "type" : [ "double", "null" ] + }, { + "name" : "date_string_col", + "type" : [ "bytes", "null" ] + }, { + "name" : "string_col", + "type" : [ "bytes", "null" ] + }, { + "name" : "timestamp_col", + "type" : [ { + "type" : "long", + "logicalType" : "timestamp-micros" + }, "null" ] + } ] + }"#, + ); + assert!(schema.is_ok(), "{:?}", schema); + let arrow_schema = to_arrow_schema(&schema.unwrap()); + assert!(arrow_schema.is_ok(), "{:?}", arrow_schema); + let expected = Schema::new(vec![ + Field::new("id", Int32, true), + Field::new("bool_col", Boolean, true), + Field::new("tinyint_col", Int32, true), + Field::new("smallint_col", Int32, true), + Field::new("int_col", Int32, true), + Field::new("bigint_col", Int64, true), + Field::new("float_col", Float32, true), + Field::new("double_col", Float64, true), + Field::new("date_string_col", Binary, true), + Field::new("string_col", Binary, true), + Field::new("timestamp_col", Timestamp(Microsecond, None), true), + ]); + assert_eq!(arrow_schema.unwrap(), expected); + } + + #[test] + fn test_non_record_schema() { + let arrow_schema = to_arrow_schema(&AvroSchema::String); + assert!(arrow_schema.is_ok(), "{:?}", arrow_schema); + assert_eq!( + arrow_schema.unwrap(), + Schema::new(vec![Field::new("", Utf8, false)]) + ); + } +} From 7fdf2330e0d68e89b3710d70d27453d6bcd895be Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Fri, 20 Aug 2021 19:20:47 +0200 Subject: [PATCH 15/58] Complete test for avro_to_arrow_reader on alltypes_dictionnary --- datafusion/src/avro_to_arrow/reader.rs | 123 ++++++++++++++++++++++++- testing | 2 +- 2 files changed, 119 insertions(+), 6 deletions(-) diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index 21994778dc6b..1d71383b1d9e 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -4,7 +4,7 @@ use crate::avro_to_arrow::arrow_array_reader::AvroArrowArrayReader; use crate::error::Result; use arrow::error::Result as ArrowResult; use avro_rs::Reader as AvroReader; -use std::io::{BufReader, Read, Seek}; +use std::io::{Read, Seek}; use std::sync::Arc; /// Avro file reader builder @@ -86,18 +86,19 @@ impl ReaderBuilder { } /// Create a new `Reader` from the `ReaderBuilder` - pub fn build<'a, R>(self, source: R) -> Result>> + pub fn build<'a, R>(self, source: R) -> Result> where R: Read + Seek, { - let mut buf_reader = BufReader::new(source); + let mut source = source; // check if schema should be inferred let schema = match self.schema { Some(schema) => schema, - None => Arc::new(infer_avro_schema_from_reader(&mut buf_reader)?), + None => Arc::new(infer_avro_schema_from_reader(&mut source)?), }; - Reader::try_new(buf_reader, schema, self.batch_size, self.projection) + source.rewind()?; + Reader::try_new(source, schema, self.batch_size, self.projection) } } @@ -157,3 +158,115 @@ pub fn infer_avro_schema_from_reader(reader: &mut R) -> Result Reader { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/{}", testdata, name); + let builder = ReaderBuilder::new().infer_schema().with_batch_size(64); + builder.build(File::open(filename).unwrap()).unwrap() + } + + fn get_col<'a, T: 'static>( + batch: &'a RecordBatch, + col: (usize, &Field), + ) -> Option<&'a T> { + batch.column(col.0).as_any().downcast_ref::() + } + + #[test] + fn test_json_basic() { + let mut reader = build_reader("alltypes_dictionary.avro"); + let batch = reader.next().unwrap().unwrap(); + + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + + let schema = reader.schema(); + let batch_schema = batch.schema(); + assert_eq!(schema, batch_schema); + + let id = schema.column_with_name("id").unwrap(); + assert_eq!(0, id.0); + assert_eq!(&DataType::Int32, id.1.data_type()); + let col = get_col::(&batch, id).unwrap(); + assert_eq!(0, col.value(0)); + assert_eq!(1, col.value(1)); + let bool_col = schema.column_with_name("bool_col").unwrap(); + assert_eq!(1, bool_col.0); + assert_eq!(&DataType::Boolean, bool_col.1.data_type()); + let col = get_col::(&batch, bool_col).unwrap(); + assert_eq!(true, col.value(0)); + assert_eq!(false, col.value(1)); + let tinyint_col = schema.column_with_name("tinyint_col").unwrap(); + assert_eq!(2, tinyint_col.0); + assert_eq!(&DataType::Int32, tinyint_col.1.data_type()); + let col = get_col::(&batch, tinyint_col).unwrap(); + assert_eq!(0, col.value(0)); + assert_eq!(1, col.value(1)); + let smallint_col = schema.column_with_name("smallint_col").unwrap(); + assert_eq!(3, smallint_col.0); + assert_eq!(&DataType::Int32, smallint_col.1.data_type()); + let col = get_col::(&batch, smallint_col).unwrap(); + assert_eq!(0, col.value(0)); + assert_eq!(1, col.value(1)); + let int_col = schema.column_with_name("int_col").unwrap(); + assert_eq!(4, int_col.0); + let col = get_col::(&batch, int_col).unwrap(); + assert_eq!(0, col.value(0)); + assert_eq!(1, col.value(1)); + assert_eq!(&DataType::Int32, int_col.1.data_type()); + let col = get_col::(&batch, int_col).unwrap(); + assert_eq!(0, col.value(0)); + assert_eq!(1, col.value(1)); + let bigint_col = schema.column_with_name("bigint_col").unwrap(); + assert_eq!(5, bigint_col.0); + let col = get_col::(&batch, bigint_col).unwrap(); + assert_eq!(0, col.value(0)); + assert_eq!(10, col.value(1)); + assert_eq!(&DataType::Int64, bigint_col.1.data_type()); + let float_col = schema.column_with_name("float_col").unwrap(); + assert_eq!(6, float_col.0); + let col = get_col::(&batch, float_col).unwrap(); + assert_eq!(0.0, col.value(0)); + assert_eq!(1.1, col.value(1)); + assert_eq!(&DataType::Float32, float_col.1.data_type()); + let col = get_col::(&batch, float_col).unwrap(); + assert_eq!(0.0, col.value(0)); + assert_eq!(1.1, col.value(1)); + let double_col = schema.column_with_name("double_col").unwrap(); + assert_eq!(7, double_col.0); + assert_eq!(&DataType::Float64, double_col.1.data_type()); + let col = get_col::(&batch, double_col).unwrap(); + assert_eq!(0.0, col.value(0)); + assert_eq!(10.1, col.value(1)); + let date_string_col = schema.column_with_name("date_string_col").unwrap(); + assert_eq!(8, date_string_col.0); + assert_eq!(&DataType::Binary, date_string_col.1.data_type()); + let col = get_col::(&batch, date_string_col).unwrap(); + assert_eq!("01/01/09".as_bytes(), col.value(0)); + assert_eq!("01/01/09".as_bytes(), col.value(1)); + let string_col = schema.column_with_name("string_col").unwrap(); + assert_eq!(9, string_col.0); + assert_eq!(&DataType::Binary, string_col.1.data_type()); + let col = get_col::(&batch, string_col).unwrap(); + assert_eq!("0".as_bytes(), col.value(0)); + assert_eq!("1".as_bytes(), col.value(1)); + let timestamp_col = schema.column_with_name("timestamp_col").unwrap(); + assert_eq!(10, timestamp_col.0); + assert_eq!( + &DataType::Timestamp(TimeUnit::Microsecond, None), + timestamp_col.1.data_type() + ); + let col = get_col::(&batch, timestamp_col).unwrap(); + assert_eq!(1230768000000000, col.value(0)); + assert_eq!(1230768060000000, col.value(1)); + } +} diff --git a/testing b/testing index 22d990e195f9..a8f7be380531 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 22d990e195f9bfda0311c29dfccf9f1fa5cecae9 +Subproject commit a8f7be380531758eb7962542a5eb020d8795aa20 From 697e3a96370465f699ec2ce70a88f7e2f7ae6f75 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Sun, 22 Aug 2021 11:07:10 +0200 Subject: [PATCH 16/58] fix_stable: .rewind is 'unstable' --- datafusion/src/avro_to_arrow/reader.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index 1d71383b1d9e..cce022048525 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -4,7 +4,7 @@ use crate::avro_to_arrow::arrow_array_reader::AvroArrowArrayReader; use crate::error::Result; use arrow::error::Result as ArrowResult; use avro_rs::Reader as AvroReader; -use std::io::{Read, Seek}; +use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; /// Avro file reader builder @@ -97,7 +97,7 @@ impl ReaderBuilder { Some(schema) => schema, None => Arc::new(infer_avro_schema_from_reader(&mut source)?), }; - source.rewind()?; + source.seek(SeekFrom::Start(0))?; Reader::try_new(source, schema, self.batch_size, self.projection) } } From e1d6df898f1dc4bd5f116594de32f6e628c681bc Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 14:22:34 +0200 Subject: [PATCH 17/58] Fix license files and remove the unused avro-converter crate --- avro-converter/Cargo.toml | 24 ---------- avro-converter/src/main.rs | 45 ------------------- .../src/avro_to_arrow/arrow_array_reader.rs | 19 ++++++++ datafusion/src/avro_to_arrow/reader.rs | 17 +++++++ 4 files changed, 36 insertions(+), 69 deletions(-) delete mode 100644 avro-converter/Cargo.toml delete mode 100644 avro-converter/src/main.rs diff --git a/avro-converter/Cargo.toml b/avro-converter/Cargo.toml deleted file mode 100644 index e128bda12230..000000000000 --- a/avro-converter/Cargo.toml +++ /dev/null @@ -1,24 +0,0 @@ -[package] -name = "avro-converter" -version = "4.0.0-SNAPSHOT" -edition = "2018" -description = "DataFusion avro to parquet converter" -homepage = "https://github.com/apache/arrow-datafusion" -repository = "https://github.com/apache/arrow-datafusion" -authors = ["Apache Arrow "] -license = "Apache-2.0" -keywords = [ "arrow", "avro", "parquet" ] - -[[bin]] -path = "src/main.rs" -name = "avro-converter" - -[dependencies] -arrow-flight = { version = "5.0" } -datafusion = { path = "../datafusion" } -prost = "0.8" -tonic = "0.5" -tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync"] } -futures = "0.3" -num_cpus = "1.13.0" -structopt = { version = "0.3" } diff --git a/avro-converter/src/main.rs b/avro-converter/src/main.rs deleted file mode 100644 index b99e6bf3d973..000000000000 --- a/avro-converter/src/main.rs +++ /dev/null @@ -1,45 +0,0 @@ -#[macro_use] -extern crate structopt; - -use datafusion::arrow::util::pretty; - -use crate::structopt::StructOpt; -use datafusion::error::Result; -use datafusion::prelude::*; -use std::path::PathBuf; - -fn parse_csv(src: &str) -> Vec { - src.split(',').map(|s| s.to_string()).collect() -} - -#[derive(StructOpt, Debug)] -#[structopt(name = "basic")] -struct ArrowConverterOptions { - #[structopt(short, long, parse(from_os_str))] - input: PathBuf, - #[structopt(short, long, parse(from_os_str))] - output: PathBuf, - #[structopt(short, long)] - partitions: Option, -} - -#[tokio::main] -async fn main() -> Result<()> { - // create local execution context - let mut ctx = ExecutionContext::new(); - - let opts: ArrowConverterOptions = ArrowConverterOptions::from_args(); - - // define the query using the DataFrame trait - let df = ctx - .read_parquet(opts.input.to_str().unwrap())? - .select_columns(&["id", "bool_col", "timestamp_col"])?; - - // execute the query - let results = df.collect().await?; - - // print the results - pretty::print_batches(&results)?; - - Ok(()) -} diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index 6f0e412fcfb2..c6fb6f0d47d1 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -1,3 +1,22 @@ +// 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. + +//! Avro to Arrow array readers + use crate::arrow::array::{ make_array, Array, ArrayBuilder, ArrayData, ArrayDataBuilder, ArrayRef, BooleanBuilder, LargeStringArray, ListBuilder, NullArray, OffsetSizeTrait, diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index cce022048525..2578f736edc9 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -1,3 +1,20 @@ +// 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. + use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::arrow::record_batch::RecordBatch; use crate::avro_to_arrow::arrow_array_reader::AvroArrowArrayReader; From 66a59015d5caa9d16669e38696c759bb749dd228 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 14:49:56 +0200 Subject: [PATCH 18/58] fix example test in avro_to_arrow --- datafusion/src/avro_to_arrow/reader.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index 2578f736edc9..260f8c469b63 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -62,11 +62,11 @@ impl ReaderBuilder { /// /// use std::fs::File; /// - /// fn example() -> avro_rs::Reader { + /// fn example() -> crate::datafusion::avro_to_arrow::Reader<'static, File> { /// let file = File::open("test/data/basic.avro").unwrap(); /// /// // create a builder, inferring the schema with the first 100 records - /// let builder = crate::datafusion::avro_to_arrow::ReaderBuilder::new().infer_schema(Some(100)); + /// let builder = crate::datafusion::avro_to_arrow::ReaderBuilder::new().infer_schema().with_batch_size(100); /// /// let reader = builder.build::(file).unwrap(); /// From 9ea942ca561ada7c83a9af29b271e5dea4721bbb Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 14:50:05 +0200 Subject: [PATCH 19/58] add avro_sql test to default workflow --- .github/workflows/rust.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index a75af64f20f5..a940e6dabf9b 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -112,6 +112,7 @@ jobs: cargo test --no-default-features cargo run --example csv_sql cargo run --example parquet_sql + cargo run --example avro_sql env: CARGO_HOME: "/github/home/.cargo" CARGO_TARGET_DIR: "/github/home/target" From 84ee28acd8ea6576e16cb60f7284758e40cdfe1c Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 15:08:07 +0200 Subject: [PATCH 20/58] Adress clippies --- .../src/avro_to_arrow/arrow_array_reader.rs | 42 ++++++++----------- datafusion/src/avro_to_arrow/mod.rs | 14 +++---- datafusion/src/avro_to_arrow/reader.rs | 4 +- datafusion/src/datasource/avro.rs | 4 +- datafusion/src/logical_plan/expr.rs | 16 +++---- .../src/physical_plan/datetime_expressions.rs | 15 ++++--- .../src/physical_plan/string_expressions.rs | 1 + 7 files changed, 42 insertions(+), 54 deletions(-) diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index c6fb6f0d47d1..8306442b0c13 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -124,11 +124,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { .collect() }; let projected_schema = Arc::new(Schema::new(projected_fields)); - arrays.and_then(|arr| { - RecordBatch::try_new(projected_schema, arr) - .map(Some) - .map_err(|e| e.into()) - }) + arrays.and_then(|arr| RecordBatch::try_new(projected_schema, arr).map(Some)) } fn build_boolean_array( @@ -291,18 +287,15 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } else if let Value::Array(n) = value { n.into_iter() .map(|v| { - if let Some(v) = resolve_string(&v) { - Some(v) - } else if matches!( - v, - Value::Array(_) | Value::Record(_) | Value::Null - ) { - // implicitly drop nested values - // TODO support deep-nesting - None - } else { - None - } + resolve_string(&v) + // else if matches!( + // v, + // Value::Array(_) | Value::Record(_) | Value::Null + // ) { + // // implicitly drop nested values + // // TODO support deep-nesting + // None + // } }) .collect() } else if let Value::Null = value { @@ -310,9 +303,9 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } else if !matches!(value, Value::Record(_)) { vec![resolve_string(&value)] } else { - return Err(SchemaError(format!( - "Only scalars are currently supported in Avro arrays", - ))); + return Err(SchemaError( + "Only scalars are currently supported in Avro arrays".to_string(), + )); }; // TODO: ARROW-10335: APIs of dictionary arrays and others are different. Unify @@ -724,7 +717,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { rows.iter() .map(|row| { let maybe_value = self.field_lookup(field.name(), row); - maybe_value.and_then(|value| resolve_bytes(value)) + maybe_value.and_then(resolve_bytes) }) .collect::(), ) @@ -915,7 +908,7 @@ fn resolve_bytes(v: Value) -> Option> { } .ok() .and_then(|v| match v { - Value::Bytes(s) => Some(s.clone()), + Value::Bytes(s) => Some(s), _ => None, }) } @@ -948,11 +941,10 @@ where fn resolve(value: &Value) -> Option { let value = if SchemaKind::from(value) == SchemaKind::Union { // Pull out the Union, and attempt to resolve against it. - let v = match value { + match value { Value::Union(b) => b, _ => unreachable!(), - }; - v + } } else { value }; diff --git a/datafusion/src/avro_to_arrow/mod.rs b/datafusion/src/avro_to_arrow/mod.rs index 15a8b41474ff..6ccb8ba2c980 100644 --- a/datafusion/src/avro_to_arrow/mod.rs +++ b/datafusion/src/avro_to_arrow/mod.rs @@ -97,7 +97,7 @@ fn schema_to_field_with_props( .iter() .find(|&schema| !matches!(schema, AvroSchema::Null)) { - schema_to_field_with_props(&schema, None, has_nullable, None)? + schema_to_field_with_props(schema, None, has_nullable, None)? .data_type() .clone() } else { @@ -107,15 +107,15 @@ fn schema_to_field_with_props( } } else { let fields = sub_schemas - .into_iter() - .map(|s| schema_to_field_with_props(&s, None, has_nullable, None)) + .iter() + .map(|s| schema_to_field_with_props(s, None, has_nullable, None)) .collect::>>()?; DataType::Union(fields) } } AvroSchema::Record { name, fields, .. } => { let fields: Result> = fields - .into_iter() + .iter() .map(|field| { let mut props = BTreeMap::new(); if let Some(doc) = &field.doc { @@ -278,7 +278,7 @@ fn external_props(schema: &AvroSchema) -> BTreeMap { .. } => { let aliases: Vec = aliases - .into_iter() + .iter() .map(|alias| aliased(alias, namespace.as_deref(), None)) .collect(); props.insert("aliases".to_string(), format!("[{}]", aliases.join(","))); @@ -295,7 +295,7 @@ fn get_metadata( ) -> BTreeMap { let mut metadata: BTreeMap = Default::default(); metadata.extend(props); - return metadata; + metadata } /// Returns the fully qualified name for a field @@ -307,7 +307,7 @@ pub fn aliased( if name.contains('.') { name.to_string() } else { - let namespace = namespace.as_ref().map(|s| s.as_ref()).or(default_namespace); + let namespace = namespace.as_ref().copied().or(default_namespace); match namespace { Some(ref namespace) => format!("{}.{}", namespace, name), diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index 260f8c469b63..f21e88d1e45d 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -220,8 +220,8 @@ mod tests { assert_eq!(1, bool_col.0); assert_eq!(&DataType::Boolean, bool_col.1.data_type()); let col = get_col::(&batch, bool_col).unwrap(); - assert_eq!(true, col.value(0)); - assert_eq!(false, col.value(1)); + assert!(col.value(0)); + assert!(!col.value(1)); let tinyint_col = schema.column_with_name("tinyint_col").unwrap(); assert_eq!(2, tinyint_col.0); assert_eq!(&DataType::Int32, tinyint_col.1.data_type()); diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index c84712dd0663..a908ea644282 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -53,7 +53,7 @@ impl AvroFile { /// Attempt to initialize a `AvroFile` from a path. The schema can be inferred automatically. pub fn try_new(path: &str, options: AvroReadOptions) -> Result { let schema = if let Some(schema) = options.schema { - schema.clone() + schema } else { let filenames = common::build_checked_file_list(path, options.file_extension)?; @@ -74,7 +74,7 @@ impl AvroFile { options: AvroReadOptions, ) -> Result { let schema = match options.schema { - Some(s) => s.clone(), + Some(s) => s, None => { return Err(DataFusionError::Execution( "Schema must be provided to CsvRead".to_string(), diff --git a/datafusion/src/logical_plan/expr.rs b/datafusion/src/logical_plan/expr.rs index 5e2feb33be0f..5d69e609d979 100644 --- a/datafusion/src/logical_plan/expr.rs +++ b/datafusion/src/logical_plan/expr.rs @@ -1951,17 +1951,13 @@ mod tests { impl ExprRewriter for FooBarRewriter { fn mutate(&mut self, expr: Expr) -> Result { match expr { - Expr::Literal(scalar) => { - if let ScalarValue::Utf8(Some(utf8_val)) = scalar { - let utf8_val = if utf8_val == "foo" { - "bar".to_string() - } else { - utf8_val - }; - Ok(lit(utf8_val)) + Expr::Literal(ScalarValue::Utf8(Some(utf8_val))) => { + let utf8_val = if utf8_val == "foo" { + "bar".to_string() } else { - Ok(Expr::Literal(scalar)) - } + utf8_val + }; + Ok(lit(utf8_val)) } // otherwise, return the expression unchanged expr => Ok(expr), diff --git a/datafusion/src/physical_plan/datetime_expressions.rs b/datafusion/src/physical_plan/datetime_expressions.rs index 39ae70d1b5d0..a776c42f3e9d 100644 --- a/datafusion/src/physical_plan/datetime_expressions.rs +++ b/datafusion/src/physical_plan/datetime_expressions.rs @@ -236,14 +236,8 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { let f = |x: Option| x.map(|x| date_trunc_single(granularity, x)).transpose(); Ok(match array { - ColumnarValue::Scalar(scalar) => { - if let ScalarValue::TimestampNanosecond(v) = scalar { - ColumnarValue::Scalar(ScalarValue::TimestampNanosecond((f)(*v)?)) - } else { - return Err(DataFusionError::Execution( - "array of `date_trunc` must be non-null scalar Utf8".to_string(), - )); - } + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(v)) => { + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond((f)(*v)?)) } ColumnarValue::Array(array) => { let array = array @@ -257,6 +251,11 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { ColumnarValue::Array(Arc::new(array)) } + _ => { + return Err(DataFusionError::Execution( + "array of `date_trunc` must be non-null scalar Utf8".to_string(), + )); + } }) } diff --git a/datafusion/src/physical_plan/string_expressions.rs b/datafusion/src/physical_plan/string_expressions.rs index 09e19c4dfa47..7cbebcec7eb3 100644 --- a/datafusion/src/physical_plan/string_expressions.rs +++ b/datafusion/src/physical_plan/string_expressions.rs @@ -290,6 +290,7 @@ pub fn concat(args: &[ColumnarValue]) -> Result { .map(|index| { let mut owned_string: String = "".to_owned(); for arg in args { + #[allow(clippy::collapsible_match)] match arg { ColumnarValue::Scalar(ScalarValue::Utf8(maybe_value)) => { if let Some(value) = maybe_value { From 9edac57869651c0438502098ba9b1d21b59b506a Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 15:25:40 +0200 Subject: [PATCH 21/58] Enable avro as a valid datasource for client execution --- ballista/rust/client/src/context.rs | 40 ++++++++++++++++++++++++++ datafusion/src/execution/context.rs | 13 +++++++++ datafusion/src/logical_plan/builder.rs | 23 +++++++++++++++ 3 files changed, 76 insertions(+) diff --git a/ballista/rust/client/src/context.rs b/ballista/rust/client/src/context.rs index ee2f65699309..3671f349d45b 100644 --- a/ballista/rust/client/src/context.rs +++ b/ballista/rust/client/src/context.rs @@ -32,6 +32,7 @@ use datafusion::dataframe::DataFrame; use datafusion::error::{DataFusionError, Result}; use datafusion::execution::dataframe_impl::DataFrameImpl; use datafusion::logical_plan::LogicalPlan; +use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::physical_plan::csv::CsvReadOptions; use datafusion::sql::parser::FileType; @@ -125,6 +126,30 @@ impl BallistaContext { }) } + /// Create a DataFrame representing an Avro table scan + + pub fn read_avro( + &self, + path: &str, + options: AvroReadOptions, + ) -> Result> { + // convert to absolute path because the executor likely has a different working directory + let path = PathBuf::from(path); + let path = fs::canonicalize(&path)?; + + // use local DataFusion context for now but later this might call the scheduler + let mut ctx = { + let guard = self.state.lock().unwrap(); + create_df_ctx_with_ballista_query_planner( + &guard.scheduler_host, + guard.scheduler_port, + guard.config(), + ) + }; + let df = ctx.read_avro(path.to_str().unwrap(), options)?; + Ok(df) + } + /// Create a DataFrame representing a Parquet table scan pub fn read_parquet(&self, path: &str) -> Result> { @@ -193,6 +218,17 @@ impl BallistaContext { self.register_table(name, df.as_ref()) } + pub fn register_avro( + &self, + name: &str, + path: &str, + options: AvroReadOptions, + ) -> Result<()> { + let df = self.read_avro(path, options)?; + self.register_table(name, df.as_ref())?; + Ok(()) + } + /// Create a DataFrame from a SQL statement pub fn sql(&self, sql: &str) -> Result> { let mut ctx = { @@ -240,6 +276,10 @@ impl BallistaContext { self.register_parquet(name, location)?; Ok(Arc::new(DataFrameImpl::new(ctx.state, &plan))) } + FileType::Avro => { + self.register_avro(name, location, AvroReadOptions::default())?; + Ok(Arc::new(DataFrameImpl::new(ctx.state, &plan))) + } _ => Err(DataFusionError::NotImplemented(format!( "Unsupported file type {:?}.", file_type diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index f9c4b5a0f24f..5327c583cf7a 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -278,6 +278,19 @@ impl ExecutionContext { .insert(f.name.clone(), Arc::new(f)); } + /// Creates a DataFrame for reading an Avro data source. + + pub fn read_avro( + &mut self, + filename: impl Into, + options: AvroReadOptions, + ) -> Result> { + Ok(Arc::new(DataFrameImpl::new( + self.state.clone(), + &LogicalPlanBuilder::scan_avro(filename, options, None)?.build()?, + ))) + } + /// Creates a DataFrame for reading a CSV data source. pub fn read_csv( &mut self, diff --git a/datafusion/src/logical_plan/builder.rs b/datafusion/src/logical_plan/builder.rs index f31dd3732883..5555939d74a8 100644 --- a/datafusion/src/logical_plan/builder.rs +++ b/datafusion/src/logical_plan/builder.rs @@ -36,10 +36,12 @@ use crate::{ use super::dfschema::ToDFSchema; use super::{exprlist_to_fields, Expr, JoinConstraint, JoinType, LogicalPlan, PlanType}; +use crate::datasource::avro::AvroFile; use crate::logical_plan::{ columnize_expr, normalize_col, normalize_cols, Column, DFField, DFSchema, DFSchemaRef, Partitioning, }; +use crate::physical_plan::avro::AvroReadOptions; /// Default table name for unnamed table pub const UNNAMED_TABLE: &str = "?table?"; @@ -154,6 +156,27 @@ impl LogicalPlanBuilder { Self::scan(table_name, provider, projection) } + /// Scan an Avro data source + pub fn scan_avro( + path: impl Into, + options: AvroReadOptions, + projection: Option>, + ) -> Result { + let path = path.into(); + Self::scan_avro_with_name(path.clone(), options, projection, path) + } + + /// Scan an Avro data source and register it with a given table name + pub fn scan_avro_with_name( + path: impl Into, + options: AvroReadOptions, + projection: Option>, + table_name: impl Into, + ) -> Result { + let provider = Arc::new(AvroFile::try_new(&path.into(), options)?); + Self::scan(table_name, provider, projection) + } + /// Scan an empty data source, mainly used in tests pub fn scan_empty( name: Option<&str>, From e8a62065797feb05e4b32240a1767f99dffc7efb Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 15:34:42 +0200 Subject: [PATCH 22/58] Add avro to available logical plan nodes --- ballista/rust/core/proto/ballista.proto | 10 +++++++ .../core/src/serde/logical_plan/from_proto.rs | 27 +++++++++++++++++++ .../core/src/serde/logical_plan/to_proto.rs | 14 ++++++++++ 3 files changed, 51 insertions(+) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 601868fdd7b2..8990e5cce9d3 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -253,6 +253,7 @@ message LogicalPlanNode { WindowNode window = 13; AnalyzeNode analyze = 14; CrossJoinNode cross_join = 15; + AvroTableScanNode avro_scan = 16; } } @@ -296,6 +297,15 @@ message ParquetTableScanNode { repeated LogicalExprNode filters = 4; } +message AvroTableScanNode { + string table_name = 1; + string path = 2; + string file_extension = 3; + ProjectionColumns projection = 4; + Schema schema = 5; + repeated LogicalExprNode filters = 6; +} + message ProjectionNode { LogicalPlanNode input = 1; repeated LogicalExprNode expr = 2; diff --git a/ballista/rust/core/src/serde/logical_plan/from_proto.rs b/ballista/rust/core/src/serde/logical_plan/from_proto.rs index 1fad748204c5..8ffdb650aa21 100644 --- a/ballista/rust/core/src/serde/logical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/from_proto.rs @@ -32,6 +32,7 @@ use datafusion::logical_plan::{ LogicalPlan, LogicalPlanBuilder, Operator, }; use datafusion::physical_plan::aggregates::AggregateFunction; +use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::physical_plan::csv::CsvReadOptions; use datafusion::physical_plan::window_functions::BuiltInWindowFunction; use datafusion::scalar::ScalarValue; @@ -171,6 +172,32 @@ impl TryInto for &protobuf::LogicalPlanNode { .build() .map_err(|e| e.into()) } + LogicalPlanType::AvroScan(scan) => { + let schema: Schema = convert_required!(scan.schema)?; + let options = AvroReadOptions { + schema: Some(Arc::new(schema.clone())), + file_extension: &scan.file_extension, + }; + + let mut projection = None; + if let Some(columns) = &scan.projection { + let column_indices = columns + .columns + .iter() + .map(|name| schema.index_of(name)) + .collect::, _>>()?; + projection = Some(column_indices); + } + + LogicalPlanBuilder::scan_avro_with_name( + &scan.path, + options, + projection, + &scan.table_name, + )? + .build() + .map_err(|e| e.into()) + } LogicalPlanType::Sort(sort) => { let input: LogicalPlan = convert_box_required!(sort.input)?; let sort_expr: Vec = sort diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index bfe8e0f26615..3ae45fc0b955 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -25,6 +25,7 @@ use crate::serde::{protobuf, BallistaError}; use datafusion::arrow::datatypes::{ DataType, Field, IntervalUnit, Schema, SchemaRef, TimeUnit, }; +use datafusion::datasource::avro::AvroFile; use datafusion::datasource::{CsvFile, PartitionedFile, TableDescriptor}; use datafusion::logical_plan::{ window_frames::{WindowFrame, WindowFrameBound, WindowFrameUnits}, @@ -793,6 +794,19 @@ impl TryInto for &LogicalPlan { }, )), }) + } else if let Some(avro) = source.downcast_ref::() { + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::AvroScan( + protobuf::AvroTableScanNode { + table_name: table_name.to_owned(), + path: avro.path().to_owned(), + projection, + schema: Some(schema), + file_extension: avro.file_extension().to_string(), + filters, + }, + )), + }) } else { Err(BallistaError::General(format!( "logical plan to_proto unsupported table provider {:?}", From 45eac7c3dc3bd3fc0acc806d007aa718076a8da6 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 15:41:34 +0200 Subject: [PATCH 23/58] Add ToTimestampMillis as a scalar function in protos --- ballista/rust/core/proto/ballista.proto | 1 + ballista/rust/core/src/serde/logical_plan/to_proto.rs | 3 +++ ballista/rust/core/src/serde/physical_plan/from_proto.rs | 1 + 3 files changed, 5 insertions(+) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 8990e5cce9d3..55825639b76a 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -152,6 +152,7 @@ enum ScalarFunction { SHA384 = 32; SHA512 = 33; LN = 34; + TOTIMESTAMPMILLIS = 35; } message ScalarFunctionNode { diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index 3ae45fc0b955..da1ede28bcdb 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -1488,6 +1488,9 @@ impl TryInto for &BuiltinScalarFunction { BuiltinScalarFunction::SHA256 => Ok(protobuf::ScalarFunction::Sha256), BuiltinScalarFunction::SHA384 => Ok(protobuf::ScalarFunction::Sha384), BuiltinScalarFunction::SHA512 => Ok(protobuf::ScalarFunction::Sha512), + BuiltinScalarFunction::ToTimestampMillis => { + Ok(protobuf::ScalarFunction::Totimestampmillis) + } _ => Err(BallistaError::General(format!( "logical_plan::to_proto() unsupported scalar function {:?}", self diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index 3cd8cf3871cf..4132deaa96b1 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -544,6 +544,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Sha384 => BuiltinScalarFunction::SHA384, ScalarFunction::Sha512 => BuiltinScalarFunction::SHA512, ScalarFunction::Ln => BuiltinScalarFunction::Ln, + ScalarFunction::Totimestampmillis => BuiltinScalarFunction::ToTimestampMillis, } } } From b4340ac836020f558b918b8a4567d45ea97ab28f Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 16:19:16 +0200 Subject: [PATCH 24/58] Allow Avro in PhysicalPlan nodes --- ballista/rust/core/proto/ballista.proto | 12 ++++++++++ .../core/src/serde/logical_plan/to_proto.rs | 10 ++++++-- .../src/serde/physical_plan/from_proto.rs | 16 +++++++++++++ .../core/src/serde/physical_plan/to_proto.rs | 23 +++++++++++++++++++ 4 files changed, 59 insertions(+), 2 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 55825639b76a..3fc291e3a83f 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -468,6 +468,7 @@ message PhysicalPlanNode { WindowAggExecNode window = 17; ShuffleWriterExecNode shuffle_writer = 18; CrossJoinExecNode cross_join = 19; + AvroScanExecNode avro_scan = 20; } } @@ -621,6 +622,17 @@ message CsvScanExecNode { repeated string filename = 8; } +message AvroScanExecNode { + string path = 1; + repeated uint32 projection = 2; + Schema schema = 3; + string file_extension = 4; + uint32 batch_size = 5; + + // partition filenames + repeated string filename = 8; +} + enum PartitionMode { COLLECT_LEFT = 0; PARTITIONED = 1; diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index da1ede28bcdb..e195e82df48e 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -1113,7 +1113,13 @@ impl TryInto for &Expr { ) } }; - let arg = &args[0]; + let arg_expr: Option> = if !args.is_empty() + { + let arg = &args[0]; + Some(Box::new(arg.try_into()?)) + } else { + None + }; let partition_by = partition_by .iter() .map(|e| e.try_into()) @@ -1126,7 +1132,7 @@ impl TryInto for &Expr { protobuf::window_expr_node::WindowFrame::Frame(window_frame.into()) }); let window_expr = Box::new(protobuf::WindowExprNode { - expr: Some(Box::new(arg.try_into()?)), + expr: arg_expr, window_function: Some(window_function), partition_by, order_by, diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index 4132deaa96b1..0d233725fc9f 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -43,6 +43,7 @@ use datafusion::logical_plan::{ window_frames::WindowFrame, DFSchema, Expr, JoinConstraint, JoinType, }; use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateFunction}; +use datafusion::physical_plan::avro::{AvroExec, AvroReadOptions}; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::hash_aggregate::{AggregateMode, HashAggregateExec}; use datafusion::physical_plan::hash_join::PartitionMode; @@ -153,6 +154,21 @@ impl TryInto> for &protobuf::PhysicalPlanNode { None, ))) } + PhysicalPlanType::AvroScan(scan) => { + let schema = Arc::new(convert_required!(scan.schema)?); + let options = AvroReadOptions { + schema: Some(schema), + file_extension: &scan.file_extension, + }; + let projection = scan.projection.iter().map(|i| *i as usize).collect(); + Ok(Arc::new(AvroExec::try_from_path( + &scan.path, + options, + Some(projection), + scan.batch_size as usize, + None, + )?)) + } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { let input: Arc = convert_box_required!(coalesce_batches.input)?; diff --git a/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/ballista/rust/core/src/serde/physical_plan/to_proto.rs index e7d4ac652874..22a49cb881ba 100644 --- a/ballista/rust/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -62,6 +62,7 @@ use crate::execution_plans::{ use crate::serde::protobuf::repartition_exec_node::PartitionMethod; use crate::serde::scheduler::PartitionLocation; use crate::serde::{protobuf, BallistaError}; +use datafusion::physical_plan::avro::AvroExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::functions::{BuiltinScalarFunction, ScalarFunctionExpr}; use datafusion::physical_plan::repartition::RepartitionExec; @@ -285,6 +286,28 @@ impl TryInto for Arc { }, )), }) + } else if let Some(exec) = plan.downcast_ref::() { + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::AvroScan( + protobuf::AvroScanExecNode { + path: exec.path().to_owned(), + filename: exec.filenames().to_vec(), + projection: exec + .projection() + .ok_or_else(|| { + BallistaError::General( + "projection in AvroExec doesn't exist.".to_owned(), + ) + })? + .iter() + .map(|n| *n as u32) + .collect(), + file_extension: exec.file_extension().to_owned(), + schema: Some(exec.file_schema().as_ref().into()), + batch_size: exec.batch_size() as u32, + }, + )), + }) } else if let Some(exec) = plan.downcast_ref::() { let mut partition = vec![]; for location in &exec.partition { From 408f759447e18a3201eb82ba288ed7a0433c4238 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 24 Aug 2021 22:11:46 +0200 Subject: [PATCH 25/58] Remove remaining confusing references to 'json' in avro mod --- datafusion/src/avro_to_arrow/arrow_array_reader.rs | 10 +++++----- datafusion/src/avro_to_arrow/reader.rs | 8 ++++---- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index 8306442b0c13..01f467cd53d7 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -315,7 +315,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let builder = builder .as_any_mut() .downcast_mut::>() - .ok_or_else(||ArrowError::JsonError( + .ok_or_else(||ArrowError::SchemaError( "Cast failed for ListBuilder during nested data parsing".to_string(), ))?; for val in vals { @@ -330,7 +330,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { builder.append(true)?; } DataType::Dictionary(_, _) => { - let builder = builder.as_any_mut().downcast_mut::>>().ok_or_else(||ArrowError::JsonError( + let builder = builder.as_any_mut().downcast_mut::>>().ok_or_else(||ArrowError::SchemaError( "Cast failed for ListBuilder during nested data parsing".to_string(), ))?; for val in vals { @@ -497,7 +497,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::UInt32 => self.read_primitive_list_values::(rows), DataType::UInt64 => self.read_primitive_list_values::(rows), DataType::Float16 => { - return Err(ArrowError::JsonError("Float16 not supported".to_string())) + return Err(ArrowError::SchemaError("Float16 not supported".to_string())) } DataType::Float32 => self.read_primitive_list_values::(rows), DataType::Float64 => self.read_primitive_list_values::(rows), @@ -506,7 +506,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { | DataType::Date64 | DataType::Time32(_) | DataType::Time64(_) => { - return Err(ArrowError::JsonError( + return Err(ArrowError::SchemaError( "Temporal types are not yet supported, see ARROW-4803".to_string(), )) } @@ -574,7 +574,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { .build() } datatype => { - return Err(ArrowError::JsonError(format!( + return Err(ArrowError::SchemaError(format!( "Nested list of {:?} not supported", datatype ))); diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index f21e88d1e45d..17d051b22073 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -27,10 +27,10 @@ use std::sync::Arc; /// Avro file reader builder #[derive(Debug)] pub struct ReaderBuilder { - /// Optional schema for the JSON file + /// Optional schema for the Avro file /// /// If the schema is not supplied, the reader will try to infer the schema - /// based on the JSON structure. + /// based on the Avro structure. schema: Option, /// Batch size (number of records to load each time) /// @@ -51,7 +51,7 @@ impl Default for ReaderBuilder { } impl ReaderBuilder { - /// Create a new builder for configuring JSON parsing options. + /// Create a new builder for configuring Avro parsing options. /// /// To convert a builder into a reader, call `Reader::from_builder` /// @@ -199,7 +199,7 @@ mod tests { } #[test] - fn test_json_basic() { + fn test_avro_basic() { let mut reader = build_reader("alltypes_dictionary.avro"); let batch = reader.next().unwrap().unwrap(); From f34b995e5d86311b66800ae93326d60499305727 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 25 Aug 2021 13:18:17 +0200 Subject: [PATCH 26/58] rename 'parquet' words in avro test and examples --- datafusion-examples/examples/avro_sql.rs | 4 ++-- datafusion/src/physical_plan/avro.rs | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion-examples/examples/avro_sql.rs b/datafusion-examples/examples/avro_sql.rs index aaa6cc6381ee..e9676a05b1fc 100644 --- a/datafusion-examples/examples/avro_sql.rs +++ b/datafusion-examples/examples/avro_sql.rs @@ -21,7 +21,7 @@ use datafusion::error::Result; use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::prelude::*; -/// This example demonstrates executing a simple query against an Arrow data source (Parquet) and +/// This example demonstrates executing a simple query against an Arrow data source (Avro) and /// fetching results #[tokio::main] async fn main() -> Result<()> { @@ -30,7 +30,7 @@ async fn main() -> Result<()> { let testdata = datafusion::arrow::util::test_util::arrow_test_data(); - // register parquet file with the execution context + // register avro file with the execution context let avro_file = &format!("{}/avro/alltypes_plain.avro", testdata); ctx.register_avro("alltypes_plain", avro_file, AvroReadOptions::default())?; diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index 5c3654ddd79e..0f82151881bf 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -361,16 +361,16 @@ mod tests { async fn test() -> Result<()> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); - let parquet_exec = AvroExec::try_from_path( + let avro_exec = AvroExec::try_from_path( &filename, AvroReadOptions::default(), Some(vec![0, 1, 2]), 1024, None, )?; - assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); + assert_eq!(avro_exec.output_partitioning().partition_count(), 1); - let mut results = parquet_exec.execute(0).await?; + let mut results = avro_exec.execute(0).await?; let batch = results.next().await.unwrap()?; assert_eq!(8, batch.num_rows()); From 6ce0904265ae9b1feb55b2aaee0a068bc6bc9b0e Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 25 Aug 2021 16:01:36 +0200 Subject: [PATCH 27/58] Handle Union of nested lists in arrow reader --- .../src/avro_to_arrow/arrow_array_reader.rs | 113 ++++++++++++++++-- 1 file changed, 103 insertions(+), 10 deletions(-) diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index 01f467cd53d7..a9bff5504c31 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -441,6 +441,8 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let list_nulls = list_nulls.as_slice_mut(); offsets.push(cur_offset); rows.iter().enumerate().for_each(|(i, v)| { + // TODO: unboxing Union(Array(Union(...))) should probably be done earlier + let v = maybe_resolve_union(v); if let Value::Array(a) = v { cur_offset += OffsetSize::from_usize(a.len()).unwrap(); bit_util::set_bit(list_nulls, i); @@ -799,7 +801,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let values = rows .iter() .flat_map(|row| { - // read values from list + let row = maybe_resolve_union(row); if let Value::Array(values) = row { values .iter() @@ -933,21 +935,25 @@ fn resolve_item(value: &Value) -> Option { T::resolve(value) } +fn maybe_resolve_union(value: &Value) -> &Value { + if SchemaKind::from(value) == SchemaKind::Union { + // Pull out the Union, and attempt to resolve against it. + match value { + Value::Union(b) => b, + _ => unreachable!(), + } + } else { + value + } +} + impl Resolver for N where N: ArrowNumericType, N::Native: num_traits::cast::NumCast, { fn resolve(value: &Value) -> Option { - let value = if SchemaKind::from(value) == SchemaKind::Union { - // Pull out the Union, and attempt to resolve against it. - match value { - Value::Union(b) => b, - _ => unreachable!(), - } - } else { - value - }; + let value = maybe_resolve_union(value); match value { Value::Int(i) | Value::TimeMillis(i) | Value::Date(i) => NumCast::from(*i), Value::Long(l) @@ -957,7 +963,94 @@ where Value::Float(f) => NumCast::from(*f), Value::Double(f) => NumCast::from(*f), Value::Duration(_d) => unimplemented!(), // shenanigans type + Value::Null => None, _ => unreachable!(), } } } + +#[cfg(test)] +mod test { + use crate::arrow::array::Array; + use crate::arrow::datatypes::Field; + use crate::avro_to_arrow::{Reader, ReaderBuilder}; + use arrow::array::{Int32Array, Int64Array, ListArray, StructArray, UnionArray}; + use arrow::datatypes::DataType; + use std::fs::File; + + fn build_reader(name: &str, batch_size: usize) -> Reader { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/{}", testdata, name); + let builder = ReaderBuilder::new() + .infer_schema() + .with_batch_size(batch_size); + builder.build(File::open(filename).unwrap()).unwrap() + } + + #[test] + fn test_avro_read_list() { + let mut reader = build_reader("list_columns.avro", 3); + let schema = reader.schema(); + let (col_id_index, _) = schema.column_with_name("int64_list").unwrap(); + let batch = reader.next().unwrap().unwrap(); + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + let a_array = batch + .column(col_id_index) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + *a_array.data_type(), + DataType::List(Box::new(Field::new("bigint", DataType::Int64, true))) + ); + let array = a_array.value(0); + assert_eq!(*array.data_type(), DataType::Int64); + + assert_eq!( + 6, + array + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .flatten() + .sum::() + ); + } + #[test] + fn test_avro_read_nested_list() { + let mut reader = build_reader("nested_lists.snappy.avro", 3); + let batch = reader.next().unwrap().unwrap(); + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + } + + #[test] + fn test_avro_iterator() { + let reader = build_reader("alltypes_plain.avro", 5); + let schema = reader.schema(); + let (col_id_index, _) = schema.column_with_name("id").unwrap(); + + let mut sum_num_rows = 0; + let mut num_batches = 0; + let mut sum_id = 0; + for batch in reader { + let batch = batch.unwrap(); + assert_eq!(11, batch.num_columns()); + sum_num_rows += batch.num_rows(); + num_batches += 1; + let batch_schema = batch.schema(); + assert_eq!(schema, batch_schema); + let a_array = batch + .column(col_id_index) + .as_any() + .downcast_ref::() + .unwrap(); + sum_id += (0..a_array.len()).map(|i| a_array.value(i)).sum::(); + } + assert_eq!(8, sum_num_rows); + assert_eq!(2, num_batches); + assert_eq!(28, sum_id); + } +} From e1e40ef28b60f6890e80216b2763c355bc53a837 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 25 Aug 2021 16:12:51 +0200 Subject: [PATCH 28/58] test timestamp arrays --- .../src/avro_to_arrow/arrow_array_reader.rs | 42 ++++++++++++++++++- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index a9bff5504c31..2b95c98e31eb 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -972,9 +972,9 @@ where #[cfg(test)] mod test { use crate::arrow::array::Array; - use crate::arrow::datatypes::Field; + use crate::arrow::datatypes::{Field, TimeUnit}; use crate::avro_to_arrow::{Reader, ReaderBuilder}; - use arrow::array::{Int32Array, Int64Array, ListArray, StructArray, UnionArray}; + use arrow::array::{Int32Array, Int64Array, ListArray, TimestampMicrosecondArray}; use arrow::datatypes::DataType; use std::fs::File; @@ -987,6 +987,44 @@ mod test { builder.build(File::open(filename).unwrap()).unwrap() } + // TODO: Fixed, Enum, Dictionary + + #[test] + fn test_time_avro_milliseconds() { + let mut reader = build_reader("alltypes_plain.avro", 10); + let batch = reader.next().unwrap().unwrap(); + + assert_eq!(11, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + + let schema = reader.schema(); + let batch_schema = batch.schema(); + assert_eq!(schema, batch_schema); + + let timestamp_col = schema.column_with_name("timestamp_col").unwrap(); + assert_eq!( + &DataType::Timestamp(TimeUnit::Microsecond, None), + timestamp_col.1.data_type() + ); + eprintln!("batch.column(a.0) = {:?}", batch.column(timestamp_col.0)); + let timestamp_array = batch + .column(timestamp_col.0) + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..timestamp_array.len() { + assert!(timestamp_array.is_valid(i)); + } + assert_eq!(1235865600000000, timestamp_array.value(0)); + assert_eq!(1235865660000000, timestamp_array.value(1)); + assert_eq!(1238544000000000, timestamp_array.value(2)); + assert_eq!(1238544060000000, timestamp_array.value(3)); + assert_eq!(1233446400000000, timestamp_array.value(4)); + assert_eq!(1233446460000000, timestamp_array.value(5)); + assert_eq!(1230768000000000, timestamp_array.value(6)); + assert_eq!(1230768060000000, timestamp_array.value(7)); + } + #[test] fn test_avro_read_list() { let mut reader = build_reader("list_columns.avro", 3); From 1c79ffbb386fae52c3798d0f39246202ce651e3e Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Wed, 25 Aug 2021 16:13:07 +0200 Subject: [PATCH 29/58] remove debug statement --- datafusion/src/avro_to_arrow/arrow_array_reader.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index 2b95c98e31eb..e6a95664c1b0 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -1006,7 +1006,6 @@ mod test { &DataType::Timestamp(TimeUnit::Microsecond, None), timestamp_col.1.data_type() ); - eprintln!("batch.column(a.0) = {:?}", batch.column(timestamp_col.0)); let timestamp_array = batch .column(timestamp_col.0) .as_any() From c96b781e1e9f9bfa20eb2165204af01ffc92a2b3 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 13:07:08 +0200 Subject: [PATCH 30/58] Make avro optional --- .github/workflows/rust.yml | 2 +- datafusion-examples/Cargo.toml | 5 +- datafusion/Cargo.toml | 5 +- .../src/avro_to_arrow/arrow_array_reader.rs | 20 +- datafusion/src/avro_to_arrow/mod.rs | 462 +----------------- datafusion/src/avro_to_arrow/reader.rs | 20 +- datafusion/src/avro_to_arrow/schema.rs | 435 +++++++++++++++++ datafusion/src/error.rs | 4 + datafusion/src/physical_plan/avro.rs | 38 +- 9 files changed, 517 insertions(+), 474 deletions(-) create mode 100644 datafusion/src/avro_to_arrow/schema.rs diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index a940e6dabf9b..1b0386be7e63 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -112,7 +112,7 @@ jobs: cargo test --no-default-features cargo run --example csv_sql cargo run --example parquet_sql - cargo run --example avro_sql + cargo run --example avro_sql --features=avro env: CARGO_HOME: "/github/home/.cargo" CARGO_TARGET_DIR: "/github/home/target" diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index f98121b0ece8..113cd5bb9103 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -27,6 +27,10 @@ keywords = [ "arrow", "query", "sql" ] edition = "2018" publish = false +[[example]] +name = "avro_sql" +path = "examples/avro_sql.rs" +required-features = ["datafusion/avro"] [dev-dependencies] arrow-flight = { version = "^5.3" } @@ -36,4 +40,3 @@ tonic = "0.5" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync"] } futures = "0.3" num_cpus = "1.13.0" -avro-rs = { version = "0.13", features = ["snappy"] } diff --git a/datafusion/Cargo.toml b/datafusion/Cargo.toml index fee3f08746d9..1d965680e1a4 100644 --- a/datafusion/Cargo.toml +++ b/datafusion/Cargo.toml @@ -44,7 +44,8 @@ regex_expressions = ["regex", "lazy_static"] unicode_expressions = ["unicode-segmentation"] # Used for testing ONLY: causes all values to hash to the same value (test for collisions) force_hash_collisions = [] - +# Used to enable the avro format +avro = ["avro-rs"] [dependencies] ahash = "0.7" @@ -69,7 +70,7 @@ regex = { version = "^1.4.3", optional = true } lazy_static = { version = "^1.4.0", optional = true } smallvec = { version = "1.6", features = ["union"] } rand = "0.8" -avro-rs = { version = "0.13", features = ["snappy"] } +avro-rs = { version = "0.13", features = ["snappy"], optional = true } num-traits = "0.2" [dev-dependencies] diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index e6a95664c1b0..b8d4d19c6c10 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -40,10 +40,11 @@ use arrow::array::{BinaryArray, GenericListArray}; use arrow::datatypes::SchemaRef; use arrow::error::ArrowError::SchemaError; use arrow::error::Result as ArrowResult; -use avro_rs::schema::Schema as AvroSchema; -use avro_rs::schema::SchemaKind; -use avro_rs::types::Value; -use avro_rs::{AvroResult, Reader as AvroReader}; +use avro_rs::{ + schema::{Schema as AvroSchema, SchemaKind}, + types::Value, + AvroResult, Error as AvroError, Reader as AvroReader, +}; use num_traits::NumCast; use std::collections::HashMap; use std::io::Read; @@ -60,10 +61,11 @@ pub struct AvroArrowArrayReader<'a, R: Read> { impl<'a, R: Read> AvroArrowArrayReader<'a, R> { pub fn try_new( - reader: AvroReader<'a, R>, + reader: R, schema: SchemaRef, projection: Option>, ) -> Result { + let reader = AvroReader::new(reader)?; let writer_schema = reader.writer_schema().clone(); let schema_lookup = Self::schema_lookup(writer_schema)?; Ok(Self { @@ -876,9 +878,9 @@ fn resolve_string(v: &Value) -> Option { match v { Value::String(s) => Ok(s.clone()), Value::Bytes(bytes) => Ok(String::from_utf8(bytes.to_vec()) - .map_err(avro_rs::Error::ConvertToUtf8) + .map_err(AvroError::ConvertToUtf8) .ok()?), - other => Err(avro_rs::Error::GetString(other.into())), + other => Err(AvroError::GetString(other.into())), } .ok() } @@ -891,7 +893,7 @@ fn resolve_u8(v: Value) -> AvroResult { } } - Err(avro_rs::Error::GetU8(int.into())) + Err(AvroError::GetU8(int.into())) } fn resolve_bytes(v: Value) -> Option> { @@ -906,7 +908,7 @@ fn resolve_bytes(v: Value) -> Option> { .collect::, _>>() .ok()?, )), - other => Err(avro_rs::Error::GetBytes(other.into())), + other => Err(AvroError::GetBytes(other.into())), } .ok() .and_then(|v| match v { diff --git a/datafusion/src/avro_to_arrow/mod.rs b/datafusion/src/avro_to_arrow/mod.rs index 6ccb8ba2c980..e0c05b277465 100644 --- a/datafusion/src/avro_to_arrow/mod.rs +++ b/datafusion/src/avro_to_arrow/mod.rs @@ -17,443 +17,31 @@ //! This module contains utilities to manipulate avro metadata. +#[cfg(feature = "avro")] mod arrow_array_reader; +#[cfg(feature = "avro")] mod reader; - -use crate::arrow::datatypes::{DataType, IntervalUnit, Schema, TimeUnit}; -use crate::error::{DataFusionError, Result}; -use arrow::datatypes::Field; -use avro_rs::schema::Name; -use avro_rs::types::Value; -use avro_rs::Schema as AvroSchema; -use std::collections::BTreeMap; -use std::convert::TryFrom; - -pub use reader::{infer_avro_schema_from_reader, Reader, ReaderBuilder}; - -/// Converts an avro schema to an arrow schema -pub fn to_arrow_schema(avro_schema: &avro_rs::Schema) -> Result { - let mut schema_fields = vec![]; - match avro_schema { - AvroSchema::Record { fields, .. } => { - for field in fields { - schema_fields.push(schema_to_field_with_props( - &field.schema, - Some(&field.name), - false, - Some(&external_props(&field.schema)), - )?) - } - } - schema => schema_fields.push(schema_to_field(schema, Some(""), false)?), - } - - let schema = Schema::new(schema_fields); - Ok(schema) -} - -fn schema_to_field( - schema: &avro_rs::Schema, - name: Option<&str>, - nullable: bool, -) -> Result { - schema_to_field_with_props(schema, name, nullable, Some(&Default::default())) -} - -fn schema_to_field_with_props( - schema: &AvroSchema, - name: Option<&str>, - nullable: bool, - props: Option<&BTreeMap>, -) -> Result { - let mut nullable = nullable; - let field_type: DataType = match schema { - AvroSchema::Null => DataType::Null, - AvroSchema::Boolean => DataType::Boolean, - AvroSchema::Int => DataType::Int32, - AvroSchema::Long => DataType::Int64, - AvroSchema::Float => DataType::Float32, - AvroSchema::Double => DataType::Float64, - AvroSchema::Bytes => DataType::Binary, - AvroSchema::String => DataType::Utf8, - AvroSchema::Array(item_schema) => DataType::List(Box::new( - schema_to_field_with_props(item_schema, None, false, None)?, - )), - AvroSchema::Map(value_schema) => { - let value_field = - schema_to_field_with_props(value_schema, Some("value"), false, None)?; - DataType::Dictionary( - Box::new(DataType::Utf8), - Box::new(value_field.data_type().clone()), - ) - } - AvroSchema::Union(us) => { - // If there are only two variants and one of them is null, set the other type as the field data type - let has_nullable = us.find_schema(&Value::Null).is_some(); - let sub_schemas = us.variants(); - if has_nullable && sub_schemas.len() == 2 { - nullable = true; - if let Some(schema) = sub_schemas - .iter() - .find(|&schema| !matches!(schema, AvroSchema::Null)) - { - schema_to_field_with_props(schema, None, has_nullable, None)? - .data_type() - .clone() - } else { - return Err(DataFusionError::AvroError( - avro_rs::Error::GetUnionDuplicate, - )); - } - } else { - let fields = sub_schemas - .iter() - .map(|s| schema_to_field_with_props(s, None, has_nullable, None)) - .collect::>>()?; - DataType::Union(fields) - } - } - AvroSchema::Record { name, fields, .. } => { - let fields: Result> = fields - .iter() - .map(|field| { - let mut props = BTreeMap::new(); - if let Some(doc) = &field.doc { - props.insert("doc".to_string(), doc.clone()); - } - /*if let Some(aliases) = fields.aliases { - props.insert("aliases", aliases); - }*/ - schema_to_field_with_props( - &field.schema, - Some(&format!("{}.{}", name.fullname(None), field.name)), - false, - Some(&props), - ) - }) - .collect(); - DataType::Struct(fields?) - } - AvroSchema::Enum { symbols, name, .. } => { - return Ok(Field::new_dict( - &name.fullname(None), - index_type(symbols.len()), - false, - 0, - false, - )) - } - AvroSchema::Fixed { size, .. } => DataType::FixedSizeBinary(*size as i32), - AvroSchema::Decimal { - precision, scale, .. - } => DataType::Decimal(*precision, *scale), - AvroSchema::Uuid => DataType::Utf8, - AvroSchema::Date => DataType::Date32, - AvroSchema::TimeMillis => DataType::Time32(TimeUnit::Millisecond), - AvroSchema::TimeMicros => DataType::Time64(TimeUnit::Microsecond), - AvroSchema::TimestampMillis => DataType::Timestamp(TimeUnit::Millisecond, None), - AvroSchema::TimestampMicros => DataType::Timestamp(TimeUnit::Microsecond, None), - AvroSchema::Duration => DataType::Duration(TimeUnit::Millisecond), - }; - - let data_type = field_type.clone(); - let name = name.unwrap_or_else(|| default_field_name(&data_type)); - - let mut field = Field::new(name, field_type, nullable); - field.set_metadata(props.cloned()); - Ok(field) -} - -fn default_field_name(dt: &DataType) -> &str { - match dt { - DataType::Null => "null", - DataType::Boolean => "bit", - DataType::Int8 => "tinyint", - DataType::Int16 => "smallint", - DataType::Int32 => "int", - DataType::Int64 => "bigint", - DataType::UInt8 => "uint1", - DataType::UInt16 => "uint2", - DataType::UInt32 => "uint4", - DataType::UInt64 => "uint8", - DataType::Float16 => "float2", - DataType::Float32 => "float4", - DataType::Float64 => "float8", - DataType::Date32 => "dateday", - DataType::Date64 => "datemilli", - DataType::Time32(tu) | DataType::Time64(tu) => match tu { - TimeUnit::Second => "timesec", - TimeUnit::Millisecond => "timemilli", - TimeUnit::Microsecond => "timemicro", - TimeUnit::Nanosecond => "timenano", - }, - DataType::Timestamp(tu, tz) => { - if tz.is_some() { - match tu { - TimeUnit::Second => "timestampsectz", - TimeUnit::Millisecond => "timestampmillitz", - TimeUnit::Microsecond => "timestampmicrotz", - TimeUnit::Nanosecond => "timestampnanotz", - } - } else { - match tu { - TimeUnit::Second => "timestampsec", - TimeUnit::Millisecond => "timestampmilli", - TimeUnit::Microsecond => "timestampmicro", - TimeUnit::Nanosecond => "timestampnano", - } - } - } - DataType::Duration(_) => "duration", - DataType::Interval(unit) => match unit { - IntervalUnit::YearMonth => "intervalyear", - IntervalUnit::DayTime => "intervalmonth", - }, - DataType::Binary => "varbinary", - DataType::FixedSizeBinary(_) => "fixedsizebinary", - DataType::LargeBinary => "largevarbinary", - DataType::Utf8 => "varchar", - DataType::LargeUtf8 => "largevarchar", - DataType::List(_) => "list", - DataType::FixedSizeList(_, _) => "fixed_size_list", - DataType::LargeList(_) => "largelist", - DataType::Struct(_) => "struct", - DataType::Union(_) => "union", - DataType::Dictionary(_, _) => "map", - DataType::Decimal(_, _) => "decimal", - } -} - -fn index_type(len: usize) -> DataType { - if len <= usize::from(u8::MAX) { - DataType::Int8 - } else if len <= usize::from(u16::MAX) { - DataType::Int16 - } else if usize::try_from(u32::MAX).map(|i| len < i).unwrap_or(false) { - DataType::Int32 - } else { - DataType::Int64 - } -} - -fn external_props(schema: &AvroSchema) -> BTreeMap { - let mut props = BTreeMap::new(); - match &schema { - AvroSchema::Record { - doc: Some(ref doc), .. - } - | AvroSchema::Enum { - doc: Some(ref doc), .. - } => { - props.insert("doc".to_string(), doc.clone()); - } - _ => {} - } - match &schema { - AvroSchema::Record { - name: - Name { - aliases: Some(aliases), - namespace, - .. - }, - .. - } - | AvroSchema::Enum { - name: - Name { - aliases: Some(aliases), - namespace, - .. - }, - .. - } - | AvroSchema::Fixed { - name: - Name { - aliases: Some(aliases), - namespace, - .. - }, - .. - } => { - let aliases: Vec = aliases - .iter() - .map(|alias| aliased(alias, namespace.as_deref(), None)) - .collect(); - props.insert("aliases".to_string(), format!("[{}]", aliases.join(","))); - } - _ => {} - } - props -} - -#[allow(dead_code)] -fn get_metadata( - _schema: AvroSchema, - props: BTreeMap, -) -> BTreeMap { - let mut metadata: BTreeMap = Default::default(); - metadata.extend(props); - metadata -} - -/// Returns the fully qualified name for a field -pub fn aliased( - name: &str, - namespace: Option<&str>, - default_namespace: Option<&str>, -) -> String { - if name.contains('.') { - name.to_string() - } else { - let namespace = namespace.as_ref().copied().or(default_namespace); - - match namespace { - Some(ref namespace) => format!("{}.{}", namespace, name), - None => name.to_string(), - } - } -} - -#[cfg(test)] -mod test { - use crate::arrow::datatypes::DataType::{Binary, Float32, Float64, Timestamp, Utf8}; - use crate::arrow::datatypes::TimeUnit::Microsecond; - use crate::arrow::datatypes::{Field, Schema}; - use crate::avro_to_arrow::{aliased, external_props, to_arrow_schema}; - use arrow::datatypes::DataType::{Boolean, Int32, Int64}; - use avro_rs::schema::Name; - use avro_rs::Schema as AvroSchema; - - #[test] - fn test_alias() { - assert_eq!(aliased("foo.bar", None, None), "foo.bar"); - assert_eq!(aliased("bar", Some("foo"), None), "foo.bar"); - assert_eq!(aliased("bar", Some("foo"), Some("cat")), "foo.bar"); - assert_eq!(aliased("bar", None, Some("cat")), "cat.bar"); - } - - #[test] - fn test_external_props() { - let record_schema = AvroSchema::Record { - name: Name { - name: "record".to_string(), - namespace: None, - aliases: Some(vec!["fooalias".to_string(), "baralias".to_string()]), - }, - doc: Some("record documentation".to_string()), - fields: vec![], - lookup: Default::default(), - }; - let props = external_props(&record_schema); - assert_eq!(props.get("doc"), Some(&"record documentation".to_string())); - assert_eq!( - props.get("aliases"), - Some(&"[fooalias,baralias]".to_string()) - ); - let enum_schema = AvroSchema::Enum { - name: Name { - name: "enum".to_string(), - namespace: None, - aliases: Some(vec!["fooenum".to_string(), "barenum".to_string()]), - }, - doc: Some("enum documentation".to_string()), - symbols: vec![], - }; - let props = external_props(&enum_schema); - assert_eq!(props.get("doc"), Some(&"enum documentation".to_string())); - assert_eq!(props.get("aliases"), Some(&"[fooenum,barenum]".to_string())); - let fixed_schema = AvroSchema::Fixed { - name: Name { - name: "fixed".to_string(), - namespace: None, - aliases: Some(vec!["foofixed".to_string(), "barfixed".to_string()]), - }, - size: 1, - }; - let props = external_props(&fixed_schema); - assert_eq!( - props.get("aliases"), - Some(&"[foofixed,barfixed]".to_string()) - ); - } - - #[test] - fn test_invalid_avro_schema() {} - - #[test] - fn test_plain_types_schema() { - let schema = AvroSchema::parse_str( - r#" - { - "type" : "record", - "name" : "topLevelRecord", - "fields" : [ { - "name" : "id", - "type" : [ "int", "null" ] - }, { - "name" : "bool_col", - "type" : [ "boolean", "null" ] - }, { - "name" : "tinyint_col", - "type" : [ "int", "null" ] - }, { - "name" : "smallint_col", - "type" : [ "int", "null" ] - }, { - "name" : "int_col", - "type" : [ "int", "null" ] - }, { - "name" : "bigint_col", - "type" : [ "long", "null" ] - }, { - "name" : "float_col", - "type" : [ "float", "null" ] - }, { - "name" : "double_col", - "type" : [ "double", "null" ] - }, { - "name" : "date_string_col", - "type" : [ "bytes", "null" ] - }, { - "name" : "string_col", - "type" : [ "bytes", "null" ] - }, { - "name" : "timestamp_col", - "type" : [ { - "type" : "long", - "logicalType" : "timestamp-micros" - }, "null" ] - } ] - }"#, - ); - assert!(schema.is_ok(), "{:?}", schema); - let arrow_schema = to_arrow_schema(&schema.unwrap()); - assert!(arrow_schema.is_ok(), "{:?}", arrow_schema); - let expected = Schema::new(vec![ - Field::new("id", Int32, true), - Field::new("bool_col", Boolean, true), - Field::new("tinyint_col", Int32, true), - Field::new("smallint_col", Int32, true), - Field::new("int_col", Int32, true), - Field::new("bigint_col", Int64, true), - Field::new("float_col", Float32, true), - Field::new("double_col", Float64, true), - Field::new("date_string_col", Binary, true), - Field::new("string_col", Binary, true), - Field::new("timestamp_col", Timestamp(Microsecond, None), true), - ]); - assert_eq!(arrow_schema.unwrap(), expected); - } - - #[test] - fn test_non_record_schema() { - let arrow_schema = to_arrow_schema(&AvroSchema::String); - assert!(arrow_schema.is_ok(), "{:?}", arrow_schema); - assert_eq!( - arrow_schema.unwrap(), - Schema::new(vec![Field::new("", Utf8, false)]) - ); - } +#[cfg(feature = "avro")] +mod schema; + +use crate::arrow::datatypes::Schema; +use crate::error::Result; +#[cfg(feature = "avro")] +pub use reader::{Reader, ReaderBuilder}; +use std::io::{Read, Seek}; + +#[cfg(feature = "avro")] +/// Infer Avro schema given a reader +pub fn infer_avro_schema_from_reader(reader: &mut R) -> Result { + let avro_reader = avro_rs::Reader::new(reader)?; + let schema = avro_reader.writer_schema(); + schema::to_arrow_schema(schema) +} + +#[cfg(not(feature = "avro"))] +/// Infer Avro schema given a reader (requires the avro feature) +pub fn infer_avro_schema_from_reader(_: &mut R) -> Result { + Err(crate::error::DataFusionError::NotImplemented( + "cannot infer avro schema without the 'avro' feature enabled".to_string(), + )) } diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index 17d051b22073..17b39a1ceb1f 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -15,12 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::arrow::datatypes::{Schema, SchemaRef}; +use super::arrow_array_reader::AvroArrowArrayReader; +use crate::arrow::datatypes::SchemaRef; use crate::arrow::record_batch::RecordBatch; -use crate::avro_to_arrow::arrow_array_reader::AvroArrowArrayReader; use crate::error::Result; use arrow::error::Result as ArrowResult; -use avro_rs::Reader as AvroReader; use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; @@ -112,7 +111,7 @@ impl ReaderBuilder { // check if schema should be inferred let schema = match self.schema { Some(schema) => schema, - None => Arc::new(infer_avro_schema_from_reader(&mut source)?), + None => Arc::new(super::infer_avro_schema_from_reader(&mut source)?), }; source.seek(SeekFrom::Start(0))?; Reader::try_new(source, schema, self.batch_size, self.projection) @@ -139,7 +138,7 @@ impl<'a, R: Read> Reader<'a, R> { ) -> Result { Ok(Self { array_reader: AvroArrowArrayReader::try_new( - AvroReader::new(reader)?, + reader, schema.clone(), projection, )?, @@ -154,8 +153,8 @@ impl<'a, R: Read> Reader<'a, R> { self.schema.clone() } - /// Read the next batch of records - #[allow(clippy::should_implement_trait)] + /// Returns the next batch of results (defined by `self.batch_size`), or `None` if there + /// are no more results pub fn next(&mut self) -> ArrowResult> { self.array_reader.next_batch(self.batch_size) } @@ -169,13 +168,6 @@ impl<'a, R: Read> Iterator for Reader<'a, R> { } } -/// Infer Avro schema given a reader -pub fn infer_avro_schema_from_reader(reader: &mut R) -> Result { - let avro_reader = avro_rs::Reader::new(reader)?; - let schema = avro_reader.writer_schema(); - super::to_arrow_schema(schema) -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/src/avro_to_arrow/schema.rs b/datafusion/src/avro_to_arrow/schema.rs new file mode 100644 index 000000000000..5706e6bba91b --- /dev/null +++ b/datafusion/src/avro_to_arrow/schema.rs @@ -0,0 +1,435 @@ +use crate::arrow::datatypes::{DataType, IntervalUnit, Schema, TimeUnit}; +use crate::error::{DataFusionError, Result}; +use arrow::datatypes::Field; +use avro_rs::schema::Name; +use avro_rs::types::Value; +use avro_rs::Schema as AvroSchema; +use std::collections::BTreeMap; +use std::convert::TryFrom; + +/// Converts an avro schema to an arrow schema +pub fn to_arrow_schema(avro_schema: &avro_rs::Schema) -> Result { + let mut schema_fields = vec![]; + match avro_schema { + AvroSchema::Record { fields, .. } => { + for field in fields { + schema_fields.push(schema_to_field_with_props( + &field.schema, + Some(&field.name), + false, + Some(&external_props(&field.schema)), + )?) + } + } + schema => schema_fields.push(schema_to_field(schema, Some(""), false)?), + } + + let schema = Schema::new(schema_fields); + Ok(schema) +} + +fn schema_to_field( + schema: &avro_rs::Schema, + name: Option<&str>, + nullable: bool, +) -> Result { + schema_to_field_with_props(schema, name, nullable, Some(&Default::default())) +} + +fn schema_to_field_with_props( + schema: &AvroSchema, + name: Option<&str>, + nullable: bool, + props: Option<&BTreeMap>, +) -> Result { + let mut nullable = nullable; + let field_type: DataType = match schema { + AvroSchema::Null => DataType::Null, + AvroSchema::Boolean => DataType::Boolean, + AvroSchema::Int => DataType::Int32, + AvroSchema::Long => DataType::Int64, + AvroSchema::Float => DataType::Float32, + AvroSchema::Double => DataType::Float64, + AvroSchema::Bytes => DataType::Binary, + AvroSchema::String => DataType::Utf8, + AvroSchema::Array(item_schema) => DataType::List(Box::new( + schema_to_field_with_props(item_schema, None, false, None)?, + )), + AvroSchema::Map(value_schema) => { + let value_field = + schema_to_field_with_props(value_schema, Some("value"), false, None)?; + DataType::Dictionary( + Box::new(DataType::Utf8), + Box::new(value_field.data_type().clone()), + ) + } + AvroSchema::Union(us) => { + // If there are only two variants and one of them is null, set the other type as the field data type + let has_nullable = us.find_schema(&Value::Null).is_some(); + let sub_schemas = us.variants(); + if has_nullable && sub_schemas.len() == 2 { + nullable = true; + if let Some(schema) = sub_schemas + .iter() + .find(|&schema| !matches!(schema, AvroSchema::Null)) + { + schema_to_field_with_props(schema, None, has_nullable, None)? + .data_type() + .clone() + } else { + return Err(DataFusionError::AvroError( + avro_rs::Error::GetUnionDuplicate, + )); + } + } else { + let fields = sub_schemas + .iter() + .map(|s| schema_to_field_with_props(s, None, has_nullable, None)) + .collect::>>()?; + DataType::Union(fields) + } + } + AvroSchema::Record { name, fields, .. } => { + let fields: Result> = fields + .iter() + .map(|field| { + let mut props = BTreeMap::new(); + if let Some(doc) = &field.doc { + props.insert("doc".to_string(), doc.clone()); + } + /*if let Some(aliases) = fields.aliases { + props.insert("aliases", aliases); + }*/ + schema_to_field_with_props( + &field.schema, + Some(&format!("{}.{}", name.fullname(None), field.name)), + false, + Some(&props), + ) + }) + .collect(); + DataType::Struct(fields?) + } + AvroSchema::Enum { symbols, name, .. } => { + return Ok(Field::new_dict( + &name.fullname(None), + index_type(symbols.len()), + false, + 0, + false, + )) + } + AvroSchema::Fixed { size, .. } => DataType::FixedSizeBinary(*size as i32), + AvroSchema::Decimal { + precision, scale, .. + } => DataType::Decimal(*precision, *scale), + AvroSchema::Uuid => DataType::Utf8, + AvroSchema::Date => DataType::Date32, + AvroSchema::TimeMillis => DataType::Time32(TimeUnit::Millisecond), + AvroSchema::TimeMicros => DataType::Time64(TimeUnit::Microsecond), + AvroSchema::TimestampMillis => DataType::Timestamp(TimeUnit::Millisecond, None), + AvroSchema::TimestampMicros => DataType::Timestamp(TimeUnit::Microsecond, None), + AvroSchema::Duration => DataType::Duration(TimeUnit::Millisecond), + }; + + let data_type = field_type.clone(); + let name = name.unwrap_or_else(|| default_field_name(&data_type)); + + let mut field = Field::new(name, field_type, nullable); + field.set_metadata(props.cloned()); + Ok(field) +} + +fn default_field_name(dt: &DataType) -> &str { + match dt { + DataType::Null => "null", + DataType::Boolean => "bit", + DataType::Int8 => "tinyint", + DataType::Int16 => "smallint", + DataType::Int32 => "int", + DataType::Int64 => "bigint", + DataType::UInt8 => "uint1", + DataType::UInt16 => "uint2", + DataType::UInt32 => "uint4", + DataType::UInt64 => "uint8", + DataType::Float16 => "float2", + DataType::Float32 => "float4", + DataType::Float64 => "float8", + DataType::Date32 => "dateday", + DataType::Date64 => "datemilli", + DataType::Time32(tu) | DataType::Time64(tu) => match tu { + TimeUnit::Second => "timesec", + TimeUnit::Millisecond => "timemilli", + TimeUnit::Microsecond => "timemicro", + TimeUnit::Nanosecond => "timenano", + }, + DataType::Timestamp(tu, tz) => { + if tz.is_some() { + match tu { + TimeUnit::Second => "timestampsectz", + TimeUnit::Millisecond => "timestampmillitz", + TimeUnit::Microsecond => "timestampmicrotz", + TimeUnit::Nanosecond => "timestampnanotz", + } + } else { + match tu { + TimeUnit::Second => "timestampsec", + TimeUnit::Millisecond => "timestampmilli", + TimeUnit::Microsecond => "timestampmicro", + TimeUnit::Nanosecond => "timestampnano", + } + } + } + DataType::Duration(_) => "duration", + DataType::Interval(unit) => match unit { + IntervalUnit::YearMonth => "intervalyear", + IntervalUnit::DayTime => "intervalmonth", + }, + DataType::Binary => "varbinary", + DataType::FixedSizeBinary(_) => "fixedsizebinary", + DataType::LargeBinary => "largevarbinary", + DataType::Utf8 => "varchar", + DataType::LargeUtf8 => "largevarchar", + DataType::List(_) => "list", + DataType::FixedSizeList(_, _) => "fixed_size_list", + DataType::LargeList(_) => "largelist", + DataType::Struct(_) => "struct", + DataType::Union(_) => "union", + DataType::Dictionary(_, _) => "map", + DataType::Decimal(_, _) => "decimal", + } +} + +fn index_type(len: usize) -> DataType { + if len <= usize::from(u8::MAX) { + DataType::Int8 + } else if len <= usize::from(u16::MAX) { + DataType::Int16 + } else if usize::try_from(u32::MAX).map(|i| len < i).unwrap_or(false) { + DataType::Int32 + } else { + DataType::Int64 + } +} + +fn external_props(schema: &AvroSchema) -> BTreeMap { + let mut props = BTreeMap::new(); + match &schema { + AvroSchema::Record { + doc: Some(ref doc), .. + } + | AvroSchema::Enum { + doc: Some(ref doc), .. + } => { + props.insert("doc".to_string(), doc.clone()); + } + _ => {} + } + match &schema { + AvroSchema::Record { + name: + Name { + aliases: Some(aliases), + namespace, + .. + }, + .. + } + | AvroSchema::Enum { + name: + Name { + aliases: Some(aliases), + namespace, + .. + }, + .. + } + | AvroSchema::Fixed { + name: + Name { + aliases: Some(aliases), + namespace, + .. + }, + .. + } => { + let aliases: Vec = aliases + .iter() + .map(|alias| aliased(alias, namespace.as_deref(), None)) + .collect(); + props.insert("aliases".to_string(), format!("[{}]", aliases.join(","))); + } + _ => {} + } + props +} + +#[allow(dead_code)] +fn get_metadata( + _schema: AvroSchema, + props: BTreeMap, +) -> BTreeMap { + let mut metadata: BTreeMap = Default::default(); + metadata.extend(props); + metadata +} + +/// Returns the fully qualified name for a field +pub fn aliased( + name: &str, + namespace: Option<&str>, + default_namespace: Option<&str>, +) -> String { + if name.contains('.') { + name.to_string() + } else { + let namespace = namespace.as_ref().copied().or(default_namespace); + + match namespace { + Some(ref namespace) => format!("{}.{}", namespace, name), + None => name.to_string(), + } + } +} + +#[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 arrow::datatypes::DataType::{Boolean, Int32, Int64}; + use avro_rs::schema::Name; + use avro_rs::Schema as AvroSchema; + + #[test] + fn test_alias() { + assert_eq!(aliased("foo.bar", None, None), "foo.bar"); + assert_eq!(aliased("bar", Some("foo"), None), "foo.bar"); + assert_eq!(aliased("bar", Some("foo"), Some("cat")), "foo.bar"); + assert_eq!(aliased("bar", None, Some("cat")), "cat.bar"); + } + + #[test] + fn test_external_props() { + let record_schema = AvroSchema::Record { + name: Name { + name: "record".to_string(), + namespace: None, + aliases: Some(vec!["fooalias".to_string(), "baralias".to_string()]), + }, + doc: Some("record documentation".to_string()), + fields: vec![], + lookup: Default::default(), + }; + let props = external_props(&record_schema); + assert_eq!(props.get("doc"), Some(&"record documentation".to_string())); + assert_eq!( + props.get("aliases"), + Some(&"[fooalias,baralias]".to_string()) + ); + let enum_schema = AvroSchema::Enum { + name: Name { + name: "enum".to_string(), + namespace: None, + aliases: Some(vec!["fooenum".to_string(), "barenum".to_string()]), + }, + doc: Some("enum documentation".to_string()), + symbols: vec![], + }; + let props = external_props(&enum_schema); + assert_eq!(props.get("doc"), Some(&"enum documentation".to_string())); + assert_eq!(props.get("aliases"), Some(&"[fooenum,barenum]".to_string())); + let fixed_schema = AvroSchema::Fixed { + name: Name { + name: "fixed".to_string(), + namespace: None, + aliases: Some(vec!["foofixed".to_string(), "barfixed".to_string()]), + }, + size: 1, + }; + let props = external_props(&fixed_schema); + assert_eq!( + props.get("aliases"), + Some(&"[foofixed,barfixed]".to_string()) + ); + } + + #[test] + fn test_invalid_avro_schema() {} + + #[test] + fn test_plain_types_schema() { + let schema = AvroSchema::parse_str( + r#" + { + "type" : "record", + "name" : "topLevelRecord", + "fields" : [ { + "name" : "id", + "type" : [ "int", "null" ] + }, { + "name" : "bool_col", + "type" : [ "boolean", "null" ] + }, { + "name" : "tinyint_col", + "type" : [ "int", "null" ] + }, { + "name" : "smallint_col", + "type" : [ "int", "null" ] + }, { + "name" : "int_col", + "type" : [ "int", "null" ] + }, { + "name" : "bigint_col", + "type" : [ "long", "null" ] + }, { + "name" : "float_col", + "type" : [ "float", "null" ] + }, { + "name" : "double_col", + "type" : [ "double", "null" ] + }, { + "name" : "date_string_col", + "type" : [ "bytes", "null" ] + }, { + "name" : "string_col", + "type" : [ "bytes", "null" ] + }, { + "name" : "timestamp_col", + "type" : [ { + "type" : "long", + "logicalType" : "timestamp-micros" + }, "null" ] + } ] + }"#, + ); + assert!(schema.is_ok(), "{:?}", schema); + let arrow_schema = to_arrow_schema(&schema.unwrap()); + assert!(arrow_schema.is_ok(), "{:?}", arrow_schema); + let expected = Schema::new(vec![ + Field::new("id", Int32, true), + Field::new("bool_col", Boolean, true), + Field::new("tinyint_col", Int32, true), + Field::new("smallint_col", Int32, true), + Field::new("int_col", Int32, true), + Field::new("bigint_col", Int64, true), + Field::new("float_col", Float32, true), + Field::new("double_col", Float64, true), + Field::new("date_string_col", Binary, true), + Field::new("string_col", Binary, true), + Field::new("timestamp_col", Timestamp(Microsecond, None), true), + ]); + assert_eq!(arrow_schema.unwrap(), expected); + } + + #[test] + fn test_non_record_schema() { + let arrow_schema = to_arrow_schema(&AvroSchema::String); + assert!(arrow_schema.is_ok(), "{:?}", arrow_schema); + assert_eq!( + arrow_schema.unwrap(), + Schema::new(vec![Field::new("", Utf8, false)]) + ); + } +} diff --git a/datafusion/src/error.rs b/datafusion/src/error.rs index 3f079859e34d..6b6bb1381111 100644 --- a/datafusion/src/error.rs +++ b/datafusion/src/error.rs @@ -23,6 +23,7 @@ use std::io; use std::result; use arrow::error::ArrowError; +#[cfg(feature = "avro")] use avro_rs::Error as AvroError; use parquet::errors::ParquetError; use sqlparser::parser::ParserError; @@ -39,6 +40,7 @@ pub enum DataFusionError { /// Wraps an error from the Parquet crate ParquetError(ParquetError), /// Wraps an error from the Avro crate + #[cfg(feature = "avro")] AvroError(AvroError), /// Error associated to I/O operations and associated traits. IoError(io::Error), @@ -86,6 +88,7 @@ impl From for DataFusionError { } } +#[cfg(feature = "avro")] impl From for DataFusionError { fn from(e: AvroError) -> Self { DataFusionError::AvroError(e) @@ -105,6 +108,7 @@ impl Display for DataFusionError { DataFusionError::ParquetError(ref desc) => { write!(f, "Parquet error: {}", desc) } + #[cfg(feature = "avro")] DataFusionError::AvroError(ref desc) => { write!(f, "Avro error: {}", desc) } diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index 0f82151881bf..da99536ddb61 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -16,23 +16,26 @@ // under the License. //! Execution plan for reading line-delimited Avro files -use async_trait::async_trait; -use futures::Stream; - -use super::{common, source::Source, ExecutionPlan, Partitioning, RecordBatchStream}; +#[cfg(feature = "avro")] +use super::RecordBatchStream; +use super::{common, source::Source, ExecutionPlan, Partitioning}; use crate::avro_to_arrow::infer_avro_schema_from_reader; use crate::error::{DataFusionError, Result}; -use arrow::{ - datatypes::{Schema, SchemaRef}, - error::Result as ArrowResult, - record_batch::RecordBatch, -}; +use arrow::datatypes::{Schema, SchemaRef}; +#[cfg(feature = "avro")] +use arrow::{error::Result as ArrowResult, record_batch::RecordBatch}; +use async_trait::async_trait; +#[cfg(feature = "avro")] +use futures::Stream; use std::fs::File; use std::{any::Any, io::Seek}; use std::{ io::Read, - pin::Pin, sync::{Arc, Mutex}, +}; +#[cfg(feature = "avro")] +use std::{ + pin::Pin, task::{Context, Poll}, }; @@ -252,6 +255,17 @@ impl ExecutionPlan for AvroExec { } } + #[cfg(not(feature = "avro"))] + async fn execute( + &self, + _partition: usize, + ) -> Result { + Err(DataFusionError::NotImplemented( + "Cannot execute avro plan without avro feature enabled".to_string(), + )) + } + + #[cfg(feature = "avro")] async fn execute( &self, partition: usize, @@ -292,11 +306,13 @@ impl ExecutionPlan for AvroExec { } } +#[cfg(feature = "avro")] struct AvroStream<'a, R: Read> { reader: crate::avro_to_arrow::Reader<'a, R>, remain: Option, } +#[cfg(feature = "avro")] impl<'a, R: Read> AvroStream<'a, R> { fn new(reader: crate::avro_to_arrow::Reader<'a, R>, limit: Option) -> Self { Self { @@ -306,6 +322,7 @@ impl<'a, R: Read> AvroStream<'a, R> { } } +#[cfg(feature = "avro")] impl Stream for AvroStream<'_, R> { type Item = ArrowResult; @@ -346,6 +363,7 @@ impl Stream for AvroStream<'_, R> { } } +#[cfg(feature = "avro")] impl RecordBatchStream for AvroStream<'_, R> { fn schema(&self) -> SchemaRef { self.reader.schema() From 060c6441f30b8f1a692ac2395738d06161cf1a16 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 13:11:00 +0200 Subject: [PATCH 31/58] Remove debug statement --- datafusion/src/logical_plan/dfschema.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/src/logical_plan/dfschema.rs b/datafusion/src/logical_plan/dfschema.rs index d2918421e4a0..b692526a2639 100644 --- a/datafusion/src/logical_plan/dfschema.rs +++ b/datafusion/src/logical_plan/dfschema.rs @@ -141,7 +141,6 @@ impl DFSchema { return Ok(i); } } - println!("{}", name); Err(DataFusionError::Plan(format!( "No field named '{}'. Valid fields are {}.", name, From 72ad35cd4a49d28c2449916ab66a8bf273190b39 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 13:14:21 +0200 Subject: [PATCH 32/58] Remove GetField usage (see #628) --- datafusion/src/lib.rs | 1 - datafusion/src/logical_plan/dfschema.rs | 7 +- datafusion/src/logical_plan/expr.rs | 40 ------- datafusion/src/optimizer/utils.rs | 6 -- .../physical_plan/expressions/get_field.rs | 100 ------------------ .../src/physical_plan/expressions/mod.rs | 2 - datafusion/src/physical_plan/planner.rs | 8 -- datafusion/src/sql/planner.rs | 35 +++--- datafusion/src/sql/utils.rs | 4 - datafusion/src/utils.rs | 45 -------- datafusion/tests/sql.rs | 25 ----- 11 files changed, 18 insertions(+), 255 deletions(-) delete mode 100644 datafusion/src/physical_plan/expressions/get_field.rs delete mode 100644 datafusion/src/utils.rs diff --git a/datafusion/src/lib.rs b/datafusion/src/lib.rs index 2452146ccb93..adaca114d2d6 100644 --- a/datafusion/src/lib.rs +++ b/datafusion/src/lib.rs @@ -225,7 +225,6 @@ pub mod physical_plan; pub mod prelude; pub mod scalar; pub mod sql; -mod utils; pub mod variable; // re-export dependencies from arrow-rs to minimise version maintenance for crate users diff --git a/datafusion/src/logical_plan/dfschema.rs b/datafusion/src/logical_plan/dfschema.rs index b692526a2639..1ef8ac764d46 100644 --- a/datafusion/src/logical_plan/dfschema.rs +++ b/datafusion/src/logical_plan/dfschema.rs @@ -25,7 +25,6 @@ use std::sync::Arc; use crate::error::{DataFusionError, Result}; use crate::logical_plan::Column; -use crate::utils::get_field; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use std::fmt::{Display, Formatter}; @@ -161,11 +160,7 @@ impl DFSchema { // field to lookup is qualified. // current field is qualified and not shared between relations, compare both // qualifer and name. - (Some(q), Some(field_q)) => { - (q == field_q && field.name() == name) - || (q == field.name() - && get_field(field.field.data_type(), name).is_ok()) - } + (Some(q), Some(field_q)) => q == field_q && field.name() == name, // field to lookup is qualified but current field is unqualified. (Some(_), None) => false, // field to lookup is unqualified, no need to compare qualifier diff --git a/datafusion/src/logical_plan/expr.rs b/datafusion/src/logical_plan/expr.rs index 5d69e609d979..eb46099dff64 100644 --- a/datafusion/src/logical_plan/expr.rs +++ b/datafusion/src/logical_plan/expr.rs @@ -25,7 +25,6 @@ use crate::physical_plan::{ aggregates, expressions::binary_operator_data_type, functions, udf::ScalarUDF, window_functions, }; -use crate::utils::get_field; use crate::{physical_plan::udaf::AggregateUDF, scalar::ScalarValue}; use aggregates::{AccumulatorFunctionImplementation, StateTypeFunction}; use arrow::{compute::can_cast_types, datatypes::DataType}; @@ -245,13 +244,6 @@ pub enum Expr { IsNull(Box), /// arithmetic negation of an expression, the operand must be of a signed numeric data type Negative(Box), - /// Returns the field of a [`StructArray`] by name - GetField { - /// the expression to take the field from - expr: Box, - /// The name of the field to take - name: String, - }, /// Whether an expression is between a given range. Between { /// The value to compare @@ -440,10 +432,6 @@ impl Expr { Expr::Wildcard => Err(DataFusionError::Internal( "Wildcard expressions are not valid in a logical query plan".to_owned(), )), - Expr::GetField { ref expr, name } => { - let data_type = expr.get_type(schema)?; - get_field(&data_type, name).map(|x| x.data_type().clone()) - } } } @@ -499,10 +487,6 @@ impl Expr { Expr::Wildcard => Err(DataFusionError::Internal( "Wildcard expressions are not valid in a logical query plan".to_owned(), )), - Expr::GetField { ref expr, name } => { - let data_type = expr.get_type(input_schema)?; - get_field(&data_type, name).map(|x| x.is_nullable()) - } } } @@ -643,14 +627,6 @@ impl Expr { Expr::IsNotNull(Box::new(self)) } - /// Returns the values of the field `name` from an expression returning a `Struct` - pub fn get_field>(self, name: I) -> Expr { - Expr::GetField { - expr: Box::new(self), - name: name.into(), - } - } - /// Create a sort expression from an existing expression. /// /// ``` @@ -786,7 +762,6 @@ impl Expr { .try_fold(visitor, |visitor, arg| arg.accept(visitor)) } Expr::Wildcard => Ok(visitor), - Expr::GetField { ref expr, .. } => expr.accept(visitor), }?; visitor.post_visit(self) @@ -944,10 +919,6 @@ impl Expr { negated, }, Expr::Wildcard => Expr::Wildcard, - Expr::GetField { expr, name } => Expr::GetField { - expr: rewrite_boxed(expr, rewriter)?, - name, - }, }; // now rewrite this expression itself @@ -1709,7 +1680,6 @@ impl fmt::Debug for Expr { } } Expr::Wildcard => write!(f, "*"), - Expr::GetField { ref expr, name } => write!(f, "({:?}).{}", expr, name), } } } @@ -1786,10 +1756,6 @@ fn create_name(e: &Expr, input_schema: &DFSchema) -> Result { let expr = create_name(expr, input_schema)?; Ok(format!("{} IS NOT NULL", expr)) } - Expr::GetField { expr, name } => { - let expr = create_name(expr, input_schema)?; - Ok(format!("{}.{}", expr, name)) - } Expr::ScalarFunction { fun, args, .. } => { create_function_name(&fun.to_string(), false, args, input_schema) } @@ -1911,12 +1877,6 @@ mod tests { ); } - #[test] - fn display_get_field() { - let col_null = col("col1").get_field("name"); - assert_eq!(format!("{:?}", col_null), "(#col1).name"); - } - #[derive(Default)] struct RecordingRewriter { v: Vec, diff --git a/datafusion/src/optimizer/utils.rs b/datafusion/src/optimizer/utils.rs index f7f4cb640138..435daefb88eb 100644 --- a/datafusion/src/optimizer/utils.rs +++ b/datafusion/src/optimizer/utils.rs @@ -79,7 +79,6 @@ impl ExpressionVisitor for ColumnNameVisitor<'_> { Expr::AggregateUDF { .. } => {} Expr::InList { .. } => {} Expr::Wildcard => {} - Expr::GetField { .. } => {} } Ok(Recursion::Continue(self)) } @@ -306,7 +305,6 @@ pub fn expr_sub_expressions(expr: &Expr) -> Result> { Expr::Wildcard { .. } => Err(DataFusionError::Internal( "Wildcard expressions are not valid in a logical query plan".to_owned(), )), - Expr::GetField { expr, .. } => Ok(vec![expr.as_ref().to_owned()]), } } @@ -322,10 +320,6 @@ pub fn rewrite_expression(expr: &Expr, expressions: &[Expr]) -> Result { }), Expr::IsNull(_) => Ok(Expr::IsNull(Box::new(expressions[0].clone()))), Expr::IsNotNull(_) => Ok(Expr::IsNotNull(Box::new(expressions[0].clone()))), - Expr::GetField { expr: _, name } => Ok(Expr::GetField { - expr: Box::new(expressions[0].clone()), - name: name.clone(), - }), Expr::ScalarFunction { fun, .. } => Ok(Expr::ScalarFunction { fun: fun.clone(), args: expressions.to_vec(), diff --git a/datafusion/src/physical_plan/expressions/get_field.rs b/datafusion/src/physical_plan/expressions/get_field.rs deleted file mode 100644 index da56a29f0777..000000000000 --- a/datafusion/src/physical_plan/expressions/get_field.rs +++ /dev/null @@ -1,100 +0,0 @@ -// 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. - -//! get field of a struct array - -use std::{any::Any, sync::Arc}; - -use arrow::{ - array::StructArray, - datatypes::{DataType, Schema}, - record_batch::RecordBatch, -}; - -use crate::{ - error::DataFusionError, - error::Result, - physical_plan::{ColumnarValue, PhysicalExpr}, - utils::get_field as get_data_type_field, -}; - -/// expression to get a field of a struct array. -#[derive(Debug)] -pub struct GetFieldExpr { - arg: Arc, - name: String, -} - -impl GetFieldExpr { - /// Create new get field expression - pub fn new(arg: Arc, name: String) -> Self { - Self { arg, name } - } - - /// Get the input expression - pub fn arg(&self) -> &Arc { - &self.arg - } -} - -impl std::fmt::Display for GetFieldExpr { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "({}).{}", self.arg, self.name) - } -} - -impl PhysicalExpr for GetFieldExpr { - fn as_any(&self) -> &dyn Any { - self - } - - fn data_type(&self, input_schema: &Schema) -> Result { - let data_type = self.arg.data_type(input_schema)?; - get_data_type_field(&data_type, &self.name).map(|f| f.data_type().clone()) - } - - fn nullable(&self, input_schema: &Schema) -> Result { - let data_type = self.arg.data_type(input_schema)?; - get_data_type_field(&data_type, &self.name).map(|f| f.is_nullable()) - } - - fn evaluate(&self, batch: &RecordBatch) -> Result { - let arg = self.arg.evaluate(batch)?; - match arg { - ColumnarValue::Array(array) => Ok(ColumnarValue::Array( - array - .as_any() - .downcast_ref::() - .unwrap() - .column_by_name(&self.name) - .unwrap() - .clone(), - )), - ColumnarValue::Scalar(_) => Err(DataFusionError::NotImplemented( - "field is not yet implemented for scalar values".to_string(), - )), - } - } -} - -/// Create an `.field` expression -pub fn get_field( - arg: Arc, - name: String, -) -> Result> { - Ok(Arc::new(GetFieldExpr::new(arg, name))) -} diff --git a/datafusion/src/physical_plan/expressions/mod.rs b/datafusion/src/physical_plan/expressions/mod.rs index d599c332d9d9..5a5a1189af05 100644 --- a/datafusion/src/physical_plan/expressions/mod.rs +++ b/datafusion/src/physical_plan/expressions/mod.rs @@ -33,7 +33,6 @@ mod cast; mod coercion; mod column; mod count; -mod get_field; mod in_list; mod is_not_null; mod is_null; @@ -63,7 +62,6 @@ pub use cast::{ }; pub use column::{col, Column}; pub use count::Count; -pub use get_field::{get_field, GetFieldExpr}; pub use in_list::{in_list, InListExpr}; pub use is_not_null::{is_not_null, IsNotNullExpr}; pub use is_null::{is_null, IsNullExpr}; diff --git a/datafusion/src/physical_plan/planner.rs b/datafusion/src/physical_plan/planner.rs index 0b994bb1d3c9..0ff595817e7c 100644 --- a/datafusion/src/physical_plan/planner.rs +++ b/datafusion/src/physical_plan/planner.rs @@ -138,10 +138,6 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { let expr = create_physical_name(expr, false)?; Ok(format!("{} IS NOT NULL", expr)) } - Expr::GetField { expr, name } => { - let expr = physical_name(expr, input_schema)?; - Ok(format!("{}.{}", expr, name)) - } Expr::ScalarFunction { fun, args, .. } => { create_function_physical_name(&fun.to_string(), false, args) } @@ -964,10 +960,6 @@ impl DefaultPhysicalPlanner { Expr::IsNotNull(expr) => expressions::is_not_null( self.create_physical_expr(expr, input_dfschema, input_schema, ctx_state)?, ), - Expr::GetField { expr, name } => expressions::get_field( - self.create_physical_expr(expr, input_dfschema, input_schema, ctx_state)?, - name.clone(), - ), Expr::ScalarFunction { fun, args } => { let physical_args = args .iter() diff --git a/datafusion/src/sql/planner.rs b/datafusion/src/sql/planner.rs index ccb5c8f903be..50c36dde5831 100644 --- a/datafusion/src/sql/planner.rs +++ b/datafusion/src/sql/planner.rs @@ -81,22 +81,6 @@ pub struct SqlToRel<'a, S: ContextProvider> { schema_provider: &'a S, } -fn plan_compound(mut identifiers: Vec) -> Expr { - if &identifiers[0][0..1] == "@" { - Expr::ScalarVariable(identifiers) - } else if identifiers.len() == 2 { - // "table.column" - let name = identifiers.pop().unwrap(); - let relation = Some(identifiers.pop().unwrap()); - Expr::Column(Column { relation, name }) - } else { - // "table.column.field..." - let name = identifiers.pop().unwrap(); - let expr = Box::new(plan_compound(identifiers)); - Expr::GetField { expr, name } - } -} - impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// Create a new query planner pub fn new(schema_provider: &'a S) -> Self { @@ -1110,8 +1094,23 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } SQLExpr::CompoundIdentifier(ids) => { - let var_names = ids.iter().map(|x| x.value.clone()).collect::>(); - Ok(plan_compound(var_names)) + let mut var_names = vec![]; + for id in ids { + var_names.push(id.value.clone()); + } + if &var_names[0][0..1] == "@" { + Ok(Expr::ScalarVariable(var_names)) + } else if var_names.len() == 2 { + // table.column identifier + let name = var_names.pop().unwrap(); + let relation = Some(var_names.pop().unwrap()); + Ok(Expr::Column(Column { relation, name })) + } else { + Err(DataFusionError::NotImplemented(format!( + "Unsupported compound identifier '{:?}'", + var_names, + ))) + } } SQLExpr::Wildcard => Ok(Expr::Wildcard), diff --git a/datafusion/src/sql/utils.rs b/datafusion/src/sql/utils.rs index cbd173f312ed..41bcd205800d 100644 --- a/datafusion/src/sql/utils.rs +++ b/datafusion/src/sql/utils.rs @@ -368,10 +368,6 @@ where Ok(expr.clone()) } Expr::Wildcard => Ok(Expr::Wildcard), - Expr::GetField { expr, name } => Ok(Expr::GetField { - expr: Box::new(clone_with_replacement(expr.as_ref(), replacement_fn)?), - name: name.clone(), - }), }, } } diff --git a/datafusion/src/utils.rs b/datafusion/src/utils.rs deleted file mode 100644 index 0d51e95091b9..000000000000 --- a/datafusion/src/utils.rs +++ /dev/null @@ -1,45 +0,0 @@ -// 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. - -use arrow::datatypes::{DataType, Field}; - -use crate::error::{DataFusionError, Result}; - -/// Returns the first field named `name` from the fields of a [`DataType::Struct`]. -/// # Error -/// Errors iff -/// * the `data_type` is not a Struct or, -/// * there is no field named `name` -pub fn get_field<'a>(data_type: &'a DataType, name: &str) -> Result<&'a Field> { - match data_type { - DataType::Struct(fields) | DataType::Union(fields) => { - let maybe_field = fields.iter().find(|x| x.name() == name); - if let Some(field) = maybe_field { - Ok(field) - } else { - Err(DataFusionError::Plan(format!( - "The `Struct` has no field named \"{}\"", - name - ))) - } - } - _ => Err(DataFusionError::Plan( - "The expression to get a field is only valid for `Struct` or 'Union'" - .to_string(), - )), - } -} diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index 3f22ad1d0390..dc5058fdcd73 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -3340,31 +3340,6 @@ async fn query_is_not_null() -> Result<()> { Ok(()) } -#[tokio::test] -async fn query_get_field() -> Result<()> { - let inner_field = Field::new("inner", DataType::Float64, true); - let field = Field::new("c1", DataType::Struct(vec![inner_field.clone()]), true); - let schema = Arc::new(Schema::new(vec![field])); - - let array = Arc::new(Float64Array::from(vec![Some(1.1), None])) as ArrayRef; - - let data = RecordBatch::try_new( - schema.clone(), - vec![Arc::new(StructArray::from(vec![(inner_field, array)]))], - )?; - - let table = MemTable::try_new(schema, vec![vec![data]])?; - - let mut ctx = ExecutionContext::new(); - ctx.register_table("test", Arc::new(table))?; - let sql = "SELECT test.c1.inner FROM test"; - let actual = execute(&mut ctx, sql).await; - let expected = vec![vec!["1.1"], vec!["NULL"]]; - - assert_eq!(expected, actual); - Ok(()) -} - #[tokio::test] async fn query_count_distinct() -> Result<()> { let schema = Arc::new(Schema::new(vec![Field::new("c1", DataType::Int32, true)])); From 3c8f6ce2f9d024cc395de75634ed7c7f3ed7720b Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 13:22:32 +0200 Subject: [PATCH 33/58] Fix docstring in parser tests --- datafusion/src/sql/parser.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/src/sql/parser.rs b/datafusion/src/sql/parser.rs index 8e70c7c15883..864801c00f72 100644 --- a/datafusion/src/sql/parser.rs +++ b/datafusion/src/sql/parser.rs @@ -393,7 +393,7 @@ mod tests { }); expect_parse_ok(sql, expected)?; - // positive case: it is ok for parquet files not to have columns specified + // positive case: it is ok for avro files not to have columns specified let sql = "CREATE EXTERNAL TABLE t STORED AS AVRO LOCATION 'foo.avro'"; let expected = Statement::CreateExternalTable(CreateExternalTable { name: "t".into(), From 0db3013dc068eaf3a6e01282668fe58fbf83edbf Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 13:43:05 +0200 Subject: [PATCH 34/58] Test batch output rather than just rows individually --- datafusion/src/datasource/avro.rs | 21 +++++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index a908ea644282..28e63c972354 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -209,6 +209,7 @@ mod tests { Ok(()) } + #[cfg(feature = "avro")] #[tokio::test] async fn read_alltypes_plain_avro() -> Result<()> { let table = load_table("alltypes_plain.avro")?; @@ -237,10 +238,22 @@ mod tests { let projection = None; let batch = get_first_batch(table, &projection).await?; - - assert_eq!(11, batch.num_columns()); - assert_eq!(8, batch.num_rows()); - + let expected = vec![ + "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", + "| id | bool_col | tinyint_col | smallint_col | int_col | bigint_col | float_col | double_col | date_string_col | string_col | timestamp_col |", + "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", + "| 4 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30332f30312f3039 | 30 | 2009-03-01 00:00:00 |", + "| 5 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30332f30312f3039 | 31 | 2009-03-01 00:01:00 |", + "| 6 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30342f30312f3039 | 30 | 2009-04-01 00:00:00 |", + "| 7 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30342f30312f3039 | 31 | 2009-04-01 00:01:00 |", + "| 2 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30322f30312f3039 | 30 | 2009-02-01 00:00:00 |", + "| 3 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30322f30312f3039 | 31 | 2009-02-01 00:01:00 |", + "| 0 | true | 0 | 0 | 0 | 0 | 0 | 0 | 30312f30312f3039 | 30 | 2009-01-01 00:00:00 |", + "| 1 | false | 1 | 1 | 1 | 10 | 1.1 | 10.1 | 30312f30312f3039 | 31 | 2009-01-01 00:01:00 |", + "+----+----------+-------------+--------------+---------+------------+-----------+------------+------------------+------------+---------------------+", + ]; + + crate::assert_batches_eq!(expected, &[batch]); Ok(()) } From faa215278ec2983bfefeb32bfcf5e515ca4b7232 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 13:48:41 +0200 Subject: [PATCH 35/58] Remove 'csv' from error strings in physical_plan::avro --- datafusion/src/physical_plan/avro.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index da99536ddb61..81be1edfa185 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -290,14 +290,14 @@ impl ExecutionPlan for AvroExec { Source::Reader(rdr) => { if partition != 0 { Err(DataFusionError::Internal( - "Only partition 0 is valid when CSV comes from a reader" + "Only partition 0 is valid when Avro comes from a reader" .to_string(), )) } else if let Some(rdr) = rdr.lock().unwrap().take() { Ok(Box::pin(AvroStream::new(builder.build(rdr)?, self.limit))) } else { Err(DataFusionError::Execution( - "Error reading CSV: Data can only be read a single time when the source is a reader" + "Error reading Avro: Data can only be read a single time when the source is a reader" .to_string(), )) } From a6cd7f10617ffb91a8d12189265de5f7c5422114 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 14:30:48 +0200 Subject: [PATCH 36/58] Avro sample sql and explain queries tests in sql.rs --- datafusion/tests/sql.rs | 120 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 120 insertions(+) diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index dc5058fdcd73..99ceafce50ac 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -35,6 +35,7 @@ use arrow::{ use datafusion::assert_batches_eq; use datafusion::assert_batches_sorted_eq; use datafusion::logical_plan::LogicalPlan; +use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::ExecutionPlanVisitor; @@ -2934,6 +2935,16 @@ fn register_alltypes_parquet(ctx: &mut ExecutionContext) { .unwrap(); } +fn register_alltypes_avro(ctx: &mut ExecutionContext) { + let testdata = datafusion::test_util::arrow_test_data(); + ctx.register_avro( + "alltypes_plain", + &format!("{}/avro/alltypes_plain.avro", testdata), + AvroReadOptions::default(), + ) + .unwrap(); +} + /// Execute query and return result set as 2-d table of Vecs /// `result[row][column]` async fn execute_to_batches(ctx: &mut ExecutionContext, sql: &str) -> Vec { @@ -4659,3 +4670,112 @@ async fn test_regexp_is_match() -> Result<()> { assert_batches_eq!(expected, &actual); Ok(()) } + +#[cfg(feature = "avro")] +#[tokio::test] +async fn avro_query() { + let mut ctx = ExecutionContext::new(); + register_alltypes_avro(&mut ctx); + // NOTE that string_col is actually a binary column and does not have the UTF8 logical type + // so we need an explicit cast + let sql = "SELECT id, CAST(string_col AS varchar) FROM alltypes_plain"; + let actual = execute_to_batches(&mut ctx, sql).await; + let expected = vec![ + "+----+--------------------------+", + "| id | CAST(string_col AS Utf8) |", + "+----+--------------------------+", + "| 4 | 0 |", + "| 5 | 1 |", + "| 6 | 0 |", + "| 7 | 1 |", + "| 2 | 0 |", + "| 3 | 1 |", + "| 0 | 0 |", + "| 1 | 1 |", + "+----+--------------------------+", + ]; + + assert_batches_eq!(expected, &actual); +} + +#[cfg(feature = "avro")] +#[tokio::test] +async fn avro_single_nan_schema() { + let mut ctx = ExecutionContext::new(); + let testdata = datafusion::test_util::arrow_test_data(); + ctx.register_avro( + "single_nan", + &format!("{}/avro/single_nan.avro", testdata), + AvroReadOptions::default(), + ) + .unwrap(); + let sql = "SELECT mycol FROM single_nan"; + let plan = ctx.create_logical_plan(sql).unwrap(); + let plan = ctx.optimize(&plan).unwrap(); + let plan = ctx.create_physical_plan(&plan).unwrap(); + let results = collect(plan).await.unwrap(); + for batch in results { + assert_eq!(1, batch.num_rows()); + assert_eq!(1, batch.num_columns()); + } +} + +#[cfg(feature = "avro")] +#[tokio::test] +async fn avro_explain() { + let mut ctx = ExecutionContext::new(); + register_alltypes_avro(&mut ctx); + + let sql = "EXPLAIN SELECT count(*) from alltypes_plain"; + let actual = execute(&mut ctx, sql).await; + let actual = normalize_vec_for_explain(actual); + let expected = vec![ + vec![ + "logical_plan", + "Projection: #COUNT(UInt8(1))\ + \n Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ + \n TableScan: alltypes_plain projection=Some([0])", + ], + vec![ + "physical_plan", + "ProjectionExec: expr=[COUNT(UInt8(1))@0 as COUNT(UInt8(1))]\ + \n HashAggregateExec: mode=Final, gby=[], aggr=[COUNT(UInt8(1))]\ + \n CoalescePartitionsExec\ + \n HashAggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]\ + \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES)\ + \n ExecutionPlan(PlaceHolder)\ + \n", + ], + ]; + assert_eq!(expected, actual); +} + +#[cfg(feature = "avro")] +#[tokio::test] +async fn avro_explain_analyze() { + // This test uses the execute function to run an actual plan under EXPLAIN ANALYZE + let mut ctx = ExecutionContext::new(); + register_alltypes_avro(&mut ctx); + + let sql = "EXPLAIN ANALYZE SELECT count(*), tinyint_col from alltypes_plain group by tinyint_col"; + let actual = execute_to_batches(&mut ctx, sql).await; + let formatted = arrow::util::pretty::pretty_format_batches(&actual).unwrap(); + let formatted = normalize_for_explain(&formatted); + + // Only test basic plumbing and try to avoid having to change too + // many things + let needle = "RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), metrics=["; + assert!( + formatted.contains(needle), + "did not find '{}' in\n{}", + needle, + formatted + ); + let verbose_needle = "Output Rows | 5"; + assert!( + !formatted.contains(verbose_needle), + "found unexpected '{}' in\n{}", + verbose_needle, + formatted + ); +} From 8b4fcaee6f470b4f9f76ccc2af2977220c90b37b Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 14:54:59 +0200 Subject: [PATCH 37/58] Activate avro feature for cargo tests in github workflow --- .github/workflows/rust.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 1b0386be7e63..b6fc9b1d4c43 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -105,8 +105,8 @@ jobs: run: | export ARROW_TEST_DATA=$(pwd)/testing/data export PARQUET_TEST_DATA=$(pwd)/parquet-testing/data - # run tests on all workspace members with default feature list - cargo test + # run tests on all workspace members with default feature list + avro + cargo test --features=avro # test datafusion examples cd datafusion-examples cargo test --no-default-features From b2b7915f3082b85971daead4a7b38d7758241db2 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 15:19:51 +0200 Subject: [PATCH 38/58] Add a test for avro registering multiple files in a single table --- datafusion/tests/sql.rs | 55 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index 99ceafce50ac..5c68b806c198 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -4698,6 +4698,61 @@ async fn avro_query() { assert_batches_eq!(expected, &actual); } +#[cfg(feature = "avro")] +#[tokio::test] +async fn avro_query_multiple_files() { + let tempdir = tempfile::tempdir().unwrap(); + let table_path = tempdir.path(); + let testdata = datafusion::test_util::arrow_test_data(); + let alltypes_plain_file = format!("{}/avro/alltypes_plain.avro", testdata); + std::fs::copy( + &alltypes_plain_file, + format!("{}/alltypes_plain1.avro", table_path.display()), + ) + .unwrap(); + std::fs::copy( + &alltypes_plain_file, + format!("{}/alltypes_plain2.avro", table_path.display()), + ) + .unwrap(); + + let mut ctx = ExecutionContext::new(); + ctx.register_avro( + "alltypes_plain", + table_path.display().to_string().as_str(), + AvroReadOptions::default(), + ) + .unwrap(); + // NOTE that string_col is actually a binary column and does not have the UTF8 logical type + // so we need an explicit cast + let sql = "SELECT id, CAST(string_col AS varchar) FROM alltypes_plain"; + let actual = execute_to_batches(&mut ctx, sql).await; + let expected = vec![ + "+----+--------------------------+", + "| id | CAST(string_col AS Utf8) |", + "+----+--------------------------+", + "| 4 | 0 |", + "| 5 | 1 |", + "| 6 | 0 |", + "| 7 | 1 |", + "| 2 | 0 |", + "| 3 | 1 |", + "| 0 | 0 |", + "| 1 | 1 |", + "| 4 | 0 |", + "| 5 | 1 |", + "| 6 | 0 |", + "| 7 | 1 |", + "| 2 | 0 |", + "| 3 | 1 |", + "| 0 | 0 |", + "| 1 | 1 |", + "+----+--------------------------+", + ]; + + assert_batches_eq!(expected, &actual); +} + #[cfg(feature = "avro")] #[tokio::test] async fn avro_single_nan_schema() { From 8acf0623d2d899ac08ee4aa487e441ab2c01f1c6 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 15:41:20 +0200 Subject: [PATCH 39/58] Switch to Result instead of Option for resolve_string --- .../src/avro_to_arrow/arrow_array_reader.rs | 41 ++++++++----------- 1 file changed, 18 insertions(+), 23 deletions(-) diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index b8d4d19c6c10..359d443d5512 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -288,22 +288,15 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { vec![Some(v.to_string())] } else if let Value::Array(n) = value { n.into_iter() - .map(|v| { - resolve_string(&v) - // else if matches!( - // v, - // Value::Array(_) | Value::Record(_) | Value::Null - // ) { - // // implicitly drop nested values - // // TODO support deep-nesting - // None - // } - }) - .collect() + .map(|v| resolve_string(&v)) + .collect::>>()? + .into_iter() + .map(Some) + .collect::>>() } else if let Value::Null = value { vec![None] } else if !matches!(value, Value::Record(_)) { - vec![resolve_string(&value)] + vec![Some(resolve_string(&value)?)] } else { return Err(SchemaError( "Only scalars are currently supported in Avro arrays".to_string(), @@ -373,7 +366,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { self.build_string_dictionary_builder(rows.len())?; for row in rows { if let Some(value) = self.field_lookup(col_name, row) { - if let Some(str_v) = resolve_string(&value) { + if let Ok(str_v) = resolve_string(&value) { builder.append(str_v).map(drop)? } else { builder.append_null()? @@ -712,9 +705,11 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { rows.iter() .map(|row| { let maybe_value = self.field_lookup(field.name(), row); - maybe_value.and_then(|value| resolve_string(&value)) + maybe_value + .map(|value| resolve_string(&value)) + .transpose() }) - .collect::(), + .collect::>()?, ) as ArrayRef), DataType::Binary | DataType::LargeBinary => Ok(Arc::new( @@ -859,12 +854,12 @@ fn flatten_string_values(values: &[Value]) -> Vec> { if let Value::Array(values) = row { values .iter() - .map(resolve_string) + .map(|s| resolve_string(s).ok()) .collect::>>() } else if let Value::Null = row { vec![] } else { - vec![resolve_string(row)] + vec![resolve_string(row).ok()] } }) .collect::>>() @@ -873,16 +868,16 @@ fn flatten_string_values(values: &[Value]) -> Vec> { /// Reads an Avro value as a string, regardless of its type. /// This is useful if the expected datatype is a string, in which case we preserve /// all the values regardless of they type. -fn resolve_string(v: &Value) -> Option { +fn resolve_string(v: &Value) -> ArrowResult { let v = if let Value::Union(b) = v { b } else { v }; match v { Value::String(s) => Ok(s.clone()), - Value::Bytes(bytes) => Ok(String::from_utf8(bytes.to_vec()) - .map_err(AvroError::ConvertToUtf8) - .ok()?), + Value::Bytes(bytes) => { + String::from_utf8(bytes.to_vec()).map_err(AvroError::ConvertToUtf8) + } other => Err(AvroError::GetString(other.into())), } - .ok() + .map_err(|e| SchemaError(format!("expected resolvable string : {}", e))) } fn resolve_u8(v: Value) -> AvroResult { From 65c9b9ed8d87ede84470a525c195994b68663086 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 26 Aug 2021 15:47:10 +0200 Subject: [PATCH 40/58] Address missing clippy warning should_implement_trait in arrow_to_avro/reader --- datafusion/src/avro_to_arrow/reader.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index 17b39a1ceb1f..c316122b8e21 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -155,6 +155,7 @@ impl<'a, R: Read> Reader<'a, R> { /// Returns the next batch of results (defined by `self.batch_size`), or `None` if there /// are no more results + #[allow(clippy::should_implement_trait)] pub fn next(&mut self) -> ArrowResult> { self.array_reader.next_batch(self.batch_size) } From 1668b96ce69c939f7d096a4aa2e7939518dd2cfb Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Mon, 30 Aug 2021 05:39:10 +0200 Subject: [PATCH 41/58] Add fmt display implementation for AvroExec --- datafusion/src/physical_plan/avro.rs | 17 +++++++++++++ datafusion/src/physical_plan/source.rs | 2 +- datafusion/tests/sql.rs | 34 +++----------------------- 3 files changed, 21 insertions(+), 32 deletions(-) diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index 81be1edfa185..c7536b0dd6c6 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -21,6 +21,7 @@ use super::RecordBatchStream; use super::{common, source::Source, ExecutionPlan, Partitioning}; use crate::avro_to_arrow::infer_avro_schema_from_reader; use crate::error::{DataFusionError, Result}; +use crate::physical_plan::DisplayFormatType; use arrow::datatypes::{Schema, SchemaRef}; #[cfg(feature = "avro")] use arrow::{error::Result as ArrowResult, record_batch::RecordBatch}; @@ -304,6 +305,22 @@ impl ExecutionPlan for AvroExec { } } } + + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default => { + write!( + f, + "AvroExec: source={}, batch_size={}, limit={:?}", + self.source, self.batch_size, self.limit + ) + } + } + } } #[cfg(feature = "avro")] diff --git a/datafusion/src/physical_plan/source.rs b/datafusion/src/physical_plan/source.rs index 012405a38a1a..32fa9c37c8a2 100644 --- a/datafusion/src/physical_plan/source.rs +++ b/datafusion/src/physical_plan/source.rs @@ -46,7 +46,7 @@ impl std::fmt::Debug for Source { Ok(()) } } -impl std::fmt::Display for Source { +impl std::fmt::Display for Source { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { Source::PartitionedFiles { path, filenames } => { diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index 5c68b806c198..e257c7b24979 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -35,6 +35,7 @@ use arrow::{ use datafusion::assert_batches_eq; use datafusion::assert_batches_sorted_eq; use datafusion::logical_plan::LogicalPlan; +#[cfg(feature = "avro")] use datafusion::physical_plan::avro::AvroReadOptions; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; @@ -2935,6 +2936,7 @@ fn register_alltypes_parquet(ctx: &mut ExecutionContext) { .unwrap(); } +#[cfg(feature = "avro")] fn register_alltypes_avro(ctx: &mut ExecutionContext) { let testdata = datafusion::test_util::arrow_test_data(); ctx.register_avro( @@ -4798,39 +4800,9 @@ async fn avro_explain() { \n CoalescePartitionsExec\ \n HashAggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]\ \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES)\ - \n ExecutionPlan(PlaceHolder)\ + \n AvroExec: source=Path(ARROW_TEST_DATA/avro/alltypes_plain.avro: [ARROW_TEST_DATA/avro/alltypes_plain.avro]), batch_size=8192, limit=None\ \n", ], ]; assert_eq!(expected, actual); } - -#[cfg(feature = "avro")] -#[tokio::test] -async fn avro_explain_analyze() { - // This test uses the execute function to run an actual plan under EXPLAIN ANALYZE - let mut ctx = ExecutionContext::new(); - register_alltypes_avro(&mut ctx); - - let sql = "EXPLAIN ANALYZE SELECT count(*), tinyint_col from alltypes_plain group by tinyint_col"; - let actual = execute_to_batches(&mut ctx, sql).await; - let formatted = arrow::util::pretty::pretty_format_batches(&actual).unwrap(); - let formatted = normalize_for_explain(&formatted); - - // Only test basic plumbing and try to avoid having to change too - // many things - let needle = "RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), metrics=["; - assert!( - formatted.contains(needle), - "did not find '{}' in\n{}", - needle, - formatted - ); - let verbose_needle = "Output Rows | 5"; - assert!( - !formatted.contains(verbose_needle), - "found unexpected '{}' in\n{}", - verbose_needle, - formatted - ); -} From c9218e2de2d9ab684f369b2e6db1011ed38a2b7c Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Mon, 30 Aug 2021 05:40:49 +0200 Subject: [PATCH 42/58] ci: fix cargo sql run example, use datafusion/avro feature instead of 'avro' --- .github/workflows/rust.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index b6fc9b1d4c43..d48233c73361 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -112,7 +112,7 @@ jobs: cargo test --no-default-features cargo run --example csv_sql cargo run --example parquet_sql - cargo run --example avro_sql --features=avro + cargo run --example avro_sql --features=datafusion/avro env: CARGO_HOME: "/github/home/.cargo" CARGO_TARGET_DIR: "/github/home/target" From be09a646b6083d7ddef4af053804ac15416d5c89 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Mon, 30 Aug 2021 05:48:57 +0200 Subject: [PATCH 43/58] license: missing license file for avro_to_arrow/schema.rs --- datafusion/src/avro_to_arrow/schema.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/datafusion/src/avro_to_arrow/schema.rs b/datafusion/src/avro_to_arrow/schema.rs index 5706e6bba91b..351a510ba3fa 100644 --- a/datafusion/src/avro_to_arrow/schema.rs +++ b/datafusion/src/avro_to_arrow/schema.rs @@ -1,3 +1,20 @@ +// 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. + use crate::arrow::datatypes::{DataType, IntervalUnit, Schema, TimeUnit}; use crate::error::{DataFusionError, Result}; use arrow::datatypes::Field; From 68df26832cf483ff32d6ea6ee0187b1213c61cfb Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 31 Aug 2021 17:24:29 +0200 Subject: [PATCH 44/58] only run avro datasource tests if features have 'avro' --- datafusion/src/datasource/avro.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index 28e63c972354..b094edb7caed 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -180,6 +180,7 @@ impl TableProvider for AvroFile { } #[cfg(test)] +#[cfg(feature = "avro")] mod tests { use arrow::array::{ BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, From 09c4ecb899ee784d57880376a9131f12b1c90d34 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 2 Sep 2021 18:37:42 +0200 Subject: [PATCH 45/58] refactor: rename infer_avro_schema_from_reader to read_avro_schema_from_reader --- datafusion/src/avro_to_arrow/arrow_array_reader.rs | 2 +- datafusion/src/avro_to_arrow/mod.rs | 10 +++++----- datafusion/src/avro_to_arrow/reader.rs | 11 +++++------ datafusion/src/datasource/avro.rs | 10 +++++----- datafusion/src/physical_plan/avro.rs | 10 +++++----- 5 files changed, 21 insertions(+), 22 deletions(-) diff --git a/datafusion/src/avro_to_arrow/arrow_array_reader.rs b/datafusion/src/avro_to_arrow/arrow_array_reader.rs index 359d443d5512..cc8ed8e66942 100644 --- a/datafusion/src/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/src/avro_to_arrow/arrow_array_reader.rs @@ -979,7 +979,7 @@ mod test { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/{}", testdata, name); let builder = ReaderBuilder::new() - .infer_schema() + .read_schema() .with_batch_size(batch_size); builder.build(File::open(filename).unwrap()).unwrap() } diff --git a/datafusion/src/avro_to_arrow/mod.rs b/datafusion/src/avro_to_arrow/mod.rs index e0c05b277465..531b1092e1d6 100644 --- a/datafusion/src/avro_to_arrow/mod.rs +++ b/datafusion/src/avro_to_arrow/mod.rs @@ -31,17 +31,17 @@ pub use reader::{Reader, ReaderBuilder}; use std::io::{Read, Seek}; #[cfg(feature = "avro")] -/// Infer Avro schema given a reader -pub fn infer_avro_schema_from_reader(reader: &mut R) -> Result { +/// Read Avro schema given a reader +pub fn read_avro_schema_from_reader(reader: &mut R) -> Result { let avro_reader = avro_rs::Reader::new(reader)?; let schema = avro_reader.writer_schema(); schema::to_arrow_schema(schema) } #[cfg(not(feature = "avro"))] -/// Infer Avro schema given a reader (requires the avro feature) -pub fn infer_avro_schema_from_reader(_: &mut R) -> Result { +/// 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 infer avro schema without the 'avro' feature enabled".to_string(), + "cannot read avro schema without the 'avro' feature enabled".to_string(), )) } diff --git a/datafusion/src/avro_to_arrow/reader.rs b/datafusion/src/avro_to_arrow/reader.rs index c316122b8e21..8baad14746d3 100644 --- a/datafusion/src/avro_to_arrow/reader.rs +++ b/datafusion/src/avro_to_arrow/reader.rs @@ -28,8 +28,7 @@ use std::sync::Arc; pub struct ReaderBuilder { /// Optional schema for the Avro file /// - /// If the schema is not supplied, the reader will try to infer the schema - /// based on the Avro structure. + /// If the schema is not supplied, the reader will try to read the schema. schema: Option, /// Batch size (number of records to load each time) /// @@ -65,7 +64,7 @@ impl ReaderBuilder { /// let file = File::open("test/data/basic.avro").unwrap(); /// /// // create a builder, inferring the schema with the first 100 records - /// let builder = crate::datafusion::avro_to_arrow::ReaderBuilder::new().infer_schema().with_batch_size(100); + /// let builder = crate::datafusion::avro_to_arrow::ReaderBuilder::new().read_schema().with_batch_size(100); /// /// let reader = builder.build::(file).unwrap(); /// @@ -83,7 +82,7 @@ impl ReaderBuilder { } /// Set the Avro reader to infer the schema of the file - pub fn infer_schema(mut self) -> Self { + pub fn read_schema(mut self) -> Self { // remove any schema that is set self.schema = None; self @@ -111,7 +110,7 @@ impl ReaderBuilder { // check if schema should be inferred let schema = match self.schema { Some(schema) => schema, - None => Arc::new(super::infer_avro_schema_from_reader(&mut source)?), + None => Arc::new(super::read_avro_schema_from_reader(&mut source)?), }; source.seek(SeekFrom::Start(0))?; Reader::try_new(source, schema, self.batch_size, self.projection) @@ -180,7 +179,7 @@ mod tests { fn build_reader(name: &str) -> Reader { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/{}", testdata, name); - let builder = ReaderBuilder::new().infer_schema().with_batch_size(64); + let builder = ReaderBuilder::new().read_schema().with_batch_size(64); builder.build(File::open(filename).unwrap()).unwrap() } diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index b094edb7caed..7b39bedae478 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -50,14 +50,14 @@ pub struct AvroFile { } impl AvroFile { - /// Attempt to initialize a `AvroFile` from a path. The schema can be inferred automatically. + /// Attempt to initialize a `AvroFile` from a path. The schema can be read automatically. pub fn try_new(path: &str, options: AvroReadOptions) -> Result { let schema = if let Some(schema) = options.schema { schema } else { let filenames = common::build_checked_file_list(path, options.file_extension)?; - Arc::new(AvroExec::try_infer_schema(&filenames)?) + Arc::new(AvroExec::try_read_schema(&filenames)?) }; Ok(Self { @@ -89,8 +89,8 @@ impl AvroFile { }) } - /// Attempt to initialize an AvroFile from a reader impls Seek. The schema can be inferred automatically. - pub fn try_new_from_reader_infer_schema( + /// Attempt to initialize an AvroFile from a reader impls Seek. The schema can be read automatically. + pub fn try_new_from_reader_schema( mut reader: R, options: AvroReadOptions, ) -> Result { @@ -98,7 +98,7 @@ impl AvroFile { if let Some(schema) = options.schema { schema } else { - Arc::new(crate::avro_to_arrow::infer_avro_schema_from_reader( + Arc::new(crate::avro_to_arrow::read_avro_schema_from_reader( &mut reader, )?) } diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index c7536b0dd6c6..54e47d1c6030 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -19,7 +19,7 @@ #[cfg(feature = "avro")] use super::RecordBatchStream; use super::{common, source::Source, ExecutionPlan, Partitioning}; -use crate::avro_to_arrow::infer_avro_schema_from_reader; +use crate::avro_to_arrow::read_avro_schema_from_reader; use crate::error::{DataFusionError, Result}; use crate::physical_plan::DisplayFormatType; use arrow::datatypes::{Schema, SchemaRef}; @@ -98,7 +98,7 @@ impl AvroExec { let schema = match options.schema { Some(s) => s, - None => Arc::new(AvroExec::try_infer_schema(filenames.as_slice())?), + None => Arc::new(AvroExec::try_read_schema(filenames.as_slice())?), }; let projected_schema = match &projection { @@ -192,12 +192,12 @@ impl AvroExec { self.limit } - /// Infer schema for given Avro dataset - pub fn try_infer_schema(filenames: &[String]) -> Result { + /// Read schema for given Avro dataset + pub fn try_read_schema(filenames: &[String]) -> Result { let mut schemas = Vec::new(); for filename in filenames { let mut file = File::open(filename)?; - let schema = infer_avro_schema_from_reader(&mut file)?; + let schema = read_avro_schema_from_reader(&mut file)?; schemas.push(schema); } From 94b9dcb8db61a1147260d264835806aeabb4e1a5 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 2 Sep 2021 18:45:30 +0200 Subject: [PATCH 46/58] Pass None as props to avro schema schema_to_field_with_props until further notice --- datafusion/src/avro_to_arrow/schema.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/src/avro_to_arrow/schema.rs b/datafusion/src/avro_to_arrow/schema.rs index 351a510ba3fa..077a08e769af 100644 --- a/datafusion/src/avro_to_arrow/schema.rs +++ b/datafusion/src/avro_to_arrow/schema.rs @@ -50,7 +50,7 @@ fn schema_to_field( name: Option<&str>, nullable: bool, ) -> Result { - schema_to_field_with_props(schema, name, nullable, Some(&Default::default())) + schema_to_field_with_props(schema, name, nullable, None) } fn schema_to_field_with_props( From a03a3c2f5f55af4e10ee96a51066a6a29e487201 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 2 Sep 2021 18:46:40 +0200 Subject: [PATCH 47/58] Change schema inferance to FixedSizeBinary(16) for Uuid --- datafusion/src/avro_to_arrow/schema.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/src/avro_to_arrow/schema.rs b/datafusion/src/avro_to_arrow/schema.rs index 077a08e769af..648227034ab7 100644 --- a/datafusion/src/avro_to_arrow/schema.rs +++ b/datafusion/src/avro_to_arrow/schema.rs @@ -140,7 +140,7 @@ fn schema_to_field_with_props( AvroSchema::Decimal { precision, scale, .. } => DataType::Decimal(*precision, *scale), - AvroSchema::Uuid => DataType::Utf8, + AvroSchema::Uuid => DataType::FixedSizeBinary(16), AvroSchema::Date => DataType::Date32, AvroSchema::TimeMillis => DataType::Time32(TimeUnit::Millisecond), AvroSchema::TimeMicros => DataType::Time64(TimeUnit::Microsecond), From 405c63ae5ec81f7f26e953333223f17b099648b9 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Thu, 2 Sep 2021 18:51:28 +0200 Subject: [PATCH 48/58] schema: prefix metadata coming from avro with 'avro' --- datafusion/src/avro_to_arrow/schema.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/datafusion/src/avro_to_arrow/schema.rs b/datafusion/src/avro_to_arrow/schema.rs index 648227034ab7..f27956f25e45 100644 --- a/datafusion/src/avro_to_arrow/schema.rs +++ b/datafusion/src/avro_to_arrow/schema.rs @@ -112,7 +112,7 @@ fn schema_to_field_with_props( .map(|field| { let mut props = BTreeMap::new(); if let Some(doc) = &field.doc { - props.insert("doc".to_string(), doc.clone()); + props.insert("avro::doc".to_string(), doc.clone()); } /*if let Some(aliases) = fields.aliases { props.insert("aliases", aliases); @@ -238,7 +238,7 @@ fn external_props(schema: &AvroSchema) -> BTreeMap { | AvroSchema::Enum { doc: Some(ref doc), .. } => { - props.insert("doc".to_string(), doc.clone()); + props.insert("avro::doc".to_string(), doc.clone()); } _ => {} } @@ -274,7 +274,10 @@ fn external_props(schema: &AvroSchema) -> BTreeMap { .iter() .map(|alias| aliased(alias, namespace.as_deref(), None)) .collect(); - props.insert("aliases".to_string(), format!("[{}]", aliases.join(","))); + props.insert( + "avro::aliases".to_string(), + format!("[{}]", aliases.join(",")), + ); } _ => {} } From 291637a8b9cacbd23fe34b4f6c7aa1ef80b6ca18 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 7 Sep 2021 10:08:56 +0200 Subject: [PATCH 49/58] make num traits optional and part of the avro feature flag --- datafusion/Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/Cargo.toml b/datafusion/Cargo.toml index 1d965680e1a4..c1998bea13bb 100644 --- a/datafusion/Cargo.toml +++ b/datafusion/Cargo.toml @@ -45,7 +45,7 @@ unicode_expressions = ["unicode-segmentation"] # Used for testing ONLY: causes all values to hash to the same value (test for collisions) force_hash_collisions = [] # Used to enable the avro format -avro = ["avro-rs"] +avro = ["avro-rs", "num-traits"] [dependencies] ahash = "0.7" @@ -71,7 +71,7 @@ lazy_static = { version = "^1.4.0", optional = true } smallvec = { version = "1.6", features = ["union"] } rand = "0.8" avro-rs = { version = "0.13", features = ["snappy"], optional = true } -num-traits = "0.2" +num-traits = { version = "0.2", optional = true } [dev-dependencies] criterion = "0.3" From 00f109c5504dccafbfd60a810c300698cb65b678 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 7 Sep 2021 10:15:54 +0200 Subject: [PATCH 50/58] Fix avro schema tests regarding external props --- datafusion/src/avro_to_arrow/schema.rs | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/datafusion/src/avro_to_arrow/schema.rs b/datafusion/src/avro_to_arrow/schema.rs index f27956f25e45..c2927f0829ba 100644 --- a/datafusion/src/avro_to_arrow/schema.rs +++ b/datafusion/src/avro_to_arrow/schema.rs @@ -343,9 +343,12 @@ mod test { lookup: Default::default(), }; let props = external_props(&record_schema); - assert_eq!(props.get("doc"), Some(&"record documentation".to_string())); assert_eq!( - props.get("aliases"), + props.get("avro::doc"), + Some(&"record documentation".to_string()) + ); + assert_eq!( + props.get("avro::aliases"), Some(&"[fooalias,baralias]".to_string()) ); let enum_schema = AvroSchema::Enum { @@ -358,8 +361,14 @@ mod test { symbols: vec![], }; let props = external_props(&enum_schema); - assert_eq!(props.get("doc"), Some(&"enum documentation".to_string())); - assert_eq!(props.get("aliases"), Some(&"[fooenum,barenum]".to_string())); + assert_eq!( + props.get("avro::doc"), + Some(&"enum documentation".to_string()) + ); + assert_eq!( + props.get("avro::aliases"), + Some(&"[fooenum,barenum]".to_string()) + ); let fixed_schema = AvroSchema::Fixed { name: Name { name: "fixed".to_string(), @@ -370,7 +379,7 @@ mod test { }; let props = external_props(&fixed_schema); assert_eq!( - props.get("aliases"), + props.get("avro::aliases"), Some(&"[foofixed,barfixed]".to_string()) ); } From ea31e020a17d70973b779ca6d7a27ae5f5327a53 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 7 Sep 2021 10:54:19 +0200 Subject: [PATCH 51/58] split avro physical plan test feature wise and add a non-implemented test --- datafusion/src/physical_plan/avro.rs | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index 54e47d1c6030..daf8ac948357 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -390,10 +390,12 @@ impl RecordBatchStream for AvroStream<'_, R> { #[cfg(test)] mod tests { use super::*; - use futures::StreamExt; #[tokio::test] + #[cfg(feature = "avro")] async fn test() -> Result<()> { + use futures::StreamExt; + let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); let avro_exec = AvroExec::try_from_path( @@ -427,4 +429,25 @@ mod tests { Ok(()) } + + #[tokio::test] + #[cfg(not(feature = "avro"))] + async fn test() -> Result<()> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/alltypes_plain.avro", testdata); + let avro_exec = AvroExec::try_from_path( + &filename, + AvroReadOptions::default(), + Some(vec![0, 1, 2]), + 1024, + None, + ); + assert!(matches!( + avro_exec, + Err(DataFusionError::NotImplemented(msg)) + if msg == "cannot read avro schema without the 'avro' feature enabled".to_string() + )); + + Ok(()) + } } From a26485802d8f509080631f86dc2b9dd43c16a2e1 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Sun, 12 Sep 2021 12:01:37 +0200 Subject: [PATCH 52/58] submodule: switch back to apache/arrow-testing --- .gitmodules | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 52454695e343..ec5d6208b8dd 100644 --- a/.gitmodules +++ b/.gitmodules @@ -3,5 +3,4 @@ url = https://github.com/apache/parquet-testing.git [submodule "testing"] path = testing - url = https://github.com/Igosuki/arrow-testing.git - branch = avro + url = https://github.com/apache/arrow-testing From c62d9310a0abad6a09a20b5e77d4d73ffe97d759 Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Sun, 12 Sep 2021 12:17:57 +0200 Subject: [PATCH 53/58] fix_test: columns are now prefixed in the plan --- datafusion/tests/sql.rs | 64 ++++++++++++++++++++--------------------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index e257c7b24979..8a091f1d124c 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -4683,18 +4683,18 @@ async fn avro_query() { let sql = "SELECT id, CAST(string_col AS varchar) FROM alltypes_plain"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ - "+----+--------------------------+", - "| id | CAST(string_col AS Utf8) |", - "+----+--------------------------+", - "| 4 | 0 |", - "| 5 | 1 |", - "| 6 | 0 |", - "| 7 | 1 |", - "| 2 | 0 |", - "| 3 | 1 |", - "| 0 | 0 |", - "| 1 | 1 |", - "+----+--------------------------+", + "+----+-----------------------------------------+", + "| id | CAST(alltypes_plain.string_col AS Utf8) |", + "+----+-----------------------------------------+", + "| 4 | 0 |", + "| 5 | 1 |", + "| 6 | 0 |", + "| 7 | 1 |", + "| 2 | 0 |", + "| 3 | 1 |", + "| 0 | 0 |", + "| 1 | 1 |", + "+----+-----------------------------------------+", ]; assert_batches_eq!(expected, &actual); @@ -4730,26 +4730,26 @@ async fn avro_query_multiple_files() { let sql = "SELECT id, CAST(string_col AS varchar) FROM alltypes_plain"; let actual = execute_to_batches(&mut ctx, sql).await; let expected = vec![ - "+----+--------------------------+", - "| id | CAST(string_col AS Utf8) |", - "+----+--------------------------+", - "| 4 | 0 |", - "| 5 | 1 |", - "| 6 | 0 |", - "| 7 | 1 |", - "| 2 | 0 |", - "| 3 | 1 |", - "| 0 | 0 |", - "| 1 | 1 |", - "| 4 | 0 |", - "| 5 | 1 |", - "| 6 | 0 |", - "| 7 | 1 |", - "| 2 | 0 |", - "| 3 | 1 |", - "| 0 | 0 |", - "| 1 | 1 |", - "+----+--------------------------+", + "+----+-----------------------------------------+", + "| id | CAST(alltypes_plain.string_col AS Utf8) |", + "+----+-----------------------------------------+", + "| 4 | 0 |", + "| 5 | 1 |", + "| 6 | 0 |", + "| 7 | 1 |", + "| 2 | 0 |", + "| 3 | 1 |", + "| 0 | 0 |", + "| 1 | 1 |", + "| 4 | 0 |", + "| 5 | 1 |", + "| 6 | 0 |", + "| 7 | 1 |", + "| 2 | 0 |", + "| 3 | 1 |", + "| 0 | 0 |", + "| 1 | 1 |", + "+----+-----------------------------------------+", ]; assert_batches_eq!(expected, &actual); From aa45189b12bfef0ad0caed844ad8d2df6f06659b Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Sun, 12 Sep 2021 13:57:37 +0200 Subject: [PATCH 54/58] avro_test: fix clippy warning cmp-owned --- datafusion/src/physical_plan/avro.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index daf8ac948357..1c60e0d51ab3 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -445,7 +445,7 @@ mod tests { assert!(matches!( avro_exec, Err(DataFusionError::NotImplemented(msg)) - if msg == "cannot read avro schema without the 'avro' feature enabled".to_string() + if msg == *"cannot read avro schema without the 'avro' feature enabled" )); Ok(()) From 0b746dbc792669734457f6a3dc6eb06491f8940a Mon Sep 17 00:00:00 2001 From: Guillaume Balaine Date: Tue, 14 Sep 2021 10:31:50 +0200 Subject: [PATCH 55/58] avro: move statistics to the physical plan --- datafusion/src/datasource/avro.rs | 10 ---------- datafusion/src/physical_plan/avro.rs | 6 +++++- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/datafusion/src/datasource/avro.rs b/datafusion/src/datasource/avro.rs index 7b39bedae478..ee0fabfe0cc6 100644 --- a/datafusion/src/datasource/avro.rs +++ b/datafusion/src/datasource/avro.rs @@ -35,8 +35,6 @@ use crate::{ physical_plan::{common, ExecutionPlan}, }; -use super::datasource::Statistics; - trait SeekRead: Read + Seek {} impl SeekRead for T {} @@ -46,7 +44,6 @@ pub struct AvroFile { source: Source>, schema: SchemaRef, file_extension: String, - statistics: Statistics, } impl AvroFile { @@ -64,7 +61,6 @@ impl AvroFile { source: Source::Path(path.to_string()), schema, file_extension: options.file_extension.to_string(), - statistics: Statistics::default(), }) } @@ -84,7 +80,6 @@ impl AvroFile { Ok(Self { source: Source::Reader(Mutex::new(Some(Box::new(reader)))), schema, - statistics: Statistics::default(), file_extension: String::new(), }) } @@ -107,7 +102,6 @@ impl AvroFile { Ok(Self { source: Source::Reader(Mutex::new(Some(Box::new(reader)))), schema, - statistics: Statistics::default(), file_extension: String::new(), }) } @@ -173,10 +167,6 @@ impl TableProvider for AvroFile { }; Ok(Arc::new(exec)) } - - fn statistics(&self) -> Statistics { - self.statistics.clone() - } } #[cfg(test)] diff --git a/datafusion/src/physical_plan/avro.rs b/datafusion/src/physical_plan/avro.rs index 1c60e0d51ab3..3f0b007b26c0 100644 --- a/datafusion/src/physical_plan/avro.rs +++ b/datafusion/src/physical_plan/avro.rs @@ -21,7 +21,7 @@ use super::RecordBatchStream; use super::{common, source::Source, ExecutionPlan, Partitioning}; use crate::avro_to_arrow::read_avro_schema_from_reader; use crate::error::{DataFusionError, Result}; -use crate::physical_plan::DisplayFormatType; +use crate::physical_plan::{DisplayFormatType, Statistics}; use arrow::datatypes::{Schema, SchemaRef}; #[cfg(feature = "avro")] use arrow::{error::Result as ArrowResult, record_batch::RecordBatch}; @@ -321,6 +321,10 @@ impl ExecutionPlan for AvroExec { } } } + + fn statistics(&self) -> Statistics { + Statistics::default() + } } #[cfg(feature = "avro")] From a709fd9248b836f5b47fc269ee049c80b1e3d939 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 14 Sep 2021 12:04:27 -0400 Subject: [PATCH 56/58] Use larger stack for tests on CI --- .github/workflows/rust.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index d48233c73361..96b405c8f30f 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -79,7 +79,9 @@ jobs: env: # Disable full debug symbol generation to speed up CI build and keep memory down # "1" means line tables only, which is useful for panic tracebacks. - RUSTFLAGS: "-C debuginfo=1" + # Use zstack-size to work around stack overflow issues in debug builds + # https://github.com/apache/arrow-datafusion/issues/419 + RUSTFLAGS: "-C debuginfo=1 -C link-args=-Wl,-zstack-size=4194304" steps: - uses: actions/checkout@v2 with: From 9e6f1349d35d0ed8273f8e324d68387bb06659dc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 14 Sep 2021 12:50:29 -0400 Subject: [PATCH 57/58] try a different command and size --- .github/workflows/rust.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 96b405c8f30f..f2e65c41e613 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -79,9 +79,9 @@ jobs: env: # Disable full debug symbol generation to speed up CI build and keep memory down # "1" means line tables only, which is useful for panic tracebacks. - # Use zstack-size to work around stack overflow issues in debug builds + # Use stack-size to work around stack overflow issues in debug builds # https://github.com/apache/arrow-datafusion/issues/419 - RUSTFLAGS: "-C debuginfo=1 -C link-args=-Wl,-zstack-size=4194304" + RUSTFLAGS: "-C debuginfo=1 -C link-args=-Wl,-stack-size=8000000" steps: - uses: actions/checkout@v2 with: From 3c917483df5c83e3e61aadaa6843ec4e9c0104b0 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 14 Sep 2021 13:43:19 -0400 Subject: [PATCH 58/58] 16MB stack --- .github/workflows/rust.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index f2e65c41e613..e9e9699f026e 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -79,9 +79,9 @@ jobs: env: # Disable full debug symbol generation to speed up CI build and keep memory down # "1" means line tables only, which is useful for panic tracebacks. - # Use stack-size to work around stack overflow issues in debug builds + # Use zstack-size to work around stack overflow issues in debug builds # https://github.com/apache/arrow-datafusion/issues/419 - RUSTFLAGS: "-C debuginfo=1 -C link-args=-Wl,-stack-size=8000000" + RUSTFLAGS: "-C debuginfo=1 -C link-args=-Wl,-zstack-size=16000000" steps: - uses: actions/checkout@v2 with: