From 10710a257ca607fab1ca4141a1783e02710ee825 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 10:21:17 -0700 Subject: [PATCH 01/37] Parquet datasource --- rust/datafusion/src/datasource/mod.rs | 1 + rust/datafusion/src/datasource/parquet.rs | 310 ++++++++++++++++++++++ 2 files changed, 311 insertions(+) create mode 100644 rust/datafusion/src/datasource/parquet.rs diff --git a/rust/datafusion/src/datasource/mod.rs b/rust/datafusion/src/datasource/mod.rs index 1a22a2370e69a..5688fb5fed85e 100644 --- a/rust/datafusion/src/datasource/mod.rs +++ b/rust/datafusion/src/datasource/mod.rs @@ -18,6 +18,7 @@ pub mod csv; pub mod datasource; pub mod memory; +pub mod parquet; pub use self::csv::{CsvBatchIterator, CsvFile}; pub use self::datasource::{RecordBatchIterator, ScanResult, Table}; diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs new file mode 100644 index 0000000000000..1f5a9d53d634e --- /dev/null +++ b/rust/datafusion/src/datasource/parquet.rs @@ -0,0 +1,310 @@ +// 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. + +//! Parquet Data source + +use std::cell::RefCell; +use std::fs::File; +use std::rc::Rc; +use std::string::String; +use std::sync::Arc; + +use arrow::array::Array; +use arrow::datatypes::{DataType, Field, Schema}; +use arrow::record_batch::RecordBatch; + +use parquet::basic; +use parquet::column::reader::*; +use parquet::data_type::ByteArray; +use parquet::file::reader::*; +use parquet::schema::types::Type; + +use crate::datasource::{RecordBatchIterator, Table}; +use crate::execution::error::{ExecutionError, Result}; +use arrow::builder::{BinaryBuilder, Float64Builder, Int32Builder}; + +pub struct ParquetTable { + filename: String, + schema: Arc, +} + +impl ParquetTable { + pub fn new(filename: &str) -> Self { + let file = File::open(filename).unwrap(); + let parquet_file = ParquetFile::open(file, None).unwrap(); + let schema = parquet_file.schema.clone(); + Self { + filename: filename.to_string(), + schema, + } + } +} + +impl Table for ParquetTable { + fn schema(&self) -> &Arc { + &self.schema + } + + fn scan( + &self, + projection: &Option>, + _batch_size: usize, + ) -> Result>> { + let file = File::open(self.filename.clone()).unwrap(); + let parquet_file = ParquetFile::open(file, projection.clone()).unwrap(); + Ok(Rc::new(RefCell::new(parquet_file))) + } +} + +pub struct ParquetFile { + reader: SerializedFileReader, + row_group_index: usize, + schema: Arc, + projection: Option>, + batch_size: usize, + current_row_group: Option>, + column_readers: Vec, +} + +impl ParquetFile { + pub fn open(file: File, projection: Option>) -> Result { + let reader = SerializedFileReader::new(file).unwrap(); + + let metadata = reader.metadata(); + let file_type = to_arrow(metadata.file_metadata().schema())?; + + match file_type.data_type() { + DataType::Struct(fields) => { + let schema = Schema::new(fields.clone()); + //println!("Parquet schema: {:?}", schema); + Ok(ParquetFile { + reader: reader, + row_group_index: 0, + schema: Arc::new(schema), + projection, + batch_size: 64 * 1024, + current_row_group: None, + column_readers: vec![], + }) + } + _ => Err(ExecutionError::General( + "Failed to read Parquet schema".to_string(), + )), + } + } + + fn load_next_row_group(&mut self) { + if self.row_group_index < self.reader.num_row_groups() { + //println!("Loading row group {} of {}", self.row_group_index, self.reader.num_row_groups()); + let reader = self.reader.get_row_group(self.row_group_index).unwrap(); + + self.column_readers = vec![]; + + match &self.projection { + None => { + for i in 0..reader.num_columns() { + self.column_readers + .push(reader.get_column_reader(i).unwrap()); + } + } + Some(proj) => { + for i in proj { + //TODO validate index in bounds + self.column_readers + .push(reader.get_column_reader(*i).unwrap()); + } + } + } + + self.current_row_group = Some(reader); + self.row_group_index += 1; + } else { + panic!() + } + } + + fn load_batch(&mut self) -> Result> { + match &self.current_row_group { + Some(reader) => { + let mut batch: Vec> = Vec::with_capacity(reader.num_columns()); + let mut row_count = 0; + for i in 0..self.column_readers.len() { + let array: Arc = match self.column_readers[i] { + ColumnReader::Int32ColumnReader(ref mut r) => { + let mut builder = Int32Builder::new(self.batch_size); + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + //TODO this isn't handling null values + Ok((count, _)) => { + builder.append_slice(&read_buffer).unwrap(); + row_count = count; + Arc::new(builder.finish()) + } + _ => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {})", + i + ))); + } + } + } + ColumnReader::DoubleColumnReader(ref mut r) => { + let mut builder = Float64Builder::new(self.batch_size); + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + //TODO this isn't handling null values + Ok((count, _)) => { + builder.append_slice(&read_buffer).unwrap(); + row_count = count; + Arc::new(builder.finish()) + } + _ => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {})", + i + ))); + } + } + } + ColumnReader::ByteArrayColumnReader(ref mut r) => { + let mut b: Vec = + Vec::with_capacity(self.batch_size); + for _ in 0..self.batch_size { + b.push(ByteArray::default()); + } + match r.read_batch(self.batch_size, None, None, &mut b) { + //TODO this isn't handling null values + Ok((count, _)) => { + row_count = count; + //TODO this is horribly inefficient + let mut builder = BinaryBuilder::new(row_count); + for j in 0..row_count { + let foo = b[j].slice(0, b[j].len()); + let bytes: &[u8] = foo.data(); + let str = + String::from_utf8(bytes.to_vec()).unwrap(); + builder.append_string(&str).unwrap(); + } + Arc::new(builder.finish()) + } + _ => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {})", + i + ))); + } + } + } + _ => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type".to_string(), + )); + } + }; + + batch.push(array); + } + + // println!("Loaded batch of {} rows", row_count); + + if row_count == 0 { + Ok(None) + } else { + Ok(Some(RecordBatch::try_new(self.schema.clone(), batch)?)) + } + } + _ => Ok(None), + } + } +} + +fn to_arrow(t: &Type) -> Result { + match t { + Type::PrimitiveType { + basic_info, + physical_type, + .. + } => { + let arrow_type = match physical_type { + basic::Type::BOOLEAN => DataType::Boolean, + basic::Type::INT32 => DataType::Int32, + basic::Type::INT64 => DataType::Int64, + basic::Type::INT96 => DataType::Int64, //TODO ??? + basic::Type::FLOAT => DataType::Float32, + basic::Type::DOUBLE => DataType::Float64, + basic::Type::BYTE_ARRAY => DataType::Utf8, /*match basic_info.logical_type() { + basic::LogicalType::UTF8 => DataType::Utf8, + _ => unimplemented!("No support for Parquet BYTE_ARRAY yet"), + }*/ + basic::Type::FIXED_LEN_BYTE_ARRAY => { + unimplemented!("No support for Parquet FIXED_LEN_BYTE_ARRAY yet") + } + }; + + Ok(Field::new(basic_info.name(), arrow_type, false)) + } + Type::GroupType { basic_info, fields } => Ok(Field::new( + basic_info.name(), + DataType::Struct( + fields + .iter() + .map(|f| to_arrow(f)) + .collect::>>()?, + ), + false, + )), + } +} + +impl RecordBatchIterator for ParquetFile { + fn schema(&self) -> &Arc { + &self.schema + } + + fn next(&mut self) -> Result> { + // advance the row group reader if necessary + if self.current_row_group.is_none() { + self.load_next_row_group(); + self.load_batch() + } else { + match self.load_batch() { + Ok(Some(b)) => Ok(Some(b)), + Ok(None) => { + if self.row_group_index < self.reader.num_row_groups() { + self.load_next_row_group(); + self.load_batch() + } else { + Ok(None) + } + } + Err(e) => Err(e), + } + } + } +} From ff3e5b72c704d1e1fd7d4c0e28be35e61476989c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 10:40:11 -0700 Subject: [PATCH 02/37] test --- rust/datafusion/src/datasource/parquet.rs | 61 ++++++++++++++++++++--- 1 file changed, 55 insertions(+), 6 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 1f5a9d53d634e..e2b50797fe55c 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -144,10 +144,16 @@ impl ParquetFile { let mut row_count = 0; for i in 0..self.column_readers.len() { let array: Arc = match self.column_readers[i] { + ColumnReader::BoolColumnReader(ref mut r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (BOOL)".to_string(), + )); + } ColumnReader::Int32ColumnReader(ref mut r) => { let mut builder = Int32Builder::new(self.batch_size); let mut read_buffer: Vec = Vec::with_capacity(self.batch_size); + match r.read_batch( self.batch_size, None, @@ -156,6 +162,8 @@ impl ParquetFile { ) { //TODO this isn't handling null values Ok((count, _)) => { + println!("Read {} rows", count); + builder.append_slice(&read_buffer).unwrap(); row_count = count; Arc::new(builder.finish()) @@ -168,6 +176,21 @@ impl ParquetFile { } } } + ColumnReader::Int64ColumnReader(ref mut r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (INT64)".to_string(), + )); + } + ColumnReader::Int96ColumnReader(ref mut r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (INT96)".to_string(), + )); + } + ColumnReader::FloatColumnReader(ref mut r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (FLOAT)".to_string(), + )); + } ColumnReader::DoubleColumnReader(ref mut r) => { let mut builder = Float64Builder::new(self.batch_size); let mut read_buffer: Vec = @@ -192,6 +215,12 @@ impl ParquetFile { } } } + ColumnReader::FixedLenByteArrayColumnReader(ref mut r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (FixedLenByteArray)" + .to_string(), + )); + } ColumnReader::ByteArrayColumnReader(ref mut r) => { let mut b: Vec = Vec::with_capacity(self.batch_size); @@ -221,17 +250,13 @@ impl ParquetFile { } } } - _ => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type".to_string(), - )); - } }; + println!("Adding array to batch"); batch.push(array); } - // println!("Loaded batch of {} rows", row_count); + println!("Loaded batch of {} rows", row_count); if row_count == 0 { Ok(None) @@ -308,3 +333,27 @@ impl RecordBatchIterator for ParquetFile { } } } + +#[cfg(test)] +mod tests { + use super::*; + use std::env; + + #[test] + fn read_parquet_file() { + let testdata = env::var("PARQUET_TEST_DATA").unwrap(); + let filename = format!("{}/alltypes_plain.parquet", testdata); + + let table = ParquetTable::new(&filename); + + println!("{:?}", table.schema()); + + let projection = Some(vec![0]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().unwrap(); + + assert_eq!(1, batch.num_columns()); + assert_eq!(1, batch.num_rows()); + } +} From 3a412b18a704d83e15bc8be6f114a4770e9d1bb4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 11:00:06 -0700 Subject: [PATCH 03/37] first parquet test passes --- rust/datafusion/src/datasource/parquet.rs | 47 ++++++++++++++++++----- 1 file changed, 37 insertions(+), 10 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index e2b50797fe55c..241ab7a3cf571 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -144,16 +144,19 @@ impl ParquetFile { let mut row_count = 0; for i in 0..self.column_readers.len() { let array: Arc = match self.column_readers[i] { - ColumnReader::BoolColumnReader(ref mut r) => { + ColumnReader::BoolColumnReader(ref mut _r) => { return Err(ExecutionError::NotImplemented( "unsupported column reader type (BOOL)".to_string(), )); } ColumnReader::Int32ColumnReader(ref mut r) => { - let mut builder = Int32Builder::new(self.batch_size); let mut read_buffer: Vec = Vec::with_capacity(self.batch_size); + for _ in 0..self.batch_size { + read_buffer.push(0); + } + match r.read_batch( self.batch_size, None, @@ -163,7 +166,7 @@ impl ParquetFile { //TODO this isn't handling null values Ok((count, _)) => { println!("Read {} rows", count); - + let mut builder = Int32Builder::new(self.batch_size); builder.append_slice(&read_buffer).unwrap(); row_count = count; Arc::new(builder.finish()) @@ -176,17 +179,17 @@ impl ParquetFile { } } } - ColumnReader::Int64ColumnReader(ref mut r) => { + ColumnReader::Int64ColumnReader(ref mut _r) => { return Err(ExecutionError::NotImplemented( "unsupported column reader type (INT64)".to_string(), )); } - ColumnReader::Int96ColumnReader(ref mut r) => { + ColumnReader::Int96ColumnReader(ref mut _r) => { return Err(ExecutionError::NotImplemented( "unsupported column reader type (INT96)".to_string(), )); } - ColumnReader::FloatColumnReader(ref mut r) => { + ColumnReader::FloatColumnReader(ref mut _r) => { return Err(ExecutionError::NotImplemented( "unsupported column reader type (FLOAT)".to_string(), )); @@ -215,7 +218,7 @@ impl ParquetFile { } } } - ColumnReader::FixedLenByteArrayColumnReader(ref mut r) => { + ColumnReader::FixedLenByteArrayColumnReader(ref mut _r) => { return Err(ExecutionError::NotImplemented( "unsupported column reader type (FixedLenByteArray)" .to_string(), @@ -261,7 +264,15 @@ impl ParquetFile { if row_count == 0 { Ok(None) } else { - Ok(Some(RecordBatch::try_new(self.schema.clone(), batch)?)) + match &self.projection { + Some(proj) => Ok(Some(RecordBatch::try_new( + self.schema.projection(proj)?, + batch, + )?)), + None => { + Ok(Some(RecordBatch::try_new(self.schema.clone(), batch)?)) + } + } } } _ => Ok(None), @@ -337,10 +348,11 @@ impl RecordBatchIterator for ParquetFile { #[cfg(test)] mod tests { use super::*; + use arrow::array::Int32Array; use std::env; #[test] - fn read_parquet_file() { + fn read_read_i32_column() { let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/alltypes_plain.parquet", testdata); @@ -354,6 +366,21 @@ mod tests { let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); - assert_eq!(1, batch.num_rows()); + assert_eq!(64 * 1024, batch.num_rows()); + + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let mut values: Vec = vec![]; + for i in 0..16 { + values.push(array.value(i)); + } + + assert_eq!( + "[4, 5, 6, 7, 2, 3, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0]", + format!("{:?}", values) + ); } } From 322fc87516b0ec791f29bedbdd20a5082b630351 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 11:15:12 -0700 Subject: [PATCH 04/37] add test for reading strings from parquet --- rust/datafusion/src/datasource/parquet.rs | 46 +++++++++++++++++++---- 1 file changed, 39 insertions(+), 7 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 241ab7a3cf571..69ec57bef73f3 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -348,17 +348,12 @@ impl RecordBatchIterator for ParquetFile { #[cfg(test)] mod tests { use super::*; - use arrow::array::Int32Array; + use arrow::array::{BinaryArray, Int32Array}; use std::env; #[test] fn read_read_i32_column() { - let testdata = env::var("PARQUET_TEST_DATA").unwrap(); - let filename = format!("{}/alltypes_plain.parquet", testdata); - - let table = ParquetTable::new(&filename); - - println!("{:?}", table.schema()); + let table = load_table("alltypes_plain.parquet"); let projection = Some(vec![0]); let scan = table.scan(&projection, 1024).unwrap(); @@ -383,4 +378,41 @@ mod tests { format!("{:?}", values) ); } + + #[test] + fn read_read_string_column() { + let table = load_table("alltypes_plain.parquet"); + + let projection = Some(vec![9]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().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..8 { + let str: String = String::from_utf8(array.value(i).to_vec()).unwrap(); + values.push(str); + } + + assert_eq!( + "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", + format!("{:?}", values) + ); + } + + fn load_table(name: &str) -> Box { + let testdata = env::var("PARQUET_TEST_DATA").unwrap(); + let filename = format!("{}/{}", testdata, name); + let table = ParquetTable::new(&filename); + println!("{:?}", table.schema()); + Box::new(table) + } } From eaddafbf9d41437b065724bc4f4ef4e6427229cd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 14:21:29 -0700 Subject: [PATCH 05/37] save --- rust/arrow/src/datatypes.rs | 8 + rust/datafusion/src/datasource/parquet.rs | 319 ++++++++++++---------- 2 files changed, 188 insertions(+), 139 deletions(-) diff --git a/rust/arrow/src/datatypes.rs b/rust/arrow/src/datatypes.rs index e0b6d706eab15..69ce7114790c8 100644 --- a/rust/arrow/src/datatypes.rs +++ b/rust/arrow/src/datatypes.rs @@ -26,6 +26,7 @@ use std::mem::size_of; use std::ops::{Add, Div, Mul, Sub}; use std::slice::from_raw_parts; use std::str::FromStr; +use std::sync::Arc; use packed_simd::*; use serde_derive::{Deserialize, Serialize}; @@ -751,6 +752,13 @@ impl Schema { "fields": self.fields.iter().map(|field| field.to_json()).collect::>(), }) } + + /// Create a new schema by applying a projection to this schema's fields + pub fn projection(&self, i: &Vec) -> Result> { + //TODO bounds checks + let fields = i.iter().map(|index| self.field(*index).clone()).collect(); + Ok(Arc::new(Schema::new(fields))) + } } impl fmt::Display for Schema { diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 69ec57bef73f3..b933e24859911 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -32,6 +32,7 @@ use parquet::column::reader::*; use parquet::data_type::ByteArray; use parquet::file::reader::*; use parquet::schema::types::Type; +use parquet::record::{Row, RowAccessor}; use crate::datasource::{RecordBatchIterator, Table}; use crate::execution::error::{ExecutionError, Result}; @@ -73,11 +74,11 @@ impl Table for ParquetTable { pub struct ParquetFile { reader: SerializedFileReader, row_group_index: usize, + /// The schema of the underlying file schema: Arc, - projection: Option>, + projection: Vec, batch_size: usize, current_row_group: Option>, - column_readers: Vec, } impl ParquetFile { @@ -91,6 +92,18 @@ impl ParquetFile { DataType::Struct(fields) => { let schema = Schema::new(fields.clone()); //println!("Parquet schema: {:?}", schema); + + let projection = match projection { + Some(p) => p, + None => { + let mut p = Vec::with_capacity(schema.fields().len()); + for i in 0..schema.fields().len() { + p.push(i); + } + p + } + }; + Ok(ParquetFile { reader: reader, row_group_index: 0, @@ -98,7 +111,6 @@ impl ParquetFile { projection, batch_size: 64 * 1024, current_row_group: None, - column_readers: vec![], }) } _ => Err(ExecutionError::General( @@ -112,23 +124,23 @@ impl ParquetFile { //println!("Loading row group {} of {}", self.row_group_index, self.reader.num_row_groups()); let reader = self.reader.get_row_group(self.row_group_index).unwrap(); - self.column_readers = vec![]; - - match &self.projection { - None => { - for i in 0..reader.num_columns() { - self.column_readers - .push(reader.get_column_reader(i).unwrap()); - } - } - Some(proj) => { - for i in proj { - //TODO validate index in bounds - self.column_readers - .push(reader.get_column_reader(*i).unwrap()); - } - } - } +// self.column_readers = vec![]; +// +// match &self.projection { +// None => { +// for i in 0..reader.num_columns() { +// self.column_readers +// .push(reader.get_column_reader(i).unwrap()); +// } +// } +// Some(proj) => { +// for i in proj { +// //TODO validate index in bounds +// self.column_readers +// .push(reader.get_column_reader(*i).unwrap()); +// } +// } +// } self.current_row_group = Some(reader); self.row_group_index += 1; @@ -140,139 +152,168 @@ impl ParquetFile { fn load_batch(&mut self) -> Result> { match &self.current_row_group { Some(reader) => { - let mut batch: Vec> = Vec::with_capacity(reader.num_columns()); - let mut row_count = 0; - for i in 0..self.column_readers.len() { - let array: Arc = match self.column_readers[i] { - ColumnReader::BoolColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (BOOL)".to_string(), - )); - } - ColumnReader::Int32ColumnReader(ref mut r) => { - let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); - for _ in 0..self.batch_size { - read_buffer.push(0); - } + // read batch of rows into memory - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - //TODO this isn't handling null values - Ok((count, _)) => { - println!("Read {} rows", count); - let mut builder = Int32Builder::new(self.batch_size); - builder.append_slice(&read_buffer).unwrap(); - row_count = count; - Arc::new(builder.finish()) - } - _ => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {})", - i - ))); - } - } - } - ColumnReader::Int64ColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (INT64)".to_string(), - )); - } - ColumnReader::Int96ColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (INT96)".to_string(), - )); - } - ColumnReader::FloatColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (FLOAT)".to_string(), - )); - } - ColumnReader::DoubleColumnReader(ref mut r) => { - let mut builder = Float64Builder::new(self.batch_size); - let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - //TODO this isn't handling null values - Ok((count, _)) => { - builder.append_slice(&read_buffer).unwrap(); - row_count = count; - Arc::new(builder.finish()) - } - _ => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {})", - i - ))); - } +// let parquet_projection = self.projection.iter().map(|i| reader.metadata().schema_descr().column(*i)).collect(); + + let mut row_iter = reader.get_row_iter(None).unwrap(); //TODO projection push down + let mut rows: Vec = Vec::with_capacity(self.batch_size); + while let Some(row) = row_iter.next() { + if rows.len() == self.batch_size { + break; + } + rows.push(row); + } + println!("Loaded {} rows into memory", rows.len()); + + // convert to columnar batch + let mut batch: Vec> = Vec::with_capacity(self.projection.len()); + for i in &self.projection { + let array: Arc = match self.schema.field(*i).data_type() { + DataType::Int32 => { + //TODO null handling + let mut builder = Int32Builder::new(rows.len()); + for row in &rows { + builder.push() } + Arc::new(builder.finish()) } - ColumnReader::FixedLenByteArrayColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (FixedLenByteArray)" - .to_string(), - )); - } - ColumnReader::ByteArrayColumnReader(ref mut r) => { - let mut b: Vec = - Vec::with_capacity(self.batch_size); - for _ in 0..self.batch_size { - b.push(ByteArray::default()); - } - match r.read_batch(self.batch_size, None, None, &mut b) { - //TODO this isn't handling null values - Ok((count, _)) => { - row_count = count; - //TODO this is horribly inefficient - let mut builder = BinaryBuilder::new(row_count); - for j in 0..row_count { - let foo = b[j].slice(0, b[j].len()); - let bytes: &[u8] = foo.data(); - let str = - String::from_utf8(bytes.to_vec()).unwrap(); - builder.append_string(&str).unwrap(); - } - Arc::new(builder.finish()) - } - _ => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {})", - i - ))); - } + DataType::Utf8 => { + //TODO null handling + let mut builder = BinaryBuilder::new(rows.len()); + for row in &rows { + let bytes = row.get_bytes(*i).unwrap(); + builder.append_string(&String::from_utf8(bytes.data().to_vec()).unwrap()).unwrap(); } + Arc::new(builder.finish()) } + _ => panic!() }; - - println!("Adding array to batch"); +// ColumnReader::BoolColumnReader(ref mut _r) => { +// return Err(ExecutionError::NotImplemented( +// "unsupported column reader type (BOOL)".to_string(), +// )); +// } +// ColumnReader::Int32ColumnReader(ref mut r) => { +// let mut read_buffer: Vec = +// Vec::with_capacity(self.batch_size); +// +// for _ in 0..self.batch_size { +// read_buffer.push(0); +// } +// r.read_ +// +// match r.read_batch( +// self.batch_size, +// None, +// None, +// &mut read_buffer, +// ) { +// //TODO this isn't handling null values +// Ok((count, _)) => { +// println!("Read {} rows", count); +// let mut builder = Int32Builder::new(self.batch_size); +// builder.append_slice(&read_buffer).unwrap(); +// row_count = count; +// Arc::new(builder.finish()) +// } +// _ => { +// return Err(ExecutionError::NotImplemented(format!( +// "Error reading parquet batch (column {})", +// i +// ))); +// } +// } +// } +// ColumnReader::Int64ColumnReader(ref mut _r) => { +// return Err(ExecutionError::NotImplemented( +// "unsupported column reader type (INT64)".to_string(), +// )); +// } +// ColumnReader::Int96ColumnReader(ref mut _r) => { +// return Err(ExecutionError::NotImplemented( +// "unsupported column reader type (INT96)".to_string(), +// )); +// } +// ColumnReader::FloatColumnReader(ref mut _r) => { +// return Err(ExecutionError::NotImplemented( +// "unsupported column reader type (FLOAT)".to_string(), +// )); +// } +// ColumnReader::DoubleColumnReader(ref mut r) => { +// let mut builder = Float64Builder::new(self.batch_size); +// let mut read_buffer: Vec = +// Vec::with_capacity(self.batch_size); +// match r.read_batch( +// self.batch_size, +// None, +// None, +// &mut read_buffer, +// ) { +// //TODO this isn't handling null values +// Ok((count, _)) => { +// builder.append_slice(&read_buffer).unwrap(); +// row_count = count; +// Arc::new(builder.finish()) +// } +// _ => { +// return Err(ExecutionError::NotImplemented(format!( +// "Error reading parquet batch (column {})", +// i +// ))); +// } +// } +// } +// ColumnReader::FixedLenByteArrayColumnReader(ref mut _r) => { +// return Err(ExecutionError::NotImplemented( +// "unsupported column reader type (FixedLenByteArray)" +// .to_string(), +// )); +// } +// ColumnReader::ByteArrayColumnReader(ref mut r) => { +// let mut b: Vec = +// Vec::with_capacity(self.batch_size); +// for _ in 0..self.batch_size { +// b.push(ByteArray::default()); +// } +// match r.read_batch(self.batch_size, None, None, &mut b) { +// //TODO this isn't handling null values +// Ok((count, _)) => { +// row_count = count; +// //TODO this is horribly inefficient +// let mut builder = BinaryBuilder::new(row_count); +// for j in 0..row_count { +// let foo = b[j].slice(0, b[j].len()); +// let bytes: &[u8] = foo.data(); +// let str = +// String::from_utf8(bytes.to_vec()).unwrap(); +// builder.append_string(&str).unwrap(); +// } +// Arc::new(builder.finish()) +// } +// _ => { +// return Err(ExecutionError::NotImplemented(format!( +// "Error reading parquet batch (column {})", +// i +// ))); +// } +// } +// } +// }; +// batch.push(array); } - println!("Loaded batch of {} rows", row_count); + println!("Loaded batch of {} rows", rows.len()); - if row_count == 0 { + if rows.len() == 0 { Ok(None) } else { - match &self.projection { - Some(proj) => Ok(Some(RecordBatch::try_new( - self.schema.projection(proj)?, + Ok(Some(RecordBatch::try_new( + self.schema.projection(&self.projection)?, batch, - )?)), - None => { - Ok(Some(RecordBatch::try_new(self.schema.clone(), batch)?)) - } - } + )?)) } } _ => Ok(None), From f46e6f7ca7f391d4d2200c793e238f8fe193fe67 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 14:25:58 -0700 Subject: [PATCH 06/37] save --- rust/datafusion/src/datasource/parquet.rs | 125 ++-------------------- 1 file changed, 7 insertions(+), 118 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index b933e24859911..4dcd8d969be2c 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -172,136 +172,25 @@ impl ParquetFile { for i in &self.projection { let array: Arc = match self.schema.field(*i).data_type() { DataType::Int32 => { - //TODO null handling let mut builder = Int32Builder::new(rows.len()); for row in &rows { - builder.push() + //TODO null handling + builder.append_value(row.get_int(*i).unwrap()).unwrap(); } Arc::new(builder.finish()) } DataType::Utf8 => { - //TODO null handling let mut builder = BinaryBuilder::new(rows.len()); for row in &rows { + //TODO null handling let bytes = row.get_bytes(*i).unwrap(); builder.append_string(&String::from_utf8(bytes.data().to_vec()).unwrap()).unwrap(); } Arc::new(builder.finish()) } - _ => panic!() + other => return Err(ExecutionError::NotImplemented( + format!("unsupported column reader type ({:?})", other))) }; -// ColumnReader::BoolColumnReader(ref mut _r) => { -// return Err(ExecutionError::NotImplemented( -// "unsupported column reader type (BOOL)".to_string(), -// )); -// } -// ColumnReader::Int32ColumnReader(ref mut r) => { -// let mut read_buffer: Vec = -// Vec::with_capacity(self.batch_size); -// -// for _ in 0..self.batch_size { -// read_buffer.push(0); -// } -// r.read_ -// -// match r.read_batch( -// self.batch_size, -// None, -// None, -// &mut read_buffer, -// ) { -// //TODO this isn't handling null values -// Ok((count, _)) => { -// println!("Read {} rows", count); -// let mut builder = Int32Builder::new(self.batch_size); -// builder.append_slice(&read_buffer).unwrap(); -// row_count = count; -// Arc::new(builder.finish()) -// } -// _ => { -// return Err(ExecutionError::NotImplemented(format!( -// "Error reading parquet batch (column {})", -// i -// ))); -// } -// } -// } -// ColumnReader::Int64ColumnReader(ref mut _r) => { -// return Err(ExecutionError::NotImplemented( -// "unsupported column reader type (INT64)".to_string(), -// )); -// } -// ColumnReader::Int96ColumnReader(ref mut _r) => { -// return Err(ExecutionError::NotImplemented( -// "unsupported column reader type (INT96)".to_string(), -// )); -// } -// ColumnReader::FloatColumnReader(ref mut _r) => { -// return Err(ExecutionError::NotImplemented( -// "unsupported column reader type (FLOAT)".to_string(), -// )); -// } -// ColumnReader::DoubleColumnReader(ref mut r) => { -// let mut builder = Float64Builder::new(self.batch_size); -// let mut read_buffer: Vec = -// Vec::with_capacity(self.batch_size); -// match r.read_batch( -// self.batch_size, -// None, -// None, -// &mut read_buffer, -// ) { -// //TODO this isn't handling null values -// Ok((count, _)) => { -// builder.append_slice(&read_buffer).unwrap(); -// row_count = count; -// Arc::new(builder.finish()) -// } -// _ => { -// return Err(ExecutionError::NotImplemented(format!( -// "Error reading parquet batch (column {})", -// i -// ))); -// } -// } -// } -// ColumnReader::FixedLenByteArrayColumnReader(ref mut _r) => { -// return Err(ExecutionError::NotImplemented( -// "unsupported column reader type (FixedLenByteArray)" -// .to_string(), -// )); -// } -// ColumnReader::ByteArrayColumnReader(ref mut r) => { -// let mut b: Vec = -// Vec::with_capacity(self.batch_size); -// for _ in 0..self.batch_size { -// b.push(ByteArray::default()); -// } -// match r.read_batch(self.batch_size, None, None, &mut b) { -// //TODO this isn't handling null values -// Ok((count, _)) => { -// row_count = count; -// //TODO this is horribly inefficient -// let mut builder = BinaryBuilder::new(row_count); -// for j in 0..row_count { -// let foo = b[j].slice(0, b[j].len()); -// let bytes: &[u8] = foo.data(); -// let str = -// String::from_utf8(bytes.to_vec()).unwrap(); -// builder.append_string(&str).unwrap(); -// } -// Arc::new(builder.finish()) -// } -// _ => { -// return Err(ExecutionError::NotImplemented(format!( -// "Error reading parquet batch (column {})", -// i -// ))); -// } -// } -// } -// }; -// batch.push(array); } @@ -402,7 +291,7 @@ mod tests { let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); - assert_eq!(64 * 1024, batch.num_rows()); + assert_eq!(8, batch.num_rows()); let array = batch .column(0) @@ -415,7 +304,7 @@ mod tests { } assert_eq!( - "[4, 5, 6, 7, 2, 3, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0]", + "[4, 5, 6, 7, 2, 3, 0, 1, 0, 0, 9, 0, 1, 0, 0, 0]", format!("{:?}", values) ); } From aea9f8a1162e59ecb70dfcd273dcba5566a11316 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 14:30:26 -0700 Subject: [PATCH 07/37] convert to use row iter --- rust/datafusion/src/datasource/parquet.rs | 78 ++++++++++++----------- 1 file changed, 40 insertions(+), 38 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 4dcd8d969be2c..1f2a66d2bb017 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -28,15 +28,13 @@ use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; use parquet::basic; -use parquet::column::reader::*; -use parquet::data_type::ByteArray; use parquet::file::reader::*; -use parquet::schema::types::Type; use parquet::record::{Row, RowAccessor}; +use parquet::schema::types::Type; use crate::datasource::{RecordBatchIterator, Table}; use crate::execution::error::{ExecutionError, Result}; -use arrow::builder::{BinaryBuilder, Float64Builder, Int32Builder}; +use arrow::builder::{BinaryBuilder, Float32Builder, Float64Builder, Int32Builder}; pub struct ParquetTable { filename: String, @@ -121,27 +119,7 @@ impl ParquetFile { fn load_next_row_group(&mut self) { if self.row_group_index < self.reader.num_row_groups() { - //println!("Loading row group {} of {}", self.row_group_index, self.reader.num_row_groups()); let reader = self.reader.get_row_group(self.row_group_index).unwrap(); - -// self.column_readers = vec![]; -// -// match &self.projection { -// None => { -// for i in 0..reader.num_columns() { -// self.column_readers -// .push(reader.get_column_reader(i).unwrap()); -// } -// } -// Some(proj) => { -// for i in proj { -// //TODO validate index in bounds -// self.column_readers -// .push(reader.get_column_reader(*i).unwrap()); -// } -// } -// } - self.current_row_group = Some(reader); self.row_group_index += 1; } else { @@ -152,10 +130,9 @@ impl ParquetFile { fn load_batch(&mut self) -> Result> { match &self.current_row_group { Some(reader) => { - // read batch of rows into memory -// let parquet_projection = self.projection.iter().map(|i| reader.metadata().schema_descr().column(*i)).collect(); + // let parquet_projection = self.projection.iter().map(|i| reader.metadata().schema_descr().column(*i)).collect(); let mut row_iter = reader.get_row_iter(None).unwrap(); //TODO projection push down let mut rows: Vec = Vec::with_capacity(self.batch_size); @@ -168,7 +145,8 @@ impl ParquetFile { println!("Loaded {} rows into memory", rows.len()); // convert to columnar batch - let mut batch: Vec> = Vec::with_capacity(self.projection.len()); + let mut batch: Vec> = + Vec::with_capacity(self.projection.len()); for i in &self.projection { let array: Arc = match self.schema.field(*i).data_type() { DataType::Int32 => { @@ -179,17 +157,44 @@ impl ParquetFile { } Arc::new(builder.finish()) } + DataType::Float32 => { + let mut builder = Float32Builder::new(rows.len()); + for row in &rows { + //TODO null handling + builder.append_value(row.get_float(*i).unwrap()).unwrap(); + } + Arc::new(builder.finish()) + } + DataType::Float64 => { + let mut builder = Float64Builder::new(rows.len()); + for row in &rows { + //TODO null handling + builder + .append_value(row.get_double(*i).unwrap()) + .unwrap(); + } + Arc::new(builder.finish()) + } DataType::Utf8 => { let mut builder = BinaryBuilder::new(rows.len()); for row in &rows { //TODO null handling let bytes = row.get_bytes(*i).unwrap(); - builder.append_string(&String::from_utf8(bytes.data().to_vec()).unwrap()).unwrap(); + builder + .append_string( + &String::from_utf8(bytes.data().to_vec()) + .unwrap(), + ) + .unwrap(); } Arc::new(builder.finish()) } - other => return Err(ExecutionError::NotImplemented( - format!("unsupported column reader type ({:?})", other))) + other => { + return Err(ExecutionError::NotImplemented(format!( + "unsupported column reader type ({:?})", + other + ))); + } }; batch.push(array); } @@ -200,9 +205,9 @@ impl ParquetFile { Ok(None) } else { Ok(Some(RecordBatch::try_new( - self.schema.projection(&self.projection)?, - batch, - )?)) + self.schema.projection(&self.projection)?, + batch, + )?)) } } _ => Ok(None), @@ -299,14 +304,11 @@ mod tests { .downcast_ref::() .unwrap(); let mut values: Vec = vec![]; - for i in 0..16 { + for i in 0..batch.num_rows() { values.push(array.value(i)); } - assert_eq!( - "[4, 5, 6, 7, 2, 3, 0, 1, 0, 0, 9, 0, 1, 0, 0, 0]", - format!("{:?}", values) - ); + assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); } #[test] From c3f71d718eb6c052322eccf43d0c707ccb731143 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 14:48:24 -0700 Subject: [PATCH 08/37] add integration test --- rust/datafusion/src/datasource/parquet.rs | 1 + rust/datafusion/tests/sql.rs | 22 ++++++++++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 1f2a66d2bb017..dbb1775836e21 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -54,6 +54,7 @@ impl ParquetTable { } impl Table for ParquetTable { + fn schema(&self) -> &Arc { &self.schema } diff --git a/rust/datafusion/tests/sql.rs b/rust/datafusion/tests/sql.rs index 954b2ee0fbf13..5ffae5dafe814 100644 --- a/rust/datafusion/tests/sql.rs +++ b/rust/datafusion/tests/sql.rs @@ -16,6 +16,7 @@ // under the License. use std::cell::RefCell; +use std::env; use std::rc::Rc; use std::sync::Arc; @@ -27,9 +28,22 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion::execution::context::ExecutionContext; use datafusion::execution::relation::Relation; +use datafusion::datasource::parquet::ParquetFile; +use datafusion::datasource::parquet::ParquetTable; +use datafusion::datasource::{RecordBatchIterator, Table}; const DEFAULT_BATCH_SIZE: usize = 1024 * 1024; +#[test] +fn parquet_query() { + let mut ctx = ExecutionContext::new(); + ctx.register_table("alltypes_plain", load_parquet_table("alltypes_plain.parquet")); + let sql = "SELECT id, string_col FROM alltypes_plain"; + let actual = execute(&mut ctx, sql); + let expected = "tbd".to_string(); + assert_eq!(expected, actual); +} + #[test] fn csv_query_with_predicate() { let mut ctx = ExecutionContext::new(); @@ -163,6 +177,14 @@ fn register_csv( ctx.register_csv(name, filename, &schema, true); } +fn load_parquet_table(name: &str) -> Rc
{ + let testdata = env::var("PARQUET_TEST_DATA").unwrap(); + let filename = format!("{}/{}", testdata, name); + let table = ParquetTable::new(&filename); + println!("{:?}", table.schema()); + Rc::new(table) +} + /// Execute query and return result set as tab delimited string fn execute(ctx: &mut ExecutionContext, sql: &str) -> String { let results = ctx.sql(&sql, DEFAULT_BATCH_SIZE).unwrap(); From 5ce3086ac959a258ff8dce33f4443edba33f20a1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 9 Mar 2019 17:23:54 -0700 Subject: [PATCH 09/37] revert to columnar reads --- rust/datafusion/src/datasource/parquet.rs | 203 +++++++++++++++------- rust/datafusion/tests/sql.rs | 16 +- 2 files changed, 147 insertions(+), 72 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index dbb1775836e21..4447fbdc55932 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -28,13 +28,14 @@ use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; use parquet::basic; +use parquet::column::reader::*; +use parquet::data_type::ByteArray; use parquet::file::reader::*; -use parquet::record::{Row, RowAccessor}; use parquet::schema::types::Type; use crate::datasource::{RecordBatchIterator, Table}; use crate::execution::error::{ExecutionError, Result}; -use arrow::builder::{BinaryBuilder, Float32Builder, Float64Builder, Int32Builder}; +use arrow::builder::{BinaryBuilder, Float64Builder, Int32Builder}; pub struct ParquetTable { filename: String, @@ -54,7 +55,6 @@ impl ParquetTable { } impl Table for ParquetTable { - fn schema(&self) -> &Arc { &self.schema } @@ -78,10 +78,13 @@ pub struct ParquetFile { projection: Vec, batch_size: usize, current_row_group: Option>, + column_readers: Vec, } impl ParquetFile { pub fn open(file: File, projection: Option>) -> Result { + println!("open()"); + let reader = SerializedFileReader::new(file).unwrap(); let metadata = reader.metadata(); @@ -103,13 +106,22 @@ impl ParquetFile { } }; + let projected_fields: Vec = projection + .iter() + .map(|i| schema.fields()[*i].clone()) + .collect(); + + let projected_schema = Arc::new(Schema::new(projected_fields)); + println!("projected schema: {:?}", projected_schema); + Ok(ParquetFile { reader: reader, row_group_index: 0, - schema: Arc::new(schema), + schema: projected_schema, projection, batch_size: 64 * 1024, current_row_group: None, + column_readers: vec![], }) } _ => Err(ExecutionError::General( @@ -120,7 +132,17 @@ impl ParquetFile { fn load_next_row_group(&mut self) { if self.row_group_index < self.reader.num_row_groups() { + //println!("Loading row group {} of {}", self.row_group_index, self.reader.num_row_groups()); let reader = self.reader.get_row_group(self.row_group_index).unwrap(); + + self.column_readers = vec![]; + + for i in &self.projection { + //TODO validate index in bounds + self.column_readers + .push(reader.get_column_reader(*i).unwrap()); + } + self.current_row_group = Some(reader); self.row_group_index += 1; } else { @@ -129,86 +151,134 @@ impl ParquetFile { } fn load_batch(&mut self) -> Result> { + println!("load_batch()"); match &self.current_row_group { Some(reader) => { - // read batch of rows into memory + let mut batch: Vec> = Vec::with_capacity(reader.num_columns()); + let mut row_count = 0; + for i in 0..self.column_readers.len() { + let array: Arc = match self.column_readers[i] { + ColumnReader::BoolColumnReader(ref mut _r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (BOOL)".to_string(), + )); + } + ColumnReader::Int32ColumnReader(ref mut r) => { + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); - // let parquet_projection = self.projection.iter().map(|i| reader.metadata().schema_descr().column(*i)).collect(); + for _ in 0..self.batch_size { + read_buffer.push(0); + } - let mut row_iter = reader.get_row_iter(None).unwrap(); //TODO projection push down - let mut rows: Vec = Vec::with_capacity(self.batch_size); - while let Some(row) = row_iter.next() { - if rows.len() == self.batch_size { - break; - } - rows.push(row); - } - println!("Loaded {} rows into memory", rows.len()); - - // convert to columnar batch - let mut batch: Vec> = - Vec::with_capacity(self.projection.len()); - for i in &self.projection { - let array: Arc = match self.schema.field(*i).data_type() { - DataType::Int32 => { - let mut builder = Int32Builder::new(rows.len()); - for row in &rows { - //TODO null handling - builder.append_value(row.get_int(*i).unwrap()).unwrap(); + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + //TODO this isn't handling null values + Ok((count, _)) => { + println!("Read {} rows", count); + let mut builder = Int32Builder::new(count); + builder.append_slice(&read_buffer[0..count]).unwrap(); + row_count = count; + Arc::new(builder.finish()) + } + _ => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {})", + i + ))); + } } - Arc::new(builder.finish()) } - DataType::Float32 => { - let mut builder = Float32Builder::new(rows.len()); - for row in &rows { - //TODO null handling - builder.append_value(row.get_float(*i).unwrap()).unwrap(); - } - Arc::new(builder.finish()) + ColumnReader::Int64ColumnReader(ref mut _r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (INT64)".to_string(), + )); } - DataType::Float64 => { - let mut builder = Float64Builder::new(rows.len()); - for row in &rows { - //TODO null handling - builder - .append_value(row.get_double(*i).unwrap()) - .unwrap(); - } - Arc::new(builder.finish()) + ColumnReader::Int96ColumnReader(ref mut _r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (INT96)".to_string(), + )); } - DataType::Utf8 => { - let mut builder = BinaryBuilder::new(rows.len()); - for row in &rows { - //TODO null handling - let bytes = row.get_bytes(*i).unwrap(); - builder - .append_string( - &String::from_utf8(bytes.data().to_vec()) - .unwrap(), - ) - .unwrap(); + ColumnReader::FloatColumnReader(ref mut _r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (FLOAT)".to_string(), + )); + } + ColumnReader::DoubleColumnReader(ref mut r) => { + let mut builder = Float64Builder::new(self.batch_size); + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + //TODO this isn't handling null values + Ok((count, _)) => { + builder.append_slice(&read_buffer).unwrap(); + row_count = count; + Arc::new(builder.finish()) + } + _ => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {})", + i + ))); + } } - Arc::new(builder.finish()) } - other => { - return Err(ExecutionError::NotImplemented(format!( - "unsupported column reader type ({:?})", - other - ))); + ColumnReader::FixedLenByteArrayColumnReader(ref mut _r) => { + return Err(ExecutionError::NotImplemented( + "unsupported column reader type (FixedLenByteArray)" + .to_string(), + )); + } + ColumnReader::ByteArrayColumnReader(ref mut r) => { + let mut b: Vec = + Vec::with_capacity(self.batch_size); + for _ in 0..self.batch_size { + b.push(ByteArray::default()); + } + match r.read_batch(self.batch_size, None, None, &mut b) { + //TODO this isn't handling null values + Ok((count, _)) => { + row_count = count; + //TODO this is horribly inefficient + let mut builder = BinaryBuilder::new(row_count); + for j in 0..row_count { + let foo = b[j].slice(0, b[j].len()); + let bytes: &[u8] = foo.data(); + let str = + String::from_utf8(bytes.to_vec()).unwrap(); + builder.append_string(&str).unwrap(); + } + Arc::new(builder.finish()) + } + _ => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {})", + i + ))); + } + } } }; + + println!("Adding array to batch"); batch.push(array); } - println!("Loaded batch of {} rows", rows.len()); + println!("Loaded batch of {} rows", row_count); - if rows.len() == 0 { + if row_count == 0 { Ok(None) } else { - Ok(Some(RecordBatch::try_new( - self.schema.projection(&self.projection)?, - batch, - )?)) + Ok(Some(RecordBatch::try_new(self.schema.clone(), batch)?)) } } _ => Ok(None), @@ -342,6 +412,7 @@ mod tests { } fn load_table(name: &str) -> Box
{ + println!("load_table"); let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); let table = ParquetTable::new(&filename); diff --git a/rust/datafusion/tests/sql.rs b/rust/datafusion/tests/sql.rs index 5ffae5dafe814..1843974055998 100644 --- a/rust/datafusion/tests/sql.rs +++ b/rust/datafusion/tests/sql.rs @@ -26,21 +26,23 @@ extern crate datafusion; use arrow::array::*; use arrow::datatypes::{DataType, Field, Schema}; +use datafusion::datasource::parquet::ParquetTable; +use datafusion::datasource::Table; use datafusion::execution::context::ExecutionContext; use datafusion::execution::relation::Relation; -use datafusion::datasource::parquet::ParquetFile; -use datafusion::datasource::parquet::ParquetTable; -use datafusion::datasource::{RecordBatchIterator, Table}; const DEFAULT_BATCH_SIZE: usize = 1024 * 1024; #[test] fn parquet_query() { let mut ctx = ExecutionContext::new(); - ctx.register_table("alltypes_plain", load_parquet_table("alltypes_plain.parquet")); + ctx.register_table( + "alltypes_plain", + load_parquet_table("alltypes_plain.parquet"), + ); let sql = "SELECT id, string_col FROM alltypes_plain"; let actual = execute(&mut ctx, sql); - let expected = "tbd".to_string(); + let expected = "4\t\"0\"\n5\t\"1\"\n6\t\"0\"\n7\t\"1\"\n2\t\"0\"\n3\t\"1\"\n0\t\"0\"\n1\t\"1\"\n".to_string(); assert_eq!(expected, actual); } @@ -187,7 +189,9 @@ fn load_parquet_table(name: &str) -> Rc
{ /// Execute query and return result set as tab delimited string fn execute(ctx: &mut ExecutionContext, sql: &str) -> String { - let results = ctx.sql(&sql, DEFAULT_BATCH_SIZE).unwrap(); + let plan = ctx.create_logical_plan(&sql).unwrap(); + println!("Plan: {:?}", plan); + let results = ctx.execute(&plan, DEFAULT_BATCH_SIZE).unwrap(); result_str(&results) } From b4981edb78c83a31a2245288599ae23f665b4703 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 10 Mar 2019 08:16:29 -0600 Subject: [PATCH 10/37] implement more parquet column types and tests --- rust/datafusion/src/datasource/parquet.rs | 256 +++++++++++++++++++--- 1 file changed, 222 insertions(+), 34 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 4447fbdc55932..cb8597febf525 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -35,7 +35,9 @@ use parquet::schema::types::Type; use crate::datasource::{RecordBatchIterator, Table}; use crate::execution::error::{ExecutionError, Result}; -use arrow::builder::{BinaryBuilder, Float64Builder, Int32Builder}; +use arrow::builder::BooleanBuilder; +use arrow::builder::Int64Builder; +use arrow::builder::{BinaryBuilder, Float32Builder, Float64Builder, Int32Builder}; pub struct ParquetTable { filename: String, @@ -112,7 +114,6 @@ impl ParquetFile { .collect(); let projected_schema = Arc::new(Schema::new(projected_fields)); - println!("projected schema: {:?}", projected_schema); Ok(ParquetFile { reader: reader, @@ -132,7 +133,6 @@ impl ParquetFile { fn load_next_row_group(&mut self) { if self.row_group_index < self.reader.num_row_groups() { - //println!("Loading row group {} of {}", self.row_group_index, self.reader.num_row_groups()); let reader = self.reader.get_row_group(self.row_group_index).unwrap(); self.column_readers = vec![]; @@ -151,17 +151,40 @@ impl ParquetFile { } fn load_batch(&mut self) -> Result> { - println!("load_batch()"); match &self.current_row_group { Some(reader) => { let mut batch: Vec> = Vec::with_capacity(reader.num_columns()); let mut row_count = 0; for i in 0..self.column_readers.len() { let array: Arc = match self.column_readers[i] { - ColumnReader::BoolColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (BOOL)".to_string(), - )); + ColumnReader::BoolColumnReader(ref mut r) => { + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); + + for _ in 0..self.batch_size { + read_buffer.push(false); + } + + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + //TODO this isn't handling null values + Ok((count, _)) => { + let mut builder = BooleanBuilder::new(count); + builder.append_slice(&read_buffer[0..count]).unwrap(); + row_count = count; + Arc::new(builder.finish()) + } + Err(e) => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {}): {:?}", + i, e + ))); + } + } } ColumnReader::Int32ColumnReader(ref mut r) => { let mut read_buffer: Vec = @@ -179,39 +202,87 @@ impl ParquetFile { ) { //TODO this isn't handling null values Ok((count, _)) => { - println!("Read {} rows", count); let mut builder = Int32Builder::new(count); builder.append_slice(&read_buffer[0..count]).unwrap(); row_count = count; Arc::new(builder.finish()) } - _ => { + Err(e) => { return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {})", - i + "Error reading parquet batch (column {}): {:?}", + i, e ))); } } } - ColumnReader::Int64ColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (INT64)".to_string(), - )); + ColumnReader::Int64ColumnReader(ref mut r) => { + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); + + for _ in 0..self.batch_size { + read_buffer.push(0); + } + + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + //TODO this isn't handling null values + Ok((count, _)) => { + let mut builder = Int64Builder::new(count); + builder.append_slice(&read_buffer[0..count]).unwrap(); + row_count = count; + Arc::new(builder.finish()) + } + Err(e) => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {}): {:?}", + i, e + ))); + } + } } ColumnReader::Int96ColumnReader(ref mut _r) => { return Err(ExecutionError::NotImplemented( "unsupported column reader type (INT96)".to_string(), )); } - ColumnReader::FloatColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (FLOAT)".to_string(), - )); + ColumnReader::FloatColumnReader(ref mut r) => { + let mut builder = Float32Builder::new(self.batch_size); + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); + for _ in 0..self.batch_size { + read_buffer.push(0.0); + } + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + //TODO this isn't handling null values + Ok((count, _)) => { + builder.append_slice(&read_buffer[0..count]).unwrap(); + row_count = count; + Arc::new(builder.finish()) + } + Err(e) => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {}): {:?}", + i, e + ))); + } + } } ColumnReader::DoubleColumnReader(ref mut r) => { let mut builder = Float64Builder::new(self.batch_size); let mut read_buffer: Vec = Vec::with_capacity(self.batch_size); + for _ in 0..self.batch_size { + read_buffer.push(0.0); + } match r.read_batch( self.batch_size, None, @@ -220,14 +291,14 @@ impl ParquetFile { ) { //TODO this isn't handling null values Ok((count, _)) => { - builder.append_slice(&read_buffer).unwrap(); + builder.append_slice(&read_buffer[0..count]).unwrap(); row_count = count; Arc::new(builder.finish()) } - _ => { + Err(e) => { return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {})", - i + "Error reading parquet batch (column {}): {:?}", + i, e ))); } } @@ -251,11 +322,13 @@ impl ParquetFile { //TODO this is horribly inefficient let mut builder = BinaryBuilder::new(row_count); for j in 0..row_count { - let foo = b[j].slice(0, b[j].len()); - let bytes: &[u8] = foo.data(); - let str = - String::from_utf8(bytes.to_vec()).unwrap(); - builder.append_string(&str).unwrap(); + let slice = b[j].slice(0, b[j].len()); + builder + .append_string( + &String::from_utf8(slice.data().to_vec()) + .unwrap(), + ) + .unwrap(); } Arc::new(builder.finish()) } @@ -354,11 +427,43 @@ impl RecordBatchIterator for ParquetFile { #[cfg(test)] mod tests { use super::*; + use arrow::array::BooleanArray; + use arrow::array::Float32Array; + use arrow::array::Float64Array; + use arrow::array::Int64Array; use arrow::array::{BinaryArray, Int32Array}; use std::env; #[test] - fn read_read_i32_column() { + fn read_bool_alltypes_plain_parquet() { + let table = load_table("alltypes_plain.parquet"); + + let projection = Some(vec![1]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().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!( + "[true, false, true, false, true, false, true, false]", + format!("{:?}", values) + ); + } + + #[test] + fn read_i32_alltypes_plain_parquet() { let table = load_table("alltypes_plain.parquet"); let projection = Some(vec![0]); @@ -383,7 +488,63 @@ mod tests { } #[test] - fn read_read_string_column() { + fn read_f32_alltypes_plain_parquet() { + let table = load_table("alltypes_plain.parquet"); + + let projection = Some(vec![6]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().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) + ); + } + + #[test] + fn read_f64_alltypes_plain_parquet() { + let table = load_table("alltypes_plain.parquet"); + + let projection = Some(vec![7]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().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, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", + format!("{:?}", values) + ); + } + + #[test] + fn read_utf8_alltypes_plain_parquet() { let table = load_table("alltypes_plain.parquet"); let projection = Some(vec![9]); @@ -400,7 +561,7 @@ mod tests { .downcast_ref::() .unwrap(); let mut values: Vec = vec![]; - for i in 0..8 { + for i in 0..batch.num_rows() { let str: String = String::from_utf8(array.value(i).to_vec()).unwrap(); values.push(str); } @@ -411,12 +572,39 @@ mod tests { ); } + #[test] + fn read_int64_nullable_impala_parquet() { + let table = load_table("nullable.impala.parquet"); + + let projection = Some(vec![0]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().unwrap(); + + assert_eq!(1, batch.num_columns()); + assert_eq!(7, 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!("[1, 2, 3, 4, 5, 6, 7]", format!("{:?}", values)); + } + fn load_table(name: &str) -> Box
{ - println!("load_table"); let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); let table = ParquetTable::new(&filename); - println!("{:?}", table.schema()); + println!("Loading file {} with schema:", name); + for field in table.schema().fields() { + println!("\t{:?}", field); + } Box::new(table) } } From 6c3b7e234123d1366718e5a301a8c9c41c7eb62b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 10 Mar 2019 08:27:57 -0600 Subject: [PATCH 11/37] add support for all primitive parquet types --- rust/datafusion/src/datasource/parquet.rs | 129 ++++++++++++++++++---- 1 file changed, 105 insertions(+), 24 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index cb8597febf525..0b1c0818c9aff 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -38,6 +38,7 @@ use crate::execution::error::{ExecutionError, Result}; use arrow::builder::BooleanBuilder; use arrow::builder::Int64Builder; use arrow::builder::{BinaryBuilder, Float32Builder, Float64Builder, Int32Builder}; +use parquet::data_type::Int96; pub struct ParquetTable { filename: String, @@ -171,7 +172,6 @@ impl ParquetFile { None, &mut read_buffer, ) { - //TODO this isn't handling null values Ok((count, _)) => { let mut builder = BooleanBuilder::new(count); builder.append_slice(&read_buffer[0..count]).unwrap(); @@ -200,7 +200,6 @@ impl ParquetFile { None, &mut read_buffer, ) { - //TODO this isn't handling null values Ok((count, _)) => { let mut builder = Int32Builder::new(count); builder.append_slice(&read_buffer[0..count]).unwrap(); @@ -229,7 +228,6 @@ impl ParquetFile { None, &mut read_buffer, ) { - //TODO this isn't handling null values Ok((count, _)) => { let mut builder = Int64Builder::new(count); builder.append_slice(&read_buffer[0..count]).unwrap(); @@ -244,10 +242,41 @@ impl ParquetFile { } } } - ColumnReader::Int96ColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (INT96)".to_string(), - )); + ColumnReader::Int96ColumnReader(ref mut r) => { + let mut read_buffer: Vec = + Vec::with_capacity(self.batch_size); + + for _ in 0..self.batch_size { + read_buffer.push(Int96::new()); + } + + match r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + ) { + Ok((count, _)) => { + let mut builder = Int64Builder::new(count); + + for i in 0..count { + let v = read_buffer[i].data(); + let value: u128 = (v[0] as u128) << 64 + | (v[1] as u128) << 32 + | (v[2] as u128); + let ms: i64 = (value / 1000000) as i64; + builder.append_value(ms).unwrap(); + } + row_count = count; + Arc::new(builder.finish()) + } + Err(e) => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {}): {:?}", + i, e + ))); + } + } } ColumnReader::FloatColumnReader(ref mut r) => { let mut builder = Float32Builder::new(self.batch_size); @@ -262,7 +291,6 @@ impl ParquetFile { None, &mut read_buffer, ) { - //TODO this isn't handling null values Ok((count, _)) => { builder.append_slice(&read_buffer[0..count]).unwrap(); row_count = count; @@ -289,7 +317,6 @@ impl ParquetFile { None, &mut read_buffer, ) { - //TODO this isn't handling null values Ok((count, _)) => { builder.append_slice(&read_buffer[0..count]).unwrap(); row_count = count; @@ -303,11 +330,34 @@ impl ParquetFile { } } } - ColumnReader::FixedLenByteArrayColumnReader(ref mut _r) => { - return Err(ExecutionError::NotImplemented( - "unsupported column reader type (FixedLenByteArray)" - .to_string(), - )); + ColumnReader::FixedLenByteArrayColumnReader(ref mut r) => { + let mut b: Vec = + Vec::with_capacity(self.batch_size); + for _ in 0..self.batch_size { + b.push(ByteArray::default()); + } + match r.read_batch(self.batch_size, None, None, &mut b) { + Ok((count, _)) => { + row_count = count; + let mut builder = BinaryBuilder::new(row_count); + for j in 0..row_count { + let slice = b[j].slice(0, b[j].len()); + builder + .append_string( + &String::from_utf8(slice.data().to_vec()) + .unwrap(), + ) + .unwrap(); + } + Arc::new(builder.finish()) + } + _ => { + return Err(ExecutionError::NotImplemented(format!( + "Error reading parquet batch (column {})", + i + ))); + } + } } ColumnReader::ByteArrayColumnReader(ref mut r) => { let mut b: Vec = @@ -316,10 +366,8 @@ impl ParquetFile { b.push(ByteArray::default()); } match r.read_batch(self.batch_size, None, None, &mut b) { - //TODO this isn't handling null values Ok((count, _)) => { row_count = count; - //TODO this is horribly inefficient let mut builder = BinaryBuilder::new(row_count); for j in 0..row_count { let slice = b[j].slice(0, b[j].len()); @@ -370,16 +418,11 @@ fn to_arrow(t: &Type) -> Result { basic::Type::BOOLEAN => DataType::Boolean, basic::Type::INT32 => DataType::Int32, basic::Type::INT64 => DataType::Int64, - basic::Type::INT96 => DataType::Int64, //TODO ??? + basic::Type::INT96 => DataType::Int64, basic::Type::FLOAT => DataType::Float32, basic::Type::DOUBLE => DataType::Float64, - basic::Type::BYTE_ARRAY => DataType::Utf8, /*match basic_info.logical_type() { - basic::LogicalType::UTF8 => DataType::Utf8, - _ => unimplemented!("No support for Parquet BYTE_ARRAY yet"), - }*/ - basic::Type::FIXED_LEN_BYTE_ARRAY => { - unimplemented!("No support for Parquet FIXED_LEN_BYTE_ARRAY yet") - } + basic::Type::BYTE_ARRAY => DataType::Utf8, + basic::Type::FIXED_LEN_BYTE_ARRAY => DataType::Utf8, }; Ok(Field::new(basic_info.name(), arrow_type, false)) @@ -434,6 +477,19 @@ mod tests { use arrow::array::{BinaryArray, Int32Array}; use std::env; + #[test] + fn read_alltypes_plain_parquet() { + let table = load_table("alltypes_plain.parquet"); + + let projection = None; + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().unwrap(); + + assert_eq!(11, batch.num_columns()); + assert_eq!(8, batch.num_rows()); + } + #[test] fn read_bool_alltypes_plain_parquet() { let table = load_table("alltypes_plain.parquet"); @@ -487,6 +543,31 @@ mod tests { assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); } + #[test] + fn read_i96_alltypes_plain_parquet() { + let table = load_table("alltypes_plain.parquet"); + + let projection = Some(vec![10]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan.borrow_mut(); + let batch = it.next().unwrap().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!("[2, 7842670136425819125, 2, 7842670136425819125, 2, 7842670136425819125, 2, 7842670136425819125]", format!("{:?}", values)); + } + #[test] fn read_f32_alltypes_plain_parquet() { let table = load_table("alltypes_plain.parquet"); From debb2fb308974b7c13bc5fd298a543853fa98d2f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 10 Mar 2019 08:37:41 -0600 Subject: [PATCH 12/37] code cleanup --- rust/arrow/src/datatypes.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/rust/arrow/src/datatypes.rs b/rust/arrow/src/datatypes.rs index 69ce7114790c8..e0fab5c9a8762 100644 --- a/rust/arrow/src/datatypes.rs +++ b/rust/arrow/src/datatypes.rs @@ -754,9 +754,18 @@ impl Schema { } /// Create a new schema by applying a projection to this schema's fields - pub fn projection(&self, i: &Vec) -> Result> { - //TODO bounds checks - let fields = i.iter().map(|index| self.field(*index).clone()).collect(); + pub fn projection(&self, projection: &Vec) -> Result> { + let mut fields: Vec = Vec::with_capacity(projection.len()); + for i in projection { + if *i < self.fields().len() { + fields.push(self.field(*i).clone()); + } else { + return Err(ArrowError::InvalidArgumentError(format!( + "Invalid column index {} in projection", + i + ))); + } + } Ok(Arc::new(Schema::new(fields))) } } From 157512e81f9e18ab0622a878e49154105ccd6e02 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 10 Mar 2019 08:40:10 -0600 Subject: [PATCH 13/37] Remove invalid TODO comment --- rust/datafusion/src/datasource/parquet.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 0b1c0818c9aff..c0287df2f6fb5 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -139,7 +139,6 @@ impl ParquetFile { self.column_readers = vec![]; for i in &self.projection { - //TODO validate index in bounds self.column_readers .push(reader.get_column_reader(*i).unwrap()); } From dddb7d738da7125c7ea7aae3a900e90c2eeb0032 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 10 Mar 2019 11:08:00 -0600 Subject: [PATCH 14/37] update to use partition-aware changes from master --- rust/datafusion/src/datasource/parquet.rs | 26 +++++++++++------------ 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index c0287df2f6fb5..da05e3ac46009 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -17,11 +17,9 @@ //! Parquet Data source -use std::cell::RefCell; use std::fs::File; -use std::rc::Rc; use std::string::String; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use arrow::array::Array; use arrow::datatypes::{DataType, Field, Schema}; @@ -33,7 +31,7 @@ use parquet::data_type::ByteArray; use parquet::file::reader::*; use parquet::schema::types::Type; -use crate::datasource::{RecordBatchIterator, Table}; +use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; use arrow::builder::BooleanBuilder; use arrow::builder::Int64Builder; @@ -66,10 +64,10 @@ impl Table for ParquetTable { &self, projection: &Option>, _batch_size: usize, - ) -> Result>> { + ) -> Result> { let file = File::open(self.filename.clone()).unwrap(); let parquet_file = ParquetFile::open(file, projection.clone()).unwrap(); - Ok(Rc::new(RefCell::new(parquet_file))) + Ok(vec![Arc::new(Mutex::new(parquet_file))]) } } @@ -482,7 +480,7 @@ mod tests { let projection = None; let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(11, batch.num_columns()); @@ -495,7 +493,7 @@ mod tests { let projection = Some(vec![1]); let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); @@ -523,7 +521,7 @@ mod tests { let projection = Some(vec![0]); let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); @@ -548,7 +546,7 @@ mod tests { let projection = Some(vec![10]); let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); @@ -573,7 +571,7 @@ mod tests { let projection = Some(vec![6]); let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); @@ -601,7 +599,7 @@ mod tests { let projection = Some(vec![7]); let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); @@ -629,7 +627,7 @@ mod tests { let projection = Some(vec![9]); let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); @@ -658,7 +656,7 @@ mod tests { let projection = Some(vec![0]); let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan.borrow_mut(); + let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); assert_eq!(1, batch.num_columns()); From 7e1a98f332ee5aa5ad2ac1a1a85b9c434e9f84e8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 10 Mar 2019 17:06:25 -0600 Subject: [PATCH 15/37] remove println and unwrap --- rust/datafusion/src/datasource/parquet.rs | 75 ++++++++++------------- rust/datafusion/src/execution/error.rs | 8 +++ rust/datafusion/tests/sql.rs | 2 +- 3 files changed, 43 insertions(+), 42 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index da05e3ac46009..6ab2a14822e9b 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -44,14 +44,14 @@ pub struct ParquetTable { } impl ParquetTable { - pub fn new(filename: &str) -> Self { - let file = File::open(filename).unwrap(); - let parquet_file = ParquetFile::open(file, None).unwrap(); + pub fn try_new(filename: &str) -> Result { + let file = File::open(filename)?; + let parquet_file = ParquetFile::open(file, None)?; let schema = parquet_file.schema.clone(); - Self { + Ok(Self { filename: filename.to_string(), schema, - } + }) } } @@ -65,8 +65,8 @@ impl Table for ParquetTable { projection: &Option>, _batch_size: usize, ) -> Result> { - let file = File::open(self.filename.clone()).unwrap(); - let parquet_file = ParquetFile::open(file, projection.clone()).unwrap(); + let file = File::open(self.filename.clone())?; + let parquet_file = ParquetFile::open(file, projection.clone())?; Ok(vec![Arc::new(Mutex::new(parquet_file))]) } } @@ -84,9 +84,7 @@ pub struct ParquetFile { impl ParquetFile { pub fn open(file: File, projection: Option>) -> Result { - println!("open()"); - - let reader = SerializedFileReader::new(file).unwrap(); + let reader = SerializedFileReader::new(file)?; let metadata = reader.metadata(); let file_type = to_arrow(metadata.file_metadata().schema())?; @@ -94,7 +92,6 @@ impl ParquetFile { match file_type.data_type() { DataType::Struct(fields) => { let schema = Schema::new(fields.clone()); - //println!("Parquet schema: {:?}", schema); let projection = match projection { Some(p) => p, @@ -130,21 +127,24 @@ impl ParquetFile { } } - fn load_next_row_group(&mut self) { + fn load_next_row_group(&mut self) -> Result<()> { if self.row_group_index < self.reader.num_row_groups() { - let reader = self.reader.get_row_group(self.row_group_index).unwrap(); + let reader = self.reader.get_row_group(self.row_group_index)?; - self.column_readers = vec![]; + self.column_readers = Vec::with_capacity(self.projection.len()); for i in &self.projection { - self.column_readers - .push(reader.get_column_reader(*i).unwrap()); + self.column_readers.push(reader.get_column_reader(*i)?); } self.current_row_group = Some(reader); self.row_group_index += 1; + + Ok(()) } else { - panic!() + Err(ExecutionError::General( + "Attempt to read past final row group".to_string(), + )) } } @@ -171,7 +171,7 @@ impl ParquetFile { ) { Ok((count, _)) => { let mut builder = BooleanBuilder::new(count); - builder.append_slice(&read_buffer[0..count]).unwrap(); + builder.append_slice(&read_buffer[0..count])?; row_count = count; Arc::new(builder.finish()) } @@ -199,7 +199,7 @@ impl ParquetFile { ) { Ok((count, _)) => { let mut builder = Int32Builder::new(count); - builder.append_slice(&read_buffer[0..count]).unwrap(); + builder.append_slice(&read_buffer[0..count])?; row_count = count; Arc::new(builder.finish()) } @@ -227,7 +227,7 @@ impl ParquetFile { ) { Ok((count, _)) => { let mut builder = Int64Builder::new(count); - builder.append_slice(&read_buffer[0..count]).unwrap(); + builder.append_slice(&read_buffer[0..count])?; row_count = count; Arc::new(builder.finish()) } @@ -262,7 +262,7 @@ impl ParquetFile { | (v[1] as u128) << 32 | (v[2] as u128); let ms: i64 = (value / 1000000) as i64; - builder.append_value(ms).unwrap(); + builder.append_value(ms)?; } row_count = count; Arc::new(builder.finish()) @@ -289,7 +289,7 @@ impl ParquetFile { &mut read_buffer, ) { Ok((count, _)) => { - builder.append_slice(&read_buffer[0..count]).unwrap(); + builder.append_slice(&read_buffer[0..count])?; row_count = count; Arc::new(builder.finish()) } @@ -315,7 +315,7 @@ impl ParquetFile { &mut read_buffer, ) { Ok((count, _)) => { - builder.append_slice(&read_buffer[0..count]).unwrap(); + builder.append_slice(&read_buffer[0..count])?; row_count = count; Arc::new(builder.finish()) } @@ -339,12 +339,10 @@ impl ParquetFile { let mut builder = BinaryBuilder::new(row_count); for j in 0..row_count { let slice = b[j].slice(0, b[j].len()); - builder - .append_string( - &String::from_utf8(slice.data().to_vec()) - .unwrap(), - ) - .unwrap(); + builder.append_string( + &String::from_utf8(slice.data().to_vec()) + .unwrap(), + )?; } Arc::new(builder.finish()) } @@ -368,12 +366,10 @@ impl ParquetFile { let mut builder = BinaryBuilder::new(row_count); for j in 0..row_count { let slice = b[j].slice(0, b[j].len()); - builder - .append_string( - &String::from_utf8(slice.data().to_vec()) - .unwrap(), - ) - .unwrap(); + builder.append_string( + &String::from_utf8(slice.data().to_vec()) + .unwrap(), + )?; } Arc::new(builder.finish()) } @@ -387,12 +383,9 @@ impl ParquetFile { } }; - println!("Adding array to batch"); batch.push(array); } - println!("Loaded batch of {} rows", row_count); - if row_count == 0 { Ok(None) } else { @@ -445,14 +438,14 @@ impl RecordBatchIterator for ParquetFile { fn next(&mut self) -> Result> { // advance the row group reader if necessary if self.current_row_group.is_none() { - self.load_next_row_group(); + self.load_next_row_group()?; self.load_batch() } else { match self.load_batch() { Ok(Some(b)) => Ok(Some(b)), Ok(None) => { if self.row_group_index < self.reader.num_row_groups() { - self.load_next_row_group(); + self.load_next_row_group()?; self.load_batch() } else { Ok(None) @@ -678,7 +671,7 @@ mod tests { fn load_table(name: &str) -> Box
{ let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); - let table = ParquetTable::new(&filename); + let table = ParquetTable::try_new(&filename).unwrap(); println!("Loading file {} with schema:", name); for field in table.schema().fields() { println!("\t{:?}", field); diff --git a/rust/datafusion/src/execution/error.rs b/rust/datafusion/src/execution/error.rs index 5b8d04d3dca34..92ce6d91c10a7 100644 --- a/rust/datafusion/src/execution/error.rs +++ b/rust/datafusion/src/execution/error.rs @@ -21,6 +21,7 @@ use std::io::Error; use std::result; use arrow::error::ArrowError; +use parquet::errors::ParquetError; use sqlparser::sqlparser::ParserError; @@ -35,6 +36,7 @@ pub enum ExecutionError { NotImplemented(String), InternalError(String), ArrowError(ArrowError), + ParquetError(ParquetError), ExecutionError(String), } @@ -62,6 +64,12 @@ impl From for ExecutionError { } } +impl From for ExecutionError { + fn from(e: ParquetError) -> Self { + ExecutionError::ParquetError(e) + } +} + impl From for ExecutionError { fn from(e: ParserError) -> Self { ExecutionError::ParserError(e) diff --git a/rust/datafusion/tests/sql.rs b/rust/datafusion/tests/sql.rs index 1843974055998..a6c368794cc7b 100644 --- a/rust/datafusion/tests/sql.rs +++ b/rust/datafusion/tests/sql.rs @@ -182,7 +182,7 @@ fn register_csv( fn load_parquet_table(name: &str) -> Rc
{ let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); - let table = ParquetTable::new(&filename); + let table = ParquetTable::try_new(&filename).unwrap(); println!("{:?}", table.schema()); Rc::new(table) } From c56510e5fd7e9790005fb256e3a023336a6eeaab Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 10 Mar 2019 17:09:42 -0600 Subject: [PATCH 16/37] projection takes slice instead of vec --- rust/arrow/src/datatypes.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/arrow/src/datatypes.rs b/rust/arrow/src/datatypes.rs index e0fab5c9a8762..34860350e7357 100644 --- a/rust/arrow/src/datatypes.rs +++ b/rust/arrow/src/datatypes.rs @@ -754,7 +754,7 @@ impl Schema { } /// Create a new schema by applying a projection to this schema's fields - pub fn projection(&self, projection: &Vec) -> Result> { + pub fn projection(&self, projection: &[usize]) -> Result> { let mut fields: Vec = Vec::with_capacity(projection.len()); for i in projection { if *i < self.fields().len() { From 6457c36c7215ea016720750fb3d7d1d7bc0bc571 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 11 Mar 2019 18:02:29 -0600 Subject: [PATCH 17/37] use parquet::reader::schema::parquet_to_arrow_schema --- rust/datafusion/src/datasource/parquet.rs | 104 +++++++--------------- rust/parquet/src/reader/schema.rs | 10 ++- 2 files changed, 37 insertions(+), 77 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 6ab2a14822e9b..6546f187fb280 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -22,14 +22,12 @@ use std::string::String; use std::sync::{Arc, Mutex}; use arrow::array::Array; -use arrow::datatypes::{DataType, Field, Schema}; +use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; -use parquet::basic; use parquet::column::reader::*; use parquet::data_type::ByteArray; use parquet::file::reader::*; -use parquet::schema::types::Type; use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; @@ -37,6 +35,7 @@ use arrow::builder::BooleanBuilder; use arrow::builder::Int64Builder; use arrow::builder::{BinaryBuilder, Float32Builder, Float64Builder, Int32Builder}; use parquet::data_type::Int96; +use parquet::reader::schema::parquet_to_arrow_schema; pub struct ParquetTable { filename: String, @@ -87,44 +86,36 @@ impl ParquetFile { let reader = SerializedFileReader::new(file)?; let metadata = reader.metadata(); - let file_type = to_arrow(metadata.file_metadata().schema())?; - - match file_type.data_type() { - DataType::Struct(fields) => { - let schema = Schema::new(fields.clone()); - - let projection = match projection { - Some(p) => p, - None => { - let mut p = Vec::with_capacity(schema.fields().len()); - for i in 0..schema.fields().len() { - p.push(i); - } - p - } - }; - - let projected_fields: Vec = projection - .iter() - .map(|i| schema.fields()[*i].clone()) - .collect(); - - let projected_schema = Arc::new(Schema::new(projected_fields)); - - Ok(ParquetFile { - reader: reader, - row_group_index: 0, - schema: projected_schema, - projection, - batch_size: 64 * 1024, - current_row_group: None, - column_readers: vec![], - }) + let schema = + parquet_to_arrow_schema(metadata.file_metadata().schema_descr_ptr())?; + + let projection = match projection { + Some(p) => p, + None => { + let mut p = Vec::with_capacity(schema.fields().len()); + for i in 0..schema.fields().len() { + p.push(i); + } + p } - _ => Err(ExecutionError::General( - "Failed to read Parquet schema".to_string(), - )), - } + }; + + let projected_fields: Vec = projection + .iter() + .map(|i| schema.fields()[*i].clone()) + .collect(); + + let projected_schema = Arc::new(Schema::new(projected_fields)); + + Ok(ParquetFile { + reader: reader, + row_group_index: 0, + schema: projected_schema, + projection, + batch_size: 64 * 1024, + current_row_group: None, + column_readers: vec![], + }) } fn load_next_row_group(&mut self) -> Result<()> { @@ -397,39 +388,6 @@ impl ParquetFile { } } -fn to_arrow(t: &Type) -> Result { - match t { - Type::PrimitiveType { - basic_info, - physical_type, - .. - } => { - let arrow_type = match physical_type { - basic::Type::BOOLEAN => DataType::Boolean, - basic::Type::INT32 => DataType::Int32, - basic::Type::INT64 => DataType::Int64, - basic::Type::INT96 => DataType::Int64, - basic::Type::FLOAT => DataType::Float32, - basic::Type::DOUBLE => DataType::Float64, - basic::Type::BYTE_ARRAY => DataType::Utf8, - basic::Type::FIXED_LEN_BYTE_ARRAY => DataType::Utf8, - }; - - Ok(Field::new(basic_info.name(), arrow_type, false)) - } - Type::GroupType { basic_info, fields } => Ok(Field::new( - basic_info.name(), - DataType::Struct( - fields - .iter() - .map(|f| to_arrow(f)) - .collect::>>()?, - ), - false, - )), - } -} - impl RecordBatchIterator for ParquetFile { fn schema(&self) -> &Arc { &self.schema diff --git a/rust/parquet/src/reader/schema.rs b/rust/parquet/src/reader/schema.rs index 34276a2d5633f..bf8c43d6cfa05 100644 --- a/rust/parquet/src/reader/schema.rs +++ b/rust/parquet/src/reader/schema.rs @@ -177,11 +177,12 @@ impl ParquetTypeConverter { PhysicalType::BOOLEAN => Ok(DataType::Boolean), PhysicalType::INT32 => self.to_int32(), PhysicalType::INT64 => self.to_int64(), + PhysicalType::INT96 => self.to_int64(), PhysicalType::FLOAT => Ok(DataType::Float32), PhysicalType::DOUBLE => Ok(DataType::Float64), PhysicalType::BYTE_ARRAY => self.to_byte_array(), other => Err(ArrowError(format!( - "Unable to convert parquet type {}", + "Unable to convert parquet type d {}", other ))), } @@ -197,7 +198,7 @@ impl ParquetTypeConverter { LogicalType::INT_16 => Ok(DataType::Int16), LogicalType::INT_32 => Ok(DataType::Int32), other => Err(ArrowError(format!( - "Unable to convert parquet logical type {}", + "Unable to convert parquet logical type a {}", other ))), } @@ -209,7 +210,7 @@ impl ParquetTypeConverter { LogicalType::INT_64 => Ok(DataType::Int64), LogicalType::UINT_64 => Ok(DataType::UInt64), other => Err(ArrowError(format!( - "Unable to convert parquet logical type {}", + "Unable to convert parquet logical type b {}", other ))), } @@ -217,9 +218,10 @@ impl ParquetTypeConverter { fn to_byte_array(&self) -> Result { match self.schema.get_basic_info().logical_type() { + LogicalType::NONE => Ok(DataType::Utf8), LogicalType::UTF8 => Ok(DataType::Utf8), other => Err(ArrowError(format!( - "Unable to convert parquet logical type {}", + "Unable to convert parquet logical type c {}", other ))), } From e8aa784a5007d52c5295e75a1599c8335d9a1454 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 11 Mar 2019 18:05:56 -0600 Subject: [PATCH 18/37] revert temp debug change to error messages --- rust/parquet/src/reader/schema.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/rust/parquet/src/reader/schema.rs b/rust/parquet/src/reader/schema.rs index bf8c43d6cfa05..239937c89b742 100644 --- a/rust/parquet/src/reader/schema.rs +++ b/rust/parquet/src/reader/schema.rs @@ -182,7 +182,7 @@ impl ParquetTypeConverter { PhysicalType::DOUBLE => Ok(DataType::Float64), PhysicalType::BYTE_ARRAY => self.to_byte_array(), other => Err(ArrowError(format!( - "Unable to convert parquet type d {}", + "Unable to convert parquet type {}", other ))), } @@ -198,7 +198,7 @@ impl ParquetTypeConverter { LogicalType::INT_16 => Ok(DataType::Int16), LogicalType::INT_32 => Ok(DataType::Int32), other => Err(ArrowError(format!( - "Unable to convert parquet logical type a {}", + "Unable to convert parquet logical type {}", other ))), } @@ -210,7 +210,7 @@ impl ParquetTypeConverter { LogicalType::INT_64 => Ok(DataType::Int64), LogicalType::UINT_64 => Ok(DataType::UInt64), other => Err(ArrowError(format!( - "Unable to convert parquet logical type b {}", + "Unable to convert parquet logical type {}", other ))), } @@ -221,7 +221,7 @@ impl ParquetTypeConverter { LogicalType::NONE => Ok(DataType::Utf8), LogicalType::UTF8 => Ok(DataType::Utf8), other => Err(ArrowError(format!( - "Unable to convert parquet logical type c {}", + "Unable to convert parquet logical type {}", other ))), } From 607a29f033bb8bce38c6834d57ae47bd30839bf0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 11 Mar 2019 20:50:37 -0600 Subject: [PATCH 19/37] return result if there are null values --- rust/datafusion/src/datasource/parquet.rs | 373 ++++++++++------------ 1 file changed, 170 insertions(+), 203 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 6546f187fb280..cd24d6357bef7 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -22,7 +22,7 @@ use std::string::String; use std::sync::{Arc, Mutex}; use arrow::array::Array; -use arrow::datatypes::{Field, Schema}; +use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; use parquet::column::reader::*; @@ -31,6 +31,7 @@ use parquet::file::reader::*; use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; +use arrow::array::BinaryArray; use arrow::builder::BooleanBuilder; use arrow::builder::Int64Builder; use arrow::builder::{BinaryBuilder, Float32Builder, Float64Builder, Int32Builder}; @@ -81,6 +82,94 @@ pub struct ParquetFile { column_readers: Vec, } +fn create_binary_array(b: &Vec, row_count: usize) -> Result> { + let mut builder = BinaryBuilder::new(b.len()); + for j in 0..row_count { + let slice = b[j].slice(0, b[j].len()); + builder.append_string(&String::from_utf8(slice.data().to_vec()).unwrap())?; + } + Ok(Arc::new(builder.finish())) +} + +macro_rules! read_column { + ($SELF:ident, $R:ident, $INDEX:expr, $BUILDER:ident, $TY:ident, $DEFAULT:expr) => {{ + //TODO: should be able to get num_rows in row group instead of defaulting to batch size + let mut read_buffer: Vec<$TY> = Vec::with_capacity($SELF.batch_size); + for _ in 0..$SELF.batch_size { + read_buffer.push($DEFAULT); + } + if $SELF.schema.field($INDEX).is_nullable() { + + let mut def_levels: Vec = Vec::with_capacity($SELF.batch_size); + for _ in 0..$SELF.batch_size { + def_levels.push(0); + } + + let (values_read, levels_read) = $R.read_batch( + $SELF.batch_size, + Some(&mut def_levels), + None, + &mut read_buffer, + )?; + let mut builder = $BUILDER::new(levels_read); + if values_read == levels_read { + builder.append_slice(&read_buffer[0..values_read])?; + } else { + return Err(ExecutionError::NotImplemented("Parquet datasource does not support null values".to_string())) + } + Arc::new(builder.finish()) + } else { + let (values_read, _) = $R.read_batch( + $SELF.batch_size, + None, + None, + &mut read_buffer, + )?; + let mut builder = $BUILDER::new(values_read); + builder.append_slice(&read_buffer[0..values_read])?; + Arc::new(builder.finish()) + } + }} +} + +macro_rules! read_binary_column { + ($SELF:ident, $R:ident, $INDEX:expr) => {{ + //TODO: should be able to get num_rows in row group instead of defaulting to batch size + let mut read_buffer: Vec = + Vec::with_capacity($SELF.batch_size); + for _ in 0..$SELF.batch_size { + read_buffer.push(ByteArray::default()); + } + if $SELF.schema.field($INDEX).is_nullable() { + + let mut def_levels: Vec = Vec::with_capacity($SELF.batch_size); + for _ in 0..$SELF.batch_size { + def_levels.push(0); + } + + let (values_read, levels_read) = $R.read_batch( + $SELF.batch_size, + Some(&mut def_levels), + None, + &mut read_buffer, + )?; + if values_read == levels_read { + create_binary_array(&read_buffer, values_read)? + } else { + return Err(ExecutionError::NotImplemented("Parquet datasource does not support null values".to_string())) + } + } else { + let (values_read, _) = $R.read_batch( + $SELF.batch_size, + None, + None, + &mut read_buffer, + )?; + create_binary_array(&read_buffer, values_read)? + } + }} +} + impl ParquetFile { pub fn open(file: File, projection: Option>) -> Result { let reader = SerializedFileReader::new(file)?; @@ -100,12 +189,7 @@ impl ParquetFile { } }; - let projected_fields: Vec = projection - .iter() - .map(|i| schema.fields()[*i].clone()) - .collect(); - - let projected_schema = Arc::new(Schema::new(projected_fields)); + let projected_schema = schema.projection(&projection)?; Ok(ParquetFile { reader: reader, @@ -124,8 +208,23 @@ impl ParquetFile { self.column_readers = Vec::with_capacity(self.projection.len()); - for i in &self.projection { - self.column_readers.push(reader.get_column_reader(*i)?); + for i in 0..self.projection.len() { + match self.schema().field(i).data_type() { + DataType::List(_) => { + return Err(ExecutionError::NotImplemented( + "Parquet datasource does not support LIST".to_string(), + )); + } + DataType::Struct(_) => { + return Err(ExecutionError::NotImplemented( + "Parquet datasource does not support STRUCT".to_string(), + )); + } + _ => {} + } + + self.column_readers + .push(reader.get_column_reader(self.projection[i])?); } self.current_row_group = Some(reader); @@ -143,92 +242,16 @@ impl ParquetFile { match &self.current_row_group { Some(reader) => { let mut batch: Vec> = Vec::with_capacity(reader.num_columns()); - let mut row_count = 0; for i in 0..self.column_readers.len() { let array: Arc = match self.column_readers[i] { ColumnReader::BoolColumnReader(ref mut r) => { - let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); - - for _ in 0..self.batch_size { - read_buffer.push(false); - } - - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - Ok((count, _)) => { - let mut builder = BooleanBuilder::new(count); - builder.append_slice(&read_buffer[0..count])?; - row_count = count; - Arc::new(builder.finish()) - } - Err(e) => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {}): {:?}", - i, e - ))); - } - } + read_column!(self, r, i, BooleanBuilder, bool, false) } ColumnReader::Int32ColumnReader(ref mut r) => { - let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); - - for _ in 0..self.batch_size { - read_buffer.push(0); - } - - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - Ok((count, _)) => { - let mut builder = Int32Builder::new(count); - builder.append_slice(&read_buffer[0..count])?; - row_count = count; - Arc::new(builder.finish()) - } - Err(e) => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {}): {:?}", - i, e - ))); - } - } + read_column!(self, r, i, Int32Builder, i32, 0) } ColumnReader::Int64ColumnReader(ref mut r) => { - let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); - - for _ in 0..self.batch_size { - read_buffer.push(0); - } - - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - Ok((count, _)) => { - let mut builder = Int64Builder::new(count); - builder.append_slice(&read_buffer[0..count])?; - row_count = count; - Arc::new(builder.finish()) - } - Err(e) => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {}): {:?}", - i, e - ))); - } - } + read_column!(self, r, i, Int64Builder, i64, 0) } ColumnReader::Int96ColumnReader(ref mut r) => { let mut read_buffer: Vec = @@ -238,16 +261,23 @@ impl ParquetFile { read_buffer.push(Int96::new()); } - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - Ok((count, _)) => { - let mut builder = Int64Builder::new(count); + if self.schema.field(i).is_nullable() { + let mut def_levels: Vec = + Vec::with_capacity(self.batch_size); + for _ in 0..self.batch_size { + def_levels.push(0); + } + let (values_read, levels_read) = r.read_batch( + self.batch_size, + Some(&mut def_levels), + None, + &mut read_buffer, + )?; + + if values_read == levels_read { + let mut builder = Int64Builder::new(values_read); - for i in 0..count { + for i in 0..values_read { let v = read_buffer[i].data(); let value: u128 = (v[0] as u128) << 64 | (v[1] as u128) << 32 @@ -255,129 +285,52 @@ impl ParquetFile { let ms: i64 = (value / 1000000) as i64; builder.append_value(ms)?; } - row_count = count; Arc::new(builder.finish()) + } else { + return Err(ExecutionError::NotImplemented( + "Parquet datasource does not support null values" + .to_string(), + )); } - Err(e) => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {}): {:?}", - i, e - ))); + } else { + let (values_read, _) = r.read_batch( + self.batch_size, + None, + None, + &mut read_buffer, + )?; + + let mut builder = Int64Builder::new(values_read); + + for i in 0..values_read { + let v = read_buffer[i].data(); + let value: u128 = (v[0] as u128) << 64 + | (v[1] as u128) << 32 + | (v[2] as u128); + let ms: i64 = (value / 1000000) as i64; + builder.append_value(ms)?; } + Arc::new(builder.finish()) } } ColumnReader::FloatColumnReader(ref mut r) => { - let mut builder = Float32Builder::new(self.batch_size); - let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); - for _ in 0..self.batch_size { - read_buffer.push(0.0); - } - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - Ok((count, _)) => { - builder.append_slice(&read_buffer[0..count])?; - row_count = count; - Arc::new(builder.finish()) - } - Err(e) => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {}): {:?}", - i, e - ))); - } - } + read_column!(self, r, i, Float32Builder, f32, 0_f32) } ColumnReader::DoubleColumnReader(ref mut r) => { - let mut builder = Float64Builder::new(self.batch_size); - let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); - for _ in 0..self.batch_size { - read_buffer.push(0.0); - } - match r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - ) { - Ok((count, _)) => { - builder.append_slice(&read_buffer[0..count])?; - row_count = count; - Arc::new(builder.finish()) - } - Err(e) => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {}): {:?}", - i, e - ))); - } - } + read_column!(self, r, i, Float64Builder, f64, 0_f64) } ColumnReader::FixedLenByteArrayColumnReader(ref mut r) => { - let mut b: Vec = - Vec::with_capacity(self.batch_size); - for _ in 0..self.batch_size { - b.push(ByteArray::default()); - } - match r.read_batch(self.batch_size, None, None, &mut b) { - Ok((count, _)) => { - row_count = count; - let mut builder = BinaryBuilder::new(row_count); - for j in 0..row_count { - let slice = b[j].slice(0, b[j].len()); - builder.append_string( - &String::from_utf8(slice.data().to_vec()) - .unwrap(), - )?; - } - Arc::new(builder.finish()) - } - _ => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {})", - i - ))); - } - } + read_binary_column!(self, r, i) } ColumnReader::ByteArrayColumnReader(ref mut r) => { - let mut b: Vec = - Vec::with_capacity(self.batch_size); - for _ in 0..self.batch_size { - b.push(ByteArray::default()); - } - match r.read_batch(self.batch_size, None, None, &mut b) { - Ok((count, _)) => { - row_count = count; - let mut builder = BinaryBuilder::new(row_count); - for j in 0..row_count { - let slice = b[j].slice(0, b[j].len()); - builder.append_string( - &String::from_utf8(slice.data().to_vec()) - .unwrap(), - )?; - } - Arc::new(builder.finish()) - } - _ => { - return Err(ExecutionError::NotImplemented(format!( - "Error reading parquet batch (column {})", - i - ))); - } - } + read_binary_column!(self, r, i) } }; batch.push(array); } - if row_count == 0 { + if batch.len() == 0 || batch[0].data().len() == 0 { Ok(None) } else { Ok(Some(RecordBatch::try_new(self.schema.clone(), batch)?)) @@ -604,8 +557,8 @@ mod tests { #[test] fn read_int64_nullable_impala_parquet() { let table = load_table("nullable.impala.parquet"); - let projection = Some(vec![0]); + let scan = table.scan(&projection, 1024).unwrap(); let mut it = scan[0].lock().unwrap(); let batch = it.next().unwrap().unwrap(); @@ -626,6 +579,20 @@ mod tests { assert_eq!("[1, 2, 3, 4, 5, 6, 7]", format!("{:?}", values)); } + #[test] + fn read_array_nullable_impala_parquet() { + let table = load_table("nullable.impala.parquet"); + let projection = Some(vec![1]); + let scan = table.scan(&projection, 1024).unwrap(); + let mut it = scan[0].lock().unwrap(); + let batch = it.next(); + + assert_eq!( + "NotImplemented(\"Parquet datasource does not support LIST\")", + format!("{:?}", batch.err().unwrap()) + ); + } + fn load_table(name: &str) -> Box
{ let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); From e6cbbaa45696ede62142b0f1bd53ec2367a0fe04 Mon Sep 17 00:00:00 2001 From: Neville Dipale Date: Wed, 13 Mar 2019 19:20:43 +0200 Subject: [PATCH 20/37] replace read_column! macro with generic --- rust/datafusion/src/datasource/parquet.rs | 118 ++++++++++++++-------- 1 file changed, 77 insertions(+), 41 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index cd24d6357bef7..1cc2d24d82529 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -21,8 +21,9 @@ use std::fs::File; use std::string::String; use std::sync::{Arc, Mutex}; -use arrow::array::Array; -use arrow::datatypes::{DataType, Schema}; +use arrow::array::{Array, PrimitiveArray}; +use arrow::builder::PrimitiveBuilder; +use arrow::datatypes::*; use arrow::record_batch::RecordBatch; use parquet::column::reader::*; @@ -32,9 +33,7 @@ use parquet::file::reader::*; use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; use arrow::array::BinaryArray; -use arrow::builder::BooleanBuilder; -use arrow::builder::Int64Builder; -use arrow::builder::{BinaryBuilder, Float32Builder, Float64Builder, Int32Builder}; +use arrow::builder::{BinaryBuilder, Int64Builder}; use parquet::data_type::Int96; use parquet::reader::schema::parquet_to_arrow_schema; @@ -91,12 +90,13 @@ fn create_binary_array(b: &Vec, row_count: usize) -> Result {{ +macro_rules! read_binary_column { + ($SELF:ident, $R:ident, $INDEX:expr) => {{ //TODO: should be able to get num_rows in row group instead of defaulting to batch size - let mut read_buffer: Vec<$TY> = Vec::with_capacity($SELF.batch_size); + let mut read_buffer: Vec = + Vec::with_capacity($SELF.batch_size); for _ in 0..$SELF.batch_size { - read_buffer.push($DEFAULT); + read_buffer.push(ByteArray::default()); } if $SELF.schema.field($INDEX).is_nullable() { @@ -111,13 +111,11 @@ macro_rules! read_column { None, &mut read_buffer, )?; - let mut builder = $BUILDER::new(levels_read); if values_read == levels_read { - builder.append_slice(&read_buffer[0..values_read])?; + create_binary_array(&read_buffer, values_read)? } else { return Err(ExecutionError::NotImplemented("Parquet datasource does not support null values".to_string())) } - Arc::new(builder.finish()) } else { let (values_read, _) = $R.read_batch( $SELF.batch_size, @@ -125,49 +123,71 @@ macro_rules! read_column { None, &mut read_buffer, )?; - let mut builder = $BUILDER::new(values_read); - builder.append_slice(&read_buffer[0..values_read])?; - Arc::new(builder.finish()) + create_binary_array(&read_buffer, values_read)? } }} } -macro_rules! read_binary_column { - ($SELF:ident, $R:ident, $INDEX:expr) => {{ - //TODO: should be able to get num_rows in row group instead of defaulting to batch size - let mut read_buffer: Vec = - Vec::with_capacity($SELF.batch_size); - for _ in 0..$SELF.batch_size { - read_buffer.push(ByteArray::default()); +trait ArrowReader where T: ArrowPrimitiveType { + fn read(&mut self, batch_size: usize, is_nullable: bool) -> Result>>; +} + +impl ArrowReader for ColumnReaderImpl

+where + A: ArrowPrimitiveType, + P: parquet::data_type::DataType, + P::T: std::convert::From, + A::Native: std::convert::From, +{ + fn read(&mut self, batch_size: usize, is_nullable: bool) -> Result>> { + + // create read buffer + let mut read_buffer: Vec = Vec::with_capacity(batch_size); + + for _ in 0..batch_size { + read_buffer.push(A::default_value().into()); } - if $SELF.schema.field($INDEX).is_nullable() { - let mut def_levels: Vec = Vec::with_capacity($SELF.batch_size); - for _ in 0..$SELF.batch_size { + if is_nullable { + let mut def_levels: Vec = Vec::with_capacity(batch_size); + for _ in 0..batch_size { def_levels.push(0); } - let (values_read, levels_read) = $R.read_batch( - $SELF.batch_size, + let (values_read, levels_read) = self.read_batch( + batch_size, Some(&mut def_levels), None, &mut read_buffer, )?; + let mut builder = PrimitiveBuilder::::new(levels_read); if values_read == levels_read { - create_binary_array(&read_buffer, values_read)? + let converted_buffer: Vec = read_buffer.into_iter().map(|v| v.into()).collect(); + builder.append_slice(&converted_buffer[0..values_read])?; } else { - return Err(ExecutionError::NotImplemented("Parquet datasource does not support null values".to_string())) + for (v, l) in read_buffer.into_iter().zip(def_levels) { + if l == 0 { + builder.append_value(v.into())?; + } else { + builder.append_null()?; + } + } } + Ok(Arc::new(builder.finish())) } else { - let (values_read, _) = $R.read_batch( - $SELF.batch_size, - None, - None, - &mut read_buffer, + let (values_read, _) = self.read_batch( + batch_size, + None, + None, + &mut read_buffer, )?; - create_binary_array(&read_buffer, values_read)? + + let mut builder = PrimitiveBuilder::::new(values_read); + let converted_buffer: Vec = read_buffer.into_iter().map(|v| v.into()).collect(); + builder.append_slice(&converted_buffer[0..values_read])?; + Ok(Arc::new(builder.finish())) } - }} + } } impl ParquetFile { @@ -243,15 +263,25 @@ impl ParquetFile { Some(reader) => { let mut batch: Vec> = Vec::with_capacity(reader.num_columns()); for i in 0..self.column_readers.len() { + let is_nullable = self.schema().field(i).is_nullable(); let array: Arc = match self.column_readers[i] { ColumnReader::BoolColumnReader(ref mut r) => { - read_column!(self, r, i, BooleanBuilder, bool, false) + match ArrowReader::::read(r, self.batch_size, is_nullable) { + Ok(array) => array, + Err(e) => return Err(e) + } } ColumnReader::Int32ColumnReader(ref mut r) => { - read_column!(self, r, i, Int32Builder, i32, 0) + match ArrowReader::::read(r, self.batch_size, is_nullable) { + Ok(array) => array, + Err(e) => return Err(e) + } } ColumnReader::Int64ColumnReader(ref mut r) => { - read_column!(self, r, i, Int64Builder, i64, 0) + match ArrowReader::::read(r, self.batch_size, is_nullable) { + Ok(array) => array, + Err(e) => return Err(e) + } } ColumnReader::Int96ColumnReader(ref mut r) => { let mut read_buffer: Vec = @@ -314,10 +344,16 @@ impl ParquetFile { } } ColumnReader::FloatColumnReader(ref mut r) => { - read_column!(self, r, i, Float32Builder, f32, 0_f32) + match ArrowReader::::read(r, self.batch_size, is_nullable) { + Ok(array) => array, + Err(e) => return Err(e) + } } ColumnReader::DoubleColumnReader(ref mut r) => { - read_column!(self, r, i, Float64Builder, f64, 0_f64) + match ArrowReader::::read(r, self.batch_size, is_nullable) { + Ok(array) => array, + Err(e) => return Err(e) + } } ColumnReader::FixedLenByteArrayColumnReader(ref mut r) => { read_binary_column!(self, r, i) From 3c711a58dd3693381b3c1ed625677205d8ed3d91 Mon Sep 17 00:00:00 2001 From: Neville Dipale Date: Wed, 13 Mar 2019 21:37:44 +0200 Subject: [PATCH 21/37] immediately allocate vec --- rust/datafusion/src/datasource/parquet.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 1cc2d24d82529..26e71407104be 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -142,11 +142,7 @@ where fn read(&mut self, batch_size: usize, is_nullable: bool) -> Result>> { // create read buffer - let mut read_buffer: Vec = Vec::with_capacity(batch_size); - - for _ in 0..batch_size { - read_buffer.push(A::default_value().into()); - } + let mut read_buffer: Vec = vec![A::default_value().into(); batch_size]; if is_nullable { let mut def_levels: Vec = Vec::with_capacity(batch_size); From 306d07a24385b7066b5c244e6c2445a9ac187099 Mon Sep 17 00:00:00 2001 From: Neville Dipale Date: Wed, 13 Mar 2019 21:38:39 +0200 Subject: [PATCH 22/37] fmt --- rust/datafusion/src/datasource/parquet.rs | 84 +++++++++++++++-------- 1 file changed, 56 insertions(+), 28 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 26e71407104be..b43af329c295f 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -128,19 +128,29 @@ macro_rules! read_binary_column { }} } -trait ArrowReader where T: ArrowPrimitiveType { - fn read(&mut self, batch_size: usize, is_nullable: bool) -> Result>>; +trait ArrowReader +where + T: ArrowPrimitiveType, +{ + fn read( + &mut self, + batch_size: usize, + is_nullable: bool, + ) -> Result>>; } -impl ArrowReader for ColumnReaderImpl

+impl ArrowReader for ColumnReaderImpl

where - A: ArrowPrimitiveType, + A: ArrowPrimitiveType, P: parquet::data_type::DataType, P::T: std::convert::From, A::Native: std::convert::From, { - fn read(&mut self, batch_size: usize, is_nullable: bool) -> Result>> { - + fn read( + &mut self, + batch_size: usize, + is_nullable: bool, + ) -> Result>> { // create read buffer let mut read_buffer: Vec = vec![A::default_value().into(); batch_size]; @@ -151,14 +161,15 @@ where } let (values_read, levels_read) = self.read_batch( - batch_size, - Some(&mut def_levels), - None, - &mut read_buffer, + batch_size, + Some(&mut def_levels), + None, + &mut read_buffer, )?; let mut builder = PrimitiveBuilder::::new(levels_read); if values_read == levels_read { - let converted_buffer: Vec = read_buffer.into_iter().map(|v| v.into()).collect(); + let converted_buffer: Vec = + read_buffer.into_iter().map(|v| v.into()).collect(); builder.append_slice(&converted_buffer[0..values_read])?; } else { for (v, l) in read_buffer.into_iter().zip(def_levels) { @@ -171,15 +182,12 @@ where } Ok(Arc::new(builder.finish())) } else { - let (values_read, _) = self.read_batch( - batch_size, - None, - None, - &mut read_buffer, - )?; + let (values_read, _) = + self.read_batch(batch_size, None, None, &mut read_buffer)?; let mut builder = PrimitiveBuilder::::new(values_read); - let converted_buffer: Vec = read_buffer.into_iter().map(|v| v.into()).collect(); + let converted_buffer: Vec = + read_buffer.into_iter().map(|v| v.into()).collect(); builder.append_slice(&converted_buffer[0..values_read])?; Ok(Arc::new(builder.finish())) } @@ -262,21 +270,33 @@ impl ParquetFile { let is_nullable = self.schema().field(i).is_nullable(); let array: Arc = match self.column_readers[i] { ColumnReader::BoolColumnReader(ref mut r) => { - match ArrowReader::::read(r, self.batch_size, is_nullable) { + match ArrowReader::::read( + r, + self.batch_size, + is_nullable, + ) { Ok(array) => array, - Err(e) => return Err(e) + Err(e) => return Err(e), } } ColumnReader::Int32ColumnReader(ref mut r) => { - match ArrowReader::::read(r, self.batch_size, is_nullable) { + match ArrowReader::::read( + r, + self.batch_size, + is_nullable, + ) { Ok(array) => array, - Err(e) => return Err(e) + Err(e) => return Err(e), } } ColumnReader::Int64ColumnReader(ref mut r) => { - match ArrowReader::::read(r, self.batch_size, is_nullable) { + match ArrowReader::::read( + r, + self.batch_size, + is_nullable, + ) { Ok(array) => array, - Err(e) => return Err(e) + Err(e) => return Err(e), } } ColumnReader::Int96ColumnReader(ref mut r) => { @@ -340,15 +360,23 @@ impl ParquetFile { } } ColumnReader::FloatColumnReader(ref mut r) => { - match ArrowReader::::read(r, self.batch_size, is_nullable) { + match ArrowReader::::read( + r, + self.batch_size, + is_nullable, + ) { Ok(array) => array, - Err(e) => return Err(e) + Err(e) => return Err(e), } } ColumnReader::DoubleColumnReader(ref mut r) => { - match ArrowReader::::read(r, self.batch_size, is_nullable) { + match ArrowReader::::read( + r, + self.batch_size, + is_nullable, + ) { Ok(array) => array, - Err(e) => return Err(e) + Err(e) => return Err(e), } } ColumnReader::FixedLenByteArrayColumnReader(ref mut r) => { From 5a3368c71d97c10419e0c86ab3443778c0fc1051 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Mar 2019 15:14:13 -0600 Subject: [PATCH 23/37] Remove unnecessary slice, fix null handling --- rust/datafusion/src/datasource/parquet.rs | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index b43af329c295f..9caade08f360c 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -83,9 +83,8 @@ pub struct ParquetFile { fn create_binary_array(b: &Vec, row_count: usize) -> Result> { let mut builder = BinaryBuilder::new(b.len()); - for j in 0..row_count { - let slice = b[j].slice(0, b[j].len()); - builder.append_string(&String::from_utf8(slice.data().to_vec()).unwrap())?; + for i in 0..row_count { + builder.append_string(&String::from_utf8(b[i].data().to_vec()).unwrap())?; } Ok(Arc::new(builder.finish())) } @@ -167,14 +166,16 @@ where &mut read_buffer, )?; let mut builder = PrimitiveBuilder::::new(levels_read); + let converted_buffer: Vec = + read_buffer.into_iter().map(|v| v.into()).collect(); if values_read == levels_read { - let converted_buffer: Vec = - read_buffer.into_iter().map(|v| v.into()).collect(); builder.append_slice(&converted_buffer[0..values_read])?; } else { - for (v, l) in read_buffer.into_iter().zip(def_levels) { - if l == 0 { - builder.append_value(v.into())?; + let mut vi = 0; + for i in 0..def_levels.len() { + if def_levels[i] != 0 { + builder.append_value(converted_buffer[vi].into())?; + vi += 1; } else { builder.append_null()?; } From 80cf3033a58596f6c140ee1d20d0f2dd45ffb954 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Mar 2019 15:44:46 -0600 Subject: [PATCH 24/37] add date support --- rust/parquet/src/reader/schema.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/rust/parquet/src/reader/schema.rs b/rust/parquet/src/reader/schema.rs index 239937c89b742..74b4c2c95f217 100644 --- a/rust/parquet/src/reader/schema.rs +++ b/rust/parquet/src/reader/schema.rs @@ -28,7 +28,7 @@ use crate::basic::{LogicalType, Repetition, Type as PhysicalType}; use crate::errors::{ParquetError::ArrowError, Result}; use crate::schema::types::{SchemaDescPtr, Type, TypePtr}; -use arrow::datatypes::{DataType, Field, Schema}; +use arrow::datatypes::{DataType, DateUnit, Field, Schema}; /// Convert parquet schema to arrow schema. pub fn parquet_to_arrow_schema(parquet_schema: SchemaDescPtr) -> Result { @@ -197,6 +197,7 @@ impl ParquetTypeConverter { LogicalType::INT_8 => Ok(DataType::Int8), LogicalType::INT_16 => Ok(DataType::Int16), LogicalType::INT_32 => Ok(DataType::Int32), + LogicalType::DATE => Ok(DataType::Date32(DateUnit::Millisecond)), other => Err(ArrowError(format!( "Unable to convert parquet logical type {}", other @@ -209,6 +210,7 @@ impl ParquetTypeConverter { LogicalType::NONE => Ok(DataType::Int64), LogicalType::INT_64 => Ok(DataType::Int64), LogicalType::UINT_64 => Ok(DataType::UInt64), + LogicalType::DATE => Ok(DataType::Date64(DateUnit::Millisecond)), other => Err(ArrowError(format!( "Unable to convert parquet logical type {}", other From 1503855fbbaf155217301d3331d26770cf3dd7d6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Mar 2019 17:12:32 -0600 Subject: [PATCH 25/37] handle nulls for binary data --- rust/datafusion/src/datasource/parquet.rs | 89 ++++++++--------------- 1 file changed, 32 insertions(+), 57 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 9caade08f360c..a61cb0d11449d 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -113,7 +113,17 @@ macro_rules! read_binary_column { if values_read == levels_read { create_binary_array(&read_buffer, values_read)? } else { - return Err(ExecutionError::NotImplemented("Parquet datasource does not support null values".to_string())) + let mut builder = BinaryBuilder::new(levels_read); + let mut value_index = 0; + for i in 0..levels_read { + if def_levels[i] > 0 { + builder.append_string(&String::from_utf8(read_buffer[value_index].data().to_vec()).unwrap())?; + value_index += 1; + } else { + builder.append_null()?; + } + } + Arc::new(builder.finish()) } } else { let (values_read, _) = $R.read_batch( @@ -171,11 +181,11 @@ where if values_read == levels_read { builder.append_slice(&converted_buffer[0..values_read])?; } else { - let mut vi = 0; + let mut value_index = 0; for i in 0..def_levels.len() { if def_levels[i] != 0 { - builder.append_value(converted_buffer[vi].into())?; - vi += 1; + builder.append_value(converted_buffer[value_index].into())?; + value_index += 1; } else { builder.append_null()?; } @@ -203,6 +213,24 @@ impl ParquetFile { let schema = parquet_to_arrow_schema(metadata.file_metadata().schema_descr_ptr())?; + // even if we aren't referencing structs or lists in our projection, column reader + // indexes will be off until we have support for nested schemas + for i in 0..schema.fields().len() { + match schema.field(i).data_type() { + DataType::List(_) => { + return Err(ExecutionError::NotImplemented( + "Parquet datasource does not support LIST".to_string(), + )); + } + DataType::Struct(_) => { + return Err(ExecutionError::NotImplemented( + "Parquet datasource does not support STRUCT".to_string(), + )); + } + _ => {} + } + } + let projection = match projection { Some(p) => p, None => { @@ -234,20 +262,6 @@ impl ParquetFile { self.column_readers = Vec::with_capacity(self.projection.len()); for i in 0..self.projection.len() { - match self.schema().field(i).data_type() { - DataType::List(_) => { - return Err(ExecutionError::NotImplemented( - "Parquet datasource does not support LIST".to_string(), - )); - } - DataType::Struct(_) => { - return Err(ExecutionError::NotImplemented( - "Parquet datasource does not support STRUCT".to_string(), - )); - } - _ => {} - } - self.column_readers .push(reader.get_column_reader(self.projection[i])?); } @@ -615,45 +629,6 @@ mod tests { ); } - #[test] - fn read_int64_nullable_impala_parquet() { - let table = load_table("nullable.impala.parquet"); - let projection = Some(vec![0]); - - let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan[0].lock().unwrap(); - let batch = it.next().unwrap().unwrap(); - - assert_eq!(1, batch.num_columns()); - assert_eq!(7, 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!("[1, 2, 3, 4, 5, 6, 7]", format!("{:?}", values)); - } - - #[test] - fn read_array_nullable_impala_parquet() { - let table = load_table("nullable.impala.parquet"); - let projection = Some(vec![1]); - let scan = table.scan(&projection, 1024).unwrap(); - let mut it = scan[0].lock().unwrap(); - let batch = it.next(); - - assert_eq!( - "NotImplemented(\"Parquet datasource does not support LIST\")", - format!("{:?}", batch.err().unwrap()) - ); - } - fn load_table(name: &str) -> Box

{ let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); From 639e13e2b0ed28122934afe1f9935aa2c8a42559 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Mar 2019 17:24:20 -0600 Subject: [PATCH 26/37] null handling for int96 --- rust/datafusion/src/datasource/parquet.rs | 101 +++++++++++----------- 1 file changed, 50 insertions(+), 51 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index a61cb0d11449d..a3ea303881eaf 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -46,7 +46,7 @@ impl ParquetTable { pub fn try_new(filename: &str) -> Result { let file = File::open(filename)?; let parquet_file = ParquetFile::open(file, None)?; - let schema = parquet_file.schema.clone(); + let schema = parquet_file.projection_schema.clone(); Ok(Self { filename: filename.to_string(), schema, @@ -72,11 +72,12 @@ impl Table for ParquetTable { pub struct ParquetFile { reader: SerializedFileReader, - row_group_index: usize, - /// The schema of the underlying file - schema: Arc, + /// Projection expressed as column indices into underlying parquet reader projection: Vec, + /// The schema of the projection + projection_schema: Arc, batch_size: usize, + row_group_index: usize, current_row_group: Option>, column_readers: Vec, } @@ -97,7 +98,7 @@ macro_rules! read_binary_column { for _ in 0..$SELF.batch_size { read_buffer.push(ByteArray::default()); } - if $SELF.schema.field($INDEX).is_nullable() { + if $SELF.projection_schema.field($INDEX).is_nullable() { let mut def_levels: Vec = Vec::with_capacity($SELF.batch_size); for _ in 0..$SELF.batch_size { @@ -247,7 +248,7 @@ impl ParquetFile { Ok(ParquetFile { reader: reader, row_group_index: 0, - schema: projected_schema, + projection_schema: projected_schema, projection, batch_size: 64 * 1024, current_row_group: None, @@ -285,34 +286,25 @@ impl ParquetFile { let is_nullable = self.schema().field(i).is_nullable(); let array: Arc = match self.column_readers[i] { ColumnReader::BoolColumnReader(ref mut r) => { - match ArrowReader::::read( + ArrowReader::::read( r, self.batch_size, is_nullable, - ) { - Ok(array) => array, - Err(e) => return Err(e), - } + )? } ColumnReader::Int32ColumnReader(ref mut r) => { - match ArrowReader::::read( + ArrowReader::::read( r, self.batch_size, is_nullable, - ) { - Ok(array) => array, - Err(e) => return Err(e), - } + )? } ColumnReader::Int64ColumnReader(ref mut r) => { - match ArrowReader::::read( + ArrowReader::::read( r, self.batch_size, is_nullable, - ) { - Ok(array) => array, - Err(e) => return Err(e), - } + )? } ColumnReader::Int96ColumnReader(ref mut r) => { let mut read_buffer: Vec = @@ -322,7 +314,7 @@ impl ParquetFile { read_buffer.push(Int96::new()); } - if self.schema.field(i).is_nullable() { + if self.projection_schema.field(i).is_nullable() { let mut def_levels: Vec = Vec::with_capacity(self.batch_size); for _ in 0..self.batch_size { @@ -337,21 +329,28 @@ impl ParquetFile { if values_read == levels_read { let mut builder = Int64Builder::new(values_read); - for i in 0..values_read { - let v = read_buffer[i].data(); - let value: u128 = (v[0] as u128) << 64 - | (v[1] as u128) << 32 - | (v[2] as u128); - let ms: i64 = (value / 1000000) as i64; - builder.append_value(ms)?; + builder.append_value(convert_int96_timestamp( + read_buffer[i].data(), + ))?; } Arc::new(builder.finish()) } else { - return Err(ExecutionError::NotImplemented( - "Parquet datasource does not support null values" - .to_string(), - )); + let mut builder = Int64Builder::new(levels_read); + let mut value_index = 0; + for i in 0..levels_read { + if def_levels[i] > 0 { + builder.append_value( + convert_int96_timestamp( + read_buffer[value_index].data(), + ), + )?; + value_index += 1; + } else { + builder.append_null()?; + } + } + Arc::new(builder.finish()) } } else { let (values_read, _) = r.read_batch( @@ -364,35 +363,26 @@ impl ParquetFile { let mut builder = Int64Builder::new(values_read); for i in 0..values_read { - let v = read_buffer[i].data(); - let value: u128 = (v[0] as u128) << 64 - | (v[1] as u128) << 32 - | (v[2] as u128); - let ms: i64 = (value / 1000000) as i64; - builder.append_value(ms)?; + builder.append_value(convert_int96_timestamp( + read_buffer[i].data(), + ))?; } Arc::new(builder.finish()) } } ColumnReader::FloatColumnReader(ref mut r) => { - match ArrowReader::::read( + ArrowReader::::read( r, self.batch_size, is_nullable, - ) { - Ok(array) => array, - Err(e) => return Err(e), - } + )? } ColumnReader::DoubleColumnReader(ref mut r) => { - match ArrowReader::::read( + ArrowReader::::read( r, self.batch_size, is_nullable, - ) { - Ok(array) => array, - Err(e) => return Err(e), - } + )? } ColumnReader::FixedLenByteArrayColumnReader(ref mut r) => { read_binary_column!(self, r, i) @@ -408,7 +398,10 @@ impl ParquetFile { if batch.len() == 0 || batch[0].data().len() == 0 { Ok(None) } else { - Ok(Some(RecordBatch::try_new(self.schema.clone(), batch)?)) + Ok(Some(RecordBatch::try_new( + self.projection_schema.clone(), + batch, + )?)) } } _ => Ok(None), @@ -416,9 +409,15 @@ impl ParquetFile { } } +/// convert a parquet timestamp in nanoseconds to a timestamp with milliseconds +fn convert_int96_timestamp(v: &[u32]) -> i64 { + let value: u128 = (v[0] as u128) << 64 | (v[1] as u128) << 32 | (v[2] as u128); + (value / 1000000) as i64 +} + impl RecordBatchIterator for ParquetFile { fn schema(&self) -> &Arc { - &self.schema + &self.projection_schema } fn next(&mut self) -> Result> { From 9d3047a6e62840ff88c989f1447826bf2bde5527 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Mar 2019 17:34:16 -0600 Subject: [PATCH 27/37] code cleanup --- rust/datafusion/src/datasource/parquet.rs | 147 ++++++---------------- 1 file changed, 38 insertions(+), 109 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index a3ea303881eaf..2656b54112612 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -32,7 +32,6 @@ use parquet::file::reader::*; use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; -use arrow::array::BinaryArray; use arrow::builder::{BinaryBuilder, Int64Builder}; use parquet::data_type::Int96; use parquet::reader::schema::parquet_to_arrow_schema; @@ -82,60 +81,31 @@ pub struct ParquetFile { column_readers: Vec, } -fn create_binary_array(b: &Vec, row_count: usize) -> Result> { - let mut builder = BinaryBuilder::new(b.len()); - for i in 0..row_count { - builder.append_string(&String::from_utf8(b[i].data().to_vec()).unwrap())?; - } - Ok(Arc::new(builder.finish())) -} - macro_rules! read_binary_column { ($SELF:ident, $R:ident, $INDEX:expr) => {{ - //TODO: should be able to get num_rows in row group instead of defaulting to batch size let mut read_buffer: Vec = - Vec::with_capacity($SELF.batch_size); - for _ in 0..$SELF.batch_size { - read_buffer.push(ByteArray::default()); - } - if $SELF.projection_schema.field($INDEX).is_nullable() { - - let mut def_levels: Vec = Vec::with_capacity($SELF.batch_size); - for _ in 0..$SELF.batch_size { - def_levels.push(0); - } - - let (values_read, levels_read) = $R.read_batch( - $SELF.batch_size, - Some(&mut def_levels), - None, - &mut read_buffer, - )?; - if values_read == levels_read { - create_binary_array(&read_buffer, values_read)? + vec![ByteArray::default(); $SELF.batch_size]; + let mut def_levels: Vec = vec![0; $SELF.batch_size]; + let (_, levels_read) = $R.read_batch( + $SELF.batch_size, + Some(&mut def_levels), + None, + &mut read_buffer, + )?; + let mut builder = BinaryBuilder::new(levels_read); + let mut value_index = 0; + for i in 0..levels_read { + if def_levels[i] > 0 { + builder.append_string( + &String::from_utf8(read_buffer[value_index].data().to_vec()).unwrap(), + )?; + value_index += 1; } else { - let mut builder = BinaryBuilder::new(levels_read); - let mut value_index = 0; - for i in 0..levels_read { - if def_levels[i] > 0 { - builder.append_string(&String::from_utf8(read_buffer[value_index].data().to_vec()).unwrap())?; - value_index += 1; - } else { - builder.append_null()?; - } - } - Arc::new(builder.finish()) + builder.append_null()?; } - } else { - let (values_read, _) = $R.read_batch( - $SELF.batch_size, - None, - None, - &mut read_buffer, - )?; - create_binary_array(&read_buffer, values_read)? } - }} + Arc::new(builder.finish()) + }}; } trait ArrowReader @@ -165,10 +135,7 @@ where let mut read_buffer: Vec = vec![A::default_value().into(); batch_size]; if is_nullable { - let mut def_levels: Vec = Vec::with_capacity(batch_size); - for _ in 0..batch_size { - def_levels.push(0); - } + let mut def_levels: Vec = vec![0; batch_size]; let (values_read, levels_read) = self.read_batch( batch_size, @@ -308,67 +275,29 @@ impl ParquetFile { } ColumnReader::Int96ColumnReader(ref mut r) => { let mut read_buffer: Vec = - Vec::with_capacity(self.batch_size); + vec![Int96::new(); self.batch_size]; - for _ in 0..self.batch_size { - read_buffer.push(Int96::new()); - } - - if self.projection_schema.field(i).is_nullable() { - let mut def_levels: Vec = - Vec::with_capacity(self.batch_size); - for _ in 0..self.batch_size { - def_levels.push(0); - } - let (values_read, levels_read) = r.read_batch( - self.batch_size, - Some(&mut def_levels), - None, - &mut read_buffer, - )?; - - if values_read == levels_read { - let mut builder = Int64Builder::new(values_read); - for i in 0..values_read { - builder.append_value(convert_int96_timestamp( - read_buffer[i].data(), - ))?; - } - Arc::new(builder.finish()) - } else { - let mut builder = Int64Builder::new(levels_read); - let mut value_index = 0; - for i in 0..levels_read { - if def_levels[i] > 0 { - builder.append_value( - convert_int96_timestamp( - read_buffer[value_index].data(), - ), - )?; - value_index += 1; - } else { - builder.append_null()?; - } - } - Arc::new(builder.finish()) - } - } else { - let (values_read, _) = r.read_batch( - self.batch_size, - None, - None, - &mut read_buffer, - )?; - - let mut builder = Int64Builder::new(values_read); - - for i in 0..values_read { + let mut def_levels: Vec = vec![0; self.batch_size]; + let (_, levels_read) = r.read_batch( + self.batch_size, + Some(&mut def_levels), + None, + &mut read_buffer, + )?; + + let mut builder = Int64Builder::new(levels_read); + let mut value_index = 0; + for i in 0..levels_read { + if def_levels[i] > 0 { builder.append_value(convert_int96_timestamp( - read_buffer[i].data(), + read_buffer[value_index].data(), ))?; + value_index += 1; + } else { + builder.append_null()?; } - Arc::new(builder.finish()) } + Arc::new(builder.finish()) } ColumnReader::FloatColumnReader(ref mut r) => { ArrowReader::::read( From 2aeea24cca94235ec397a07ba6e47dc8dd00fc49 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Mar 2019 17:38:38 -0600 Subject: [PATCH 28/37] remove println from tests --- rust/datafusion/src/datasource/parquet.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 2656b54112612..7b251689bd8c0 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -561,10 +561,6 @@ mod tests { let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); let table = ParquetTable::try_new(&filename).unwrap(); - println!("Loading file {} with schema:", name); - for field in table.schema().fields() { - println!("\t{:?}", field); - } Box::new(table) } } From 02b2ed331af9dd918639caa69aa620268e383fee Mon Sep 17 00:00:00 2001 From: Neville Dipale Date: Thu, 14 Mar 2019 13:03:40 +0200 Subject: [PATCH 29/37] fix int96 conversion to read timestamps correctly --- rust/datafusion/src/datasource/parquet.rs | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 7b251689bd8c0..91b14adb3c59d 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -340,8 +340,14 @@ impl ParquetFile { /// convert a parquet timestamp in nanoseconds to a timestamp with milliseconds fn convert_int96_timestamp(v: &[u32]) -> i64 { - let value: u128 = (v[0] as u128) << 64 | (v[1] as u128) << 32 | (v[2] as u128); - (value / 1000000) as i64 + const JULIAN_DAY_OF_EPOCH: i64 = 2_440_588; + const SECONDS_PER_DAY: i64 = 86_400; + const MILLIS_PER_SECOND: i64 = 1_000; + + let day = v[2] as i64; + let nanoseconds = ((v[1] as i64) << 32) + v[0] as i64; + let seconds = (day - JULIAN_DAY_OF_EPOCH) * SECONDS_PER_DAY; + seconds * MILLIS_PER_SECOND + nanoseconds / 1_000_000 } impl RecordBatchIterator for ParquetFile { @@ -469,7 +475,7 @@ mod tests { values.push(array.value(i)); } - assert_eq!("[2, 7842670136425819125, 2, 7842670136425819125, 2, 7842670136425819125, 2, 7842670136425819125]", format!("{:?}", values)); + assert_eq!("[1235865600000, 1235865660000, 1238544000000, 1238544060000, 1233446400000, 1233446460000, 1230768000000, 1230768060000]", format!("{:?}", values)); } #[test] From 1ec815bc14e269798aa26a95c554cca5b6fe2b70 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Mar 2019 07:15:37 -0600 Subject: [PATCH 30/37] Clean up imports --- rust/datafusion/src/datasource/parquet.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 91b14adb3c59d..e74645f67d20c 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -22,19 +22,17 @@ use std::string::String; use std::sync::{Arc, Mutex}; use arrow::array::{Array, PrimitiveArray}; -use arrow::builder::PrimitiveBuilder; +use arrow::builder::{BinaryBuilder, Int64Builder, PrimitiveBuilder}; use arrow::datatypes::*; use arrow::record_batch::RecordBatch; use parquet::column::reader::*; -use parquet::data_type::ByteArray; +use parquet::data_type::{ByteArray, Int96}; use parquet::file::reader::*; +use parquet::reader::schema::parquet_to_arrow_schema; use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; -use arrow::builder::{BinaryBuilder, Int64Builder}; -use parquet::data_type::Int96; -use parquet::reader::schema::parquet_to_arrow_schema; pub struct ParquetTable { filename: String, From 9b1308f7babbf45f8ca38f1a890fbb600e783978 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Mar 2019 07:25:02 -0600 Subject: [PATCH 31/37] clean up handling of INT96 and DATE/TIME/TIMESTAMP types in schema converter --- rust/parquet/src/reader/schema.rs | 38 +++++++++++++++++++++++++++---- 1 file changed, 34 insertions(+), 4 deletions(-) diff --git a/rust/parquet/src/reader/schema.rs b/rust/parquet/src/reader/schema.rs index 74b4c2c95f217..2e02c6c8476c7 100644 --- a/rust/parquet/src/reader/schema.rs +++ b/rust/parquet/src/reader/schema.rs @@ -28,6 +28,7 @@ use crate::basic::{LogicalType, Repetition, Type as PhysicalType}; use crate::errors::{ParquetError::ArrowError, Result}; use crate::schema::types::{SchemaDescPtr, Type, TypePtr}; +use arrow::datatypes::TimeUnit; use arrow::datatypes::{DataType, DateUnit, Field, Schema}; /// Convert parquet schema to arrow schema. @@ -177,12 +178,12 @@ impl ParquetTypeConverter { PhysicalType::BOOLEAN => Ok(DataType::Boolean), PhysicalType::INT32 => self.to_int32(), PhysicalType::INT64 => self.to_int64(), - PhysicalType::INT96 => self.to_int64(), + PhysicalType::INT96 => self.to_int96(), PhysicalType::FLOAT => Ok(DataType::Float32), PhysicalType::DOUBLE => Ok(DataType::Float64), PhysicalType::BYTE_ARRAY => self.to_byte_array(), other => Err(ArrowError(format!( - "Unable to convert parquet type {}", + "Unable to convert parquet physical type {}", other ))), } @@ -198,8 +199,10 @@ impl ParquetTypeConverter { LogicalType::INT_16 => Ok(DataType::Int16), LogicalType::INT_32 => Ok(DataType::Int32), LogicalType::DATE => Ok(DataType::Date32(DateUnit::Millisecond)), + LogicalType::TIME_MICROS => Ok(DataType::Time32(TimeUnit::Microsecond)), + LogicalType::TIME_MILLIS => Ok(DataType::Time32(TimeUnit::Millisecond)), other => Err(ArrowError(format!( - "Unable to convert parquet logical type {}", + "Unable to convert parquet INT32 logical type {}", other ))), } @@ -211,8 +214,35 @@ impl ParquetTypeConverter { LogicalType::INT_64 => Ok(DataType::Int64), LogicalType::UINT_64 => Ok(DataType::UInt64), LogicalType::DATE => Ok(DataType::Date64(DateUnit::Millisecond)), + LogicalType::TIME_MICROS => Ok(DataType::Time64(TimeUnit::Microsecond)), + LogicalType::TIME_MILLIS => Ok(DataType::Time64(TimeUnit::Millisecond)), + LogicalType::TIMESTAMP_MICROS => { + Ok(DataType::Timestamp(TimeUnit::Microsecond)) + } + LogicalType::TIMESTAMP_MILLIS => { + Ok(DataType::Timestamp(TimeUnit::Millisecond)) + } other => Err(ArrowError(format!( - "Unable to convert parquet logical type {}", + "Unable to convert parquet INT64 logical type {}", + other + ))), + } + } + + fn to_int96(&self) -> Result { + match self.schema.get_basic_info().logical_type() { + LogicalType::NONE => Ok(DataType::Int64), + LogicalType::DATE => Ok(DataType::Date64(DateUnit::Millisecond)), + LogicalType::TIME_MICROS => Ok(DataType::Time64(TimeUnit::Microsecond)), + LogicalType::TIME_MILLIS => Ok(DataType::Time64(TimeUnit::Millisecond)), + LogicalType::TIMESTAMP_MICROS => { + Ok(DataType::Timestamp(TimeUnit::Microsecond)) + } + LogicalType::TIMESTAMP_MILLIS => { + Ok(DataType::Timestamp(TimeUnit::Millisecond)) + } + other => Err(ArrowError(format!( + "Unable to convert parquet INT96 logical type {}", other ))), } From 25d34ac9f7d735dd8d1fe051cd552a7ac17757f4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Mar 2019 07:37:07 -0600 Subject: [PATCH 32/37] Make INT32/64/96 handling consistent with C++ implementation --- rust/datafusion/src/datasource/parquet.rs | 17 +++++----- rust/parquet/src/reader/schema.rs | 38 +++++------------------ 2 files changed, 17 insertions(+), 38 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index e74645f67d20c..a5804ac91964f 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -22,7 +22,7 @@ use std::string::String; use std::sync::{Arc, Mutex}; use arrow::array::{Array, PrimitiveArray}; -use arrow::builder::{BinaryBuilder, Int64Builder, PrimitiveBuilder}; +use arrow::builder::{BinaryBuilder, PrimitiveBuilder}; use arrow::datatypes::*; use arrow::record_batch::RecordBatch; @@ -33,6 +33,7 @@ use parquet::reader::schema::parquet_to_arrow_schema; use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; +use arrow::builder::TimestampNanosecondBuilder; pub struct ParquetTable { filename: String, @@ -283,7 +284,8 @@ impl ParquetFile { &mut read_buffer, )?; - let mut builder = Int64Builder::new(levels_read); + let mut builder = + TimestampNanosecondBuilder::new(levels_read); let mut value_index = 0; for i in 0..levels_read { if def_levels[i] > 0 { @@ -378,11 +380,10 @@ impl RecordBatchIterator for ParquetFile { #[cfg(test)] mod tests { use super::*; - use arrow::array::BooleanArray; - use arrow::array::Float32Array; - use arrow::array::Float64Array; - use arrow::array::Int64Array; - use arrow::array::{BinaryArray, Int32Array}; + use arrow::array::{ + BinaryArray, BooleanArray, Float32Array, Float64Array, Int32Array, + TimestampNanosecondArray, + }; use std::env; #[test] @@ -466,7 +467,7 @@ mod tests { let array = batch .column(0) .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); let mut values: Vec = vec![]; for i in 0..batch.num_rows() { diff --git a/rust/parquet/src/reader/schema.rs b/rust/parquet/src/reader/schema.rs index 2e02c6c8476c7..5af07be7460f6 100644 --- a/rust/parquet/src/reader/schema.rs +++ b/rust/parquet/src/reader/schema.rs @@ -176,12 +176,12 @@ impl ParquetTypeConverter { fn to_primitive_type_inner(&self) -> Result { match self.schema.get_physical_type() { PhysicalType::BOOLEAN => Ok(DataType::Boolean), - PhysicalType::INT32 => self.to_int32(), - PhysicalType::INT64 => self.to_int64(), - PhysicalType::INT96 => self.to_int96(), + PhysicalType::INT32 => self.from_int32(), + PhysicalType::INT64 => self.from_int64(), + PhysicalType::INT96 => Ok(DataType::Timestamp(TimeUnit::Nanosecond)), PhysicalType::FLOAT => Ok(DataType::Float32), PhysicalType::DOUBLE => Ok(DataType::Float64), - PhysicalType::BYTE_ARRAY => self.to_byte_array(), + PhysicalType::BYTE_ARRAY => self.from_byte_array(), other => Err(ArrowError(format!( "Unable to convert parquet physical type {}", other @@ -189,7 +189,7 @@ impl ParquetTypeConverter { } } - fn to_int32(&self) -> Result { + fn from_int32(&self) -> Result { match self.schema.get_basic_info().logical_type() { LogicalType::NONE => Ok(DataType::Int32), LogicalType::UINT_8 => Ok(DataType::UInt8), @@ -199,7 +199,6 @@ impl ParquetTypeConverter { LogicalType::INT_16 => Ok(DataType::Int16), LogicalType::INT_32 => Ok(DataType::Int32), LogicalType::DATE => Ok(DataType::Date32(DateUnit::Millisecond)), - LogicalType::TIME_MICROS => Ok(DataType::Time32(TimeUnit::Microsecond)), LogicalType::TIME_MILLIS => Ok(DataType::Time32(TimeUnit::Millisecond)), other => Err(ArrowError(format!( "Unable to convert parquet INT32 logical type {}", @@ -208,14 +207,12 @@ impl ParquetTypeConverter { } } - fn to_int64(&self) -> Result { + fn from_int64(&self) -> Result { match self.schema.get_basic_info().logical_type() { LogicalType::NONE => Ok(DataType::Int64), LogicalType::INT_64 => Ok(DataType::Int64), LogicalType::UINT_64 => Ok(DataType::UInt64), - LogicalType::DATE => Ok(DataType::Date64(DateUnit::Millisecond)), LogicalType::TIME_MICROS => Ok(DataType::Time64(TimeUnit::Microsecond)), - LogicalType::TIME_MILLIS => Ok(DataType::Time64(TimeUnit::Millisecond)), LogicalType::TIMESTAMP_MICROS => { Ok(DataType::Timestamp(TimeUnit::Microsecond)) } @@ -229,31 +226,12 @@ impl ParquetTypeConverter { } } - fn to_int96(&self) -> Result { - match self.schema.get_basic_info().logical_type() { - LogicalType::NONE => Ok(DataType::Int64), - LogicalType::DATE => Ok(DataType::Date64(DateUnit::Millisecond)), - LogicalType::TIME_MICROS => Ok(DataType::Time64(TimeUnit::Microsecond)), - LogicalType::TIME_MILLIS => Ok(DataType::Time64(TimeUnit::Millisecond)), - LogicalType::TIMESTAMP_MICROS => { - Ok(DataType::Timestamp(TimeUnit::Microsecond)) - } - LogicalType::TIMESTAMP_MILLIS => { - Ok(DataType::Timestamp(TimeUnit::Millisecond)) - } - other => Err(ArrowError(format!( - "Unable to convert parquet INT96 logical type {}", - other - ))), - } - } - - fn to_byte_array(&self) -> Result { + fn from_byte_array(&self) -> Result { match self.schema.get_basic_info().logical_type() { LogicalType::NONE => Ok(DataType::Utf8), LogicalType::UTF8 => Ok(DataType::Utf8), other => Err(ArrowError(format!( - "Unable to convert parquet logical type {}", + "Unable to convert parquet BYTE_ARRAY logical type {}", other ))), } From 73aa934a192f9f0fdbabeb909550c8759478eae6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Mar 2019 07:40:24 -0600 Subject: [PATCH 33/37] Remove println from test --- rust/datafusion/tests/sql.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/rust/datafusion/tests/sql.rs b/rust/datafusion/tests/sql.rs index a6c368794cc7b..9c24a504bd394 100644 --- a/rust/datafusion/tests/sql.rs +++ b/rust/datafusion/tests/sql.rs @@ -183,14 +183,12 @@ fn load_parquet_table(name: &str) -> Rc
{ let testdata = env::var("PARQUET_TEST_DATA").unwrap(); let filename = format!("{}/{}", testdata, name); let table = ParquetTable::try_new(&filename).unwrap(); - println!("{:?}", table.schema()); Rc::new(table) } /// Execute query and return result set as tab delimited string fn execute(ctx: &mut ExecutionContext, sql: &str) -> String { let plan = ctx.create_logical_plan(&sql).unwrap(); - println!("Plan: {:?}", plan); let results = ctx.execute(&plan, DEFAULT_BATCH_SIZE).unwrap(); result_str(&results) } From 204db837193ba35f0ca61deebf3b1ee2ba542ab9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Mar 2019 16:49:35 -0600 Subject: [PATCH 34/37] fix timestamp nano issue --- rust/datafusion/src/datasource/parquet.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index a5804ac91964f..74cc813ba06d5 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -22,7 +22,7 @@ use std::string::String; use std::sync::{Arc, Mutex}; use arrow::array::{Array, PrimitiveArray}; -use arrow::builder::{BinaryBuilder, PrimitiveBuilder}; +use arrow::builder::{BinaryBuilder, PrimitiveBuilder, TimestampNanosecondBuilder}; use arrow::datatypes::*; use arrow::record_batch::RecordBatch; @@ -33,7 +33,6 @@ use parquet::reader::schema::parquet_to_arrow_schema; use crate::datasource::{RecordBatchIterator, ScanResult, Table}; use crate::execution::error::{ExecutionError, Result}; -use arrow::builder::TimestampNanosecondBuilder; pub struct ParquetTable { filename: String, @@ -338,7 +337,7 @@ impl ParquetFile { } } -/// convert a parquet timestamp in nanoseconds to a timestamp with milliseconds +/// convert a Parquet INT96 to an Arrow timestamp in nanoseconds fn convert_int96_timestamp(v: &[u32]) -> i64 { const JULIAN_DAY_OF_EPOCH: i64 = 2_440_588; const SECONDS_PER_DAY: i64 = 86_400; @@ -347,7 +346,7 @@ fn convert_int96_timestamp(v: &[u32]) -> i64 { let day = v[2] as i64; let nanoseconds = ((v[1] as i64) << 32) + v[0] as i64; let seconds = (day - JULIAN_DAY_OF_EPOCH) * SECONDS_PER_DAY; - seconds * MILLIS_PER_SECOND + nanoseconds / 1_000_000 + seconds * MILLIS_PER_SECOND * 1_000_000 + nanoseconds } impl RecordBatchIterator for ParquetFile { @@ -474,7 +473,7 @@ mod tests { values.push(array.value(i)); } - assert_eq!("[1235865600000, 1235865660000, 1238544000000, 1238544060000, 1233446400000, 1233446460000, 1230768000000, 1230768060000]", format!("{:?}", values)); + assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); } #[test] From 8d2df06ca909ed3a0c99cc19685d708390ce31a0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Mar 2019 22:18:24 -0600 Subject: [PATCH 35/37] move schema projection function from arrow into datafusion --- rust/arrow/src/datatypes.rs | 17 ----------------- rust/datafusion/src/datasource/parquet.rs | 18 +++++++++++++++++- 2 files changed, 17 insertions(+), 18 deletions(-) diff --git a/rust/arrow/src/datatypes.rs b/rust/arrow/src/datatypes.rs index 34860350e7357..e0b6d706eab15 100644 --- a/rust/arrow/src/datatypes.rs +++ b/rust/arrow/src/datatypes.rs @@ -26,7 +26,6 @@ use std::mem::size_of; use std::ops::{Add, Div, Mul, Sub}; use std::slice::from_raw_parts; use std::str::FromStr; -use std::sync::Arc; use packed_simd::*; use serde_derive::{Deserialize, Serialize}; @@ -752,22 +751,6 @@ impl Schema { "fields": self.fields.iter().map(|field| field.to_json()).collect::>(), }) } - - /// Create a new schema by applying a projection to this schema's fields - pub fn projection(&self, projection: &[usize]) -> Result> { - let mut fields: Vec = Vec::with_capacity(projection.len()); - for i in projection { - if *i < self.fields().len() { - fields.push(self.field(*i).clone()); - } else { - return Err(ArrowError::InvalidArgumentError(format!( - "Invalid column index {} in projection", - i - ))); - } - } - Ok(Arc::new(Schema::new(fields))) - } } impl fmt::Display for Schema { diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 74cc813ba06d5..efb1857be9b8f 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -208,7 +208,7 @@ impl ParquetFile { } }; - let projected_schema = schema.projection(&projection)?; + let projected_schema = schema_projection(&schema, &projection)?; Ok(ParquetFile { reader: reader, @@ -337,6 +337,22 @@ impl ParquetFile { } } +/// Create a new schema by applying a projection to this schema's fields +fn schema_projection(schema: &Schema, projection: &[usize]) -> Result> { + let mut fields: Vec = Vec::with_capacity(projection.len()); + for i in projection { + if *i < schema.fields().len() { + fields.push(schema.field(*i).clone()); + } else { + return Err(ExecutionError::InvalidColumn(format!( + "Invalid column index {} in projection", + i + ))); + } + } + Ok(Arc::new(Schema::new(fields))) +} + /// convert a Parquet INT96 to an Arrow timestamp in nanoseconds fn convert_int96_timestamp(v: &[u32]) -> i64 { const JULIAN_DAY_OF_EPOCH: i64 = 2_440_588; From 549c8298c5d9b4b38602e9b030bf0275ad7309fd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 15 Mar 2019 11:16:45 -0600 Subject: [PATCH 36/37] Remove hard-coded batch size, fix nits --- rust/datafusion/src/datasource/parquet.rs | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index efb1857be9b8f..4cf2b5f87d96b 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -42,7 +42,7 @@ pub struct ParquetTable { impl ParquetTable { pub fn try_new(filename: &str) -> Result { let file = File::open(filename)?; - let parquet_file = ParquetFile::open(file, None)?; + let parquet_file = ParquetFile::open(file, None, 0)?; let schema = parquet_file.projection_schema.clone(); Ok(Self { filename: filename.to_string(), @@ -59,10 +59,10 @@ impl Table for ParquetTable { fn scan( &self, projection: &Option>, - _batch_size: usize, + batch_size: usize, ) -> Result> { let file = File::open(self.filename.clone())?; - let parquet_file = ParquetFile::open(file, projection.clone())?; + let parquet_file = ParquetFile::open(file, projection.clone(), batch_size)?; Ok(vec![Arc::new(Mutex::new(parquet_file))]) } } @@ -172,7 +172,11 @@ where } impl ParquetFile { - pub fn open(file: File, projection: Option>) -> Result { + pub fn open( + file: File, + projection: Option>, + batch_size: usize, + ) -> Result { let reader = SerializedFileReader::new(file)?; let metadata = reader.metadata(); @@ -215,7 +219,7 @@ impl ParquetFile { row_group_index: 0, projection_schema: projected_schema, projection, - batch_size: 64 * 1024, + batch_size, current_row_group: None, column_readers: vec![], }) @@ -225,6 +229,7 @@ impl ParquetFile { if self.row_group_index < self.reader.num_row_groups() { let reader = self.reader.get_row_group(self.row_group_index)?; + self.column_readers.clear(); self.column_readers = Vec::with_capacity(self.projection.len()); for i in 0..self.projection.len() { From 3158529f465bab24589b7fbefe37f6d5d9dac710 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 15 Mar 2019 11:20:36 -0600 Subject: [PATCH 37/37] add test for reading small batches --- rust/datafusion/src/datasource/parquet.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/rust/datafusion/src/datasource/parquet.rs b/rust/datafusion/src/datasource/parquet.rs index 4cf2b5f87d96b..3fb4a3c07f460 100644 --- a/rust/datafusion/src/datasource/parquet.rs +++ b/rust/datafusion/src/datasource/parquet.rs @@ -406,6 +406,25 @@ mod tests { }; use std::env; + #[test] + fn read_small_batches() { + let table = load_table("alltypes_plain.parquet"); + + let projection = None; + let scan = table.scan(&projection, 2).unwrap(); + let mut it = scan[0].lock().unwrap(); + + let mut count = 0; + while let Some(batch) = it.next().unwrap() { + assert_eq!(11, batch.num_columns()); + assert_eq!(2, batch.num_rows()); + count += 1; + } + + // we should have seen 4 batches of 2 rows + assert_eq!(4, count); + } + #[test] fn read_alltypes_plain_parquet() { let table = load_table("alltypes_plain.parquet");