Skip to content

Commit 6402200

Browse files
Avro Table Provider (#910)
* Add avro as a datasource, file and table provider * wip * Added support composite identifiers for struct type. * Fixed build. * cheat and add unions to valid composite column types * Implement the AvroArrayReader * Add binary types * Enable Avro as a FileType * Enable registering an avro table in the sql parsing * Change package name for datafusion/avro * Implement Avro datasource tests and fix avro_rs::Value resolution to Arrow types * Test for AvroExec::try_from_path * external table avro test * Basic schema conversion tests * Complete test for avro_to_arrow_reader on alltypes_dictionnary * fix_stable: .rewind is 'unstable' * Fix license files and remove the unused avro-converter crate * fix example test in avro_to_arrow * add avro_sql test to default workflow * Adress clippies * Enable avro as a valid datasource for client execution * Add avro to available logical plan nodes * Add ToTimestampMillis as a scalar function in protos * Allow Avro in PhysicalPlan nodes * Remove remaining confusing references to 'json' in avro mod * rename 'parquet' words in avro test and examples * Handle Union of nested lists in arrow reader * test timestamp arrays * remove debug statement * Make avro optional * Remove debug statement * Remove GetField usage (see #628) * Fix docstring in parser tests * Test batch output rather than just rows individually * Remove 'csv' from error strings in physical_plan::avro * Avro sample sql and explain queries tests in sql.rs * Activate avro feature for cargo tests in github workflow * Add a test for avro registering multiple files in a single table * Switch to Result instead of Option for resolve_string * Address missing clippy warning should_implement_trait in arrow_to_avro/reader * Add fmt display implementation for AvroExec * ci: fix cargo sql run example, use datafusion/avro feature instead of 'avro' * license: missing license file for avro_to_arrow/schema.rs * only run avro datasource tests if features have 'avro' * refactor: rename infer_avro_schema_from_reader to read_avro_schema_from_reader * Pass None as props to avro schema schema_to_field_with_props until further notice * Change schema inferance to FixedSizeBinary(16) for Uuid * schema: prefix metadata coming from avro with 'avro' * make num traits optional and part of the avro feature flag * Fix avro schema tests regarding external props * split avro physical plan test feature wise and add a non-implemented test * submodule: switch back to apache/arrow-testing * fix_test: columns are now prefixed in the plan * avro_test: fix clippy warning cmp-owned * avro: move statistics to the physical plan * Increase min stack size for cargo tests Co-authored-by: Jorge C. Leitao <[email protected]>
1 parent 195b699 commit 6402200

File tree

33 files changed

+3258
-38
lines changed

33 files changed

+3258
-38
lines changed

.github/workflows/rust.yml

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -105,13 +105,14 @@ jobs:
105105
run: |
106106
export ARROW_TEST_DATA=$(pwd)/testing/data
107107
export PARQUET_TEST_DATA=$(pwd)/parquet-testing/data
108-
# run tests on all workspace members with default feature list
109-
cargo test
108+
# run tests on all workspace members with default feature list + avro
109+
RUST_MIN_STACK=10485760 cargo test --features=avro
110110
# test datafusion examples
111111
cd datafusion-examples
112112
cargo test --no-default-features
113113
cargo run --example csv_sql
114114
cargo run --example parquet_sql
115+
cargo run --example avro_sql --features=datafusion/avro
115116
env:
116117
CARGO_HOME: "/github/home/.cargo"
117118
CARGO_TARGET_DIR: "/github/home/target"

ballista/rust/client/src/context.rs

Lines changed: 40 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ use datafusion::dataframe::DataFrame;
3232
use datafusion::error::{DataFusionError, Result};
3333
use datafusion::execution::dataframe_impl::DataFrameImpl;
3434
use datafusion::logical_plan::LogicalPlan;
35+
use datafusion::physical_plan::avro::AvroReadOptions;
3536
use datafusion::physical_plan::csv::CsvReadOptions;
3637
use datafusion::sql::parser::FileType;
3738

@@ -125,6 +126,30 @@ impl BallistaContext {
125126
})
126127
}
127128

129+
/// Create a DataFrame representing an Avro table scan
130+
131+
pub fn read_avro(
132+
&self,
133+
path: &str,
134+
options: AvroReadOptions,
135+
) -> Result<Arc<dyn DataFrame>> {
136+
// convert to absolute path because the executor likely has a different working directory
137+
let path = PathBuf::from(path);
138+
let path = fs::canonicalize(&path)?;
139+
140+
// use local DataFusion context for now but later this might call the scheduler
141+
let mut ctx = {
142+
let guard = self.state.lock().unwrap();
143+
create_df_ctx_with_ballista_query_planner(
144+
&guard.scheduler_host,
145+
guard.scheduler_port,
146+
guard.config(),
147+
)
148+
};
149+
let df = ctx.read_avro(path.to_str().unwrap(), options)?;
150+
Ok(df)
151+
}
152+
128153
/// Create a DataFrame representing a Parquet table scan
129154
130155
pub fn read_parquet(&self, path: &str) -> Result<Arc<dyn DataFrame>> {
@@ -193,6 +218,17 @@ impl BallistaContext {
193218
self.register_table(name, df.as_ref())
194219
}
195220

221+
pub fn register_avro(
222+
&self,
223+
name: &str,
224+
path: &str,
225+
options: AvroReadOptions,
226+
) -> Result<()> {
227+
let df = self.read_avro(path, options)?;
228+
self.register_table(name, df.as_ref())?;
229+
Ok(())
230+
}
231+
196232
/// Create a DataFrame from a SQL statement
197233
pub fn sql(&self, sql: &str) -> Result<Arc<dyn DataFrame>> {
198234
let mut ctx = {
@@ -240,6 +276,10 @@ impl BallistaContext {
240276
self.register_parquet(name, location)?;
241277
Ok(Arc::new(DataFrameImpl::new(ctx.state, &plan)))
242278
}
279+
FileType::Avro => {
280+
self.register_avro(name, location, AvroReadOptions::default())?;
281+
Ok(Arc::new(DataFrameImpl::new(ctx.state, &plan)))
282+
}
243283
_ => Err(DataFusionError::NotImplemented(format!(
244284
"Unsupported file type {:?}.",
245285
file_type

ballista/rust/core/proto/ballista.proto

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -152,6 +152,7 @@ enum ScalarFunction {
152152
SHA384 = 32;
153153
SHA512 = 33;
154154
LN = 34;
155+
TOTIMESTAMPMILLIS = 35;
155156
}
156157

157158
message ScalarFunctionNode {
@@ -253,6 +254,7 @@ message LogicalPlanNode {
253254
WindowNode window = 13;
254255
AnalyzeNode analyze = 14;
255256
CrossJoinNode cross_join = 15;
257+
AvroTableScanNode avro_scan = 16;
256258
}
257259
}
258260

@@ -296,6 +298,15 @@ message ParquetTableScanNode {
296298
repeated LogicalExprNode filters = 4;
297299
}
298300

301+
message AvroTableScanNode {
302+
string table_name = 1;
303+
string path = 2;
304+
string file_extension = 3;
305+
ProjectionColumns projection = 4;
306+
Schema schema = 5;
307+
repeated LogicalExprNode filters = 6;
308+
}
309+
299310
message ProjectionNode {
300311
LogicalPlanNode input = 1;
301312
repeated LogicalExprNode expr = 2;
@@ -340,6 +351,7 @@ enum FileType{
340351
NdJson = 0;
341352
Parquet = 1;
342353
CSV = 2;
354+
Avro = 3;
343355
}
344356

345357
message AnalyzeNode {
@@ -456,6 +468,7 @@ message PhysicalPlanNode {
456468
WindowAggExecNode window = 17;
457469
ShuffleWriterExecNode shuffle_writer = 18;
458470
CrossJoinExecNode cross_join = 19;
471+
AvroScanExecNode avro_scan = 20;
459472
}
460473
}
461474

@@ -609,6 +622,17 @@ message CsvScanExecNode {
609622
repeated string filename = 8;
610623
}
611624

625+
message AvroScanExecNode {
626+
string path = 1;
627+
repeated uint32 projection = 2;
628+
Schema schema = 3;
629+
string file_extension = 4;
630+
uint32 batch_size = 5;
631+
632+
// partition filenames
633+
repeated string filename = 8;
634+
}
635+
612636
enum PartitionMode {
613637
COLLECT_LEFT = 0;
614638
PARTITIONED = 1;

ballista/rust/core/src/serde/logical_plan/from_proto.rs

Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ use datafusion::logical_plan::{
3232
LogicalPlan, LogicalPlanBuilder, Operator,
3333
};
3434
use datafusion::physical_plan::aggregates::AggregateFunction;
35+
use datafusion::physical_plan::avro::AvroReadOptions;
3536
use datafusion::physical_plan::csv::CsvReadOptions;
3637
use datafusion::physical_plan::window_functions::BuiltInWindowFunction;
3738
use datafusion::scalar::ScalarValue;
@@ -171,6 +172,32 @@ impl TryInto<LogicalPlan> for &protobuf::LogicalPlanNode {
171172
.build()
172173
.map_err(|e| e.into())
173174
}
175+
LogicalPlanType::AvroScan(scan) => {
176+
let schema: Schema = convert_required!(scan.schema)?;
177+
let options = AvroReadOptions {
178+
schema: Some(Arc::new(schema.clone())),
179+
file_extension: &scan.file_extension,
180+
};
181+
182+
let mut projection = None;
183+
if let Some(columns) = &scan.projection {
184+
let column_indices = columns
185+
.columns
186+
.iter()
187+
.map(|name| schema.index_of(name))
188+
.collect::<Result<Vec<usize>, _>>()?;
189+
projection = Some(column_indices);
190+
}
191+
192+
LogicalPlanBuilder::scan_avro_with_name(
193+
&scan.path,
194+
options,
195+
projection,
196+
&scan.table_name,
197+
)?
198+
.build()
199+
.map_err(|e| e.into())
200+
}
174201
LogicalPlanType::Sort(sort) => {
175202
let input: LogicalPlan = convert_box_required!(sort.input)?;
176203
let sort_expr: Vec<Expr> = sort
@@ -1193,6 +1220,7 @@ impl TryFrom<i32> for protobuf::FileType {
11931220
_x if _x == FileType::NdJson as i32 => Ok(FileType::NdJson),
11941221
_x if _x == FileType::Parquet as i32 => Ok(FileType::Parquet),
11951222
_x if _x == FileType::Csv as i32 => Ok(FileType::Csv),
1223+
_x if _x == FileType::Avro as i32 => Ok(FileType::Avro),
11961224
invalid => Err(BallistaError::General(format!(
11971225
"Attempted to convert invalid i32 to protobuf::Filetype: {}",
11981226
invalid
@@ -1209,6 +1237,7 @@ impl Into<datafusion::sql::parser::FileType> for protobuf::FileType {
12091237
protobuf::FileType::NdJson => FileType::NdJson,
12101238
protobuf::FileType::Parquet => FileType::Parquet,
12111239
protobuf::FileType::Csv => FileType::CSV,
1240+
protobuf::FileType::Avro => FileType::Avro,
12121241
}
12131242
}
12141243
}

ballista/rust/core/src/serde/logical_plan/mod.rs

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -643,8 +643,12 @@ mod roundtrip_tests {
643643

644644
let df_schema_ref = schema.to_dfschema_ref()?;
645645

646-
let filetypes: [FileType; 3] =
647-
[FileType::NdJson, FileType::Parquet, FileType::CSV];
646+
let filetypes: [FileType; 4] = [
647+
FileType::NdJson,
648+
FileType::Parquet,
649+
FileType::CSV,
650+
FileType::Avro,
651+
];
648652

649653
for file in filetypes.iter() {
650654
let create_table_node = LogicalPlan::CreateExternalTable {

ballista/rust/core/src/serde/logical_plan/to_proto.rs

Lines changed: 27 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ use crate::serde::{protobuf, BallistaError};
2525
use datafusion::arrow::datatypes::{
2626
DataType, Field, IntervalUnit, Schema, SchemaRef, TimeUnit,
2727
};
28+
use datafusion::datasource::avro::AvroFile;
2829
use datafusion::datasource::{CsvFile, PartitionedFile, TableDescriptor};
2930
use datafusion::logical_plan::{
3031
window_frames::{WindowFrame, WindowFrameBound, WindowFrameUnits},
@@ -793,6 +794,19 @@ impl TryInto<protobuf::LogicalPlanNode> for &LogicalPlan {
793794
},
794795
)),
795796
})
797+
} else if let Some(avro) = source.downcast_ref::<AvroFile>() {
798+
Ok(protobuf::LogicalPlanNode {
799+
logical_plan_type: Some(LogicalPlanType::AvroScan(
800+
protobuf::AvroTableScanNode {
801+
table_name: table_name.to_owned(),
802+
path: avro.path().to_owned(),
803+
projection,
804+
schema: Some(schema),
805+
file_extension: avro.file_extension().to_string(),
806+
filters,
807+
},
808+
)),
809+
})
796810
} else {
797811
Err(BallistaError::General(format!(
798812
"logical plan to_proto unsupported table provider {:?}",
@@ -974,6 +988,7 @@ impl TryInto<protobuf::LogicalPlanNode> for &LogicalPlan {
974988
FileType::NdJson => protobuf::FileType::NdJson,
975989
FileType::Parquet => protobuf::FileType::Parquet,
976990
FileType::CSV => protobuf::FileType::Csv,
991+
FileType::Avro => protobuf::FileType::Avro,
977992
};
978993

979994
Ok(protobuf::LogicalPlanNode {
@@ -1098,7 +1113,13 @@ impl TryInto<protobuf::LogicalExprNode> for &Expr {
10981113
)
10991114
}
11001115
};
1101-
let arg = &args[0];
1116+
let arg_expr: Option<Box<protobuf::LogicalExprNode>> = if !args.is_empty()
1117+
{
1118+
let arg = &args[0];
1119+
Some(Box::new(arg.try_into()?))
1120+
} else {
1121+
None
1122+
};
11021123
let partition_by = partition_by
11031124
.iter()
11041125
.map(|e| e.try_into())
@@ -1111,7 +1132,7 @@ impl TryInto<protobuf::LogicalExprNode> for &Expr {
11111132
protobuf::window_expr_node::WindowFrame::Frame(window_frame.into())
11121133
});
11131134
let window_expr = Box::new(protobuf::WindowExprNode {
1114-
expr: Some(Box::new(arg.try_into()?)),
1135+
expr: arg_expr,
11151136
window_function: Some(window_function),
11161137
partition_by,
11171138
order_by,
@@ -1284,7 +1305,7 @@ impl TryInto<protobuf::LogicalExprNode> for &Expr {
12841305
Expr::Wildcard => Ok(protobuf::LogicalExprNode {
12851306
expr_type: Some(protobuf::logical_expr_node::ExprType::Wildcard(true)),
12861307
}),
1287-
Expr::TryCast { .. } => unimplemented!(),
1308+
_ => unimplemented!(),
12881309
}
12891310
}
12901311
}
@@ -1473,6 +1494,9 @@ impl TryInto<protobuf::ScalarFunction> for &BuiltinScalarFunction {
14731494
BuiltinScalarFunction::SHA256 => Ok(protobuf::ScalarFunction::Sha256),
14741495
BuiltinScalarFunction::SHA384 => Ok(protobuf::ScalarFunction::Sha384),
14751496
BuiltinScalarFunction::SHA512 => Ok(protobuf::ScalarFunction::Sha512),
1497+
BuiltinScalarFunction::ToTimestampMillis => {
1498+
Ok(protobuf::ScalarFunction::Totimestampmillis)
1499+
}
14761500
_ => Err(BallistaError::General(format!(
14771501
"logical_plan::to_proto() unsupported scalar function {:?}",
14781502
self

ballista/rust/core/src/serde/physical_plan/from_proto.rs

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ use datafusion::logical_plan::{
4343
window_frames::WindowFrame, DFSchema, Expr, JoinConstraint, JoinType,
4444
};
4545
use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateFunction};
46+
use datafusion::physical_plan::avro::{AvroExec, AvroReadOptions};
4647
use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
4748
use datafusion::physical_plan::hash_aggregate::{AggregateMode, HashAggregateExec};
4849
use datafusion::physical_plan::hash_join::PartitionMode;
@@ -153,6 +154,21 @@ impl TryInto<Arc<dyn ExecutionPlan>> for &protobuf::PhysicalPlanNode {
153154
None,
154155
)))
155156
}
157+
PhysicalPlanType::AvroScan(scan) => {
158+
let schema = Arc::new(convert_required!(scan.schema)?);
159+
let options = AvroReadOptions {
160+
schema: Some(schema),
161+
file_extension: &scan.file_extension,
162+
};
163+
let projection = scan.projection.iter().map(|i| *i as usize).collect();
164+
Ok(Arc::new(AvroExec::try_from_path(
165+
&scan.path,
166+
options,
167+
Some(projection),
168+
scan.batch_size as usize,
169+
None,
170+
)?))
171+
}
156172
PhysicalPlanType::CoalesceBatches(coalesce_batches) => {
157173
let input: Arc<dyn ExecutionPlan> =
158174
convert_box_required!(coalesce_batches.input)?;
@@ -544,6 +560,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction {
544560
ScalarFunction::Sha384 => BuiltinScalarFunction::SHA384,
545561
ScalarFunction::Sha512 => BuiltinScalarFunction::SHA512,
546562
ScalarFunction::Ln => BuiltinScalarFunction::Ln,
563+
ScalarFunction::Totimestampmillis => BuiltinScalarFunction::ToTimestampMillis,
547564
}
548565
}
549566
}

ballista/rust/core/src/serde/physical_plan/to_proto.rs

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -62,6 +62,7 @@ use crate::execution_plans::{
6262
use crate::serde::protobuf::repartition_exec_node::PartitionMethod;
6363
use crate::serde::scheduler::PartitionLocation;
6464
use crate::serde::{protobuf, BallistaError};
65+
use datafusion::physical_plan::avro::AvroExec;
6566
use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
6667
use datafusion::physical_plan::functions::{BuiltinScalarFunction, ScalarFunctionExpr};
6768
use datafusion::physical_plan::repartition::RepartitionExec;
@@ -285,6 +286,28 @@ impl TryInto<protobuf::PhysicalPlanNode> for Arc<dyn ExecutionPlan> {
285286
},
286287
)),
287288
})
289+
} else if let Some(exec) = plan.downcast_ref::<AvroExec>() {
290+
Ok(protobuf::PhysicalPlanNode {
291+
physical_plan_type: Some(PhysicalPlanType::AvroScan(
292+
protobuf::AvroScanExecNode {
293+
path: exec.path().to_owned(),
294+
filename: exec.filenames().to_vec(),
295+
projection: exec
296+
.projection()
297+
.ok_or_else(|| {
298+
BallistaError::General(
299+
"projection in AvroExec doesn't exist.".to_owned(),
300+
)
301+
})?
302+
.iter()
303+
.map(|n| *n as u32)
304+
.collect(),
305+
file_extension: exec.file_extension().to_owned(),
306+
schema: Some(exec.file_schema().as_ref().into()),
307+
batch_size: exec.batch_size() as u32,
308+
},
309+
)),
310+
})
288311
} else if let Some(exec) = plan.downcast_ref::<ShuffleReaderExec>() {
289312
let mut partition = vec![];
290313
for location in &exec.partition {

datafusion-examples/Cargo.toml

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,10 @@ keywords = [ "arrow", "query", "sql" ]
2727
edition = "2018"
2828
publish = false
2929

30+
[[example]]
31+
name = "avro_sql"
32+
path = "examples/avro_sql.rs"
33+
required-features = ["datafusion/avro"]
3034

3135
[dev-dependencies]
3236
arrow-flight = { version = "^5.3" }

0 commit comments

Comments
 (0)