diff --git a/arrow-parquet-integration-testing/src/main.rs b/arrow-parquet-integration-testing/src/main.rs index ba60e7b8462..ee28ed94cd7 100644 --- a/arrow-parquet-integration-testing/src/main.rs +++ b/arrow-parquet-integration-testing/src/main.rs @@ -2,8 +2,10 @@ use std::fs::File; use std::sync::Arc; use std::{collections::HashMap, io::Read}; +use arrow2::array::Array; use arrow2::io::ipc::IpcField; use arrow2::{ + chunk::Chunk, datatypes::{DataType, Schema}, error::Result, io::{ @@ -13,7 +15,6 @@ use arrow2::{ write_file, Compression, Encoding, RowGroupIterator, Version, WriteOptions, }, }, - record_batch::RecordBatch, }; use clap::{App, Arg}; @@ -21,7 +22,10 @@ use clap::{App, Arg}; use flate2::read::GzDecoder; /// Read gzipped JSON file -fn read_gzip_json(version: &str, file_name: &str) -> (Schema, Vec, Vec) { +pub fn read_gzip_json( + version: &str, + file_name: &str, +) -> Result<(Schema, Vec, Vec>>)> { let path = format!( "../testing/arrow-testing/data/arrow-ipc-stream/integration/{}/{}.json.gz", version, file_name @@ -31,10 +35,11 @@ fn read_gzip_json(version: &str, file_name: &str) -> (Schema, Vec, Vec let mut s = String::new(); gz.read_to_string(&mut s).unwrap(); // convert to Arrow JSON - let arrow_json: ArrowJson = serde_json::from_str(&s).unwrap(); + let arrow_json: ArrowJson = serde_json::from_str(&s)?; let schema = serde_json::to_value(arrow_json.schema).unwrap(); - let (schema, ipc_fields) = read::deserialize_schema(&schema).unwrap(); + + let (schema, ipc_fields) = read::deserialize_schema(&schema)?; // read dictionaries let mut dictionaries = HashMap::new(); @@ -48,11 +53,10 @@ fn read_gzip_json(version: &str, file_name: &str) -> (Schema, Vec, Vec let batches = arrow_json .batches .iter() - .map(|batch| read::to_record_batch(&schema, &ipc_fields, batch, &dictionaries)) - .collect::>>() - .unwrap(); + .map(|batch| read::deserialize_chunk(&schema, &ipc_fields, batch, &dictionaries)) + .collect::>>()?; - (schema, ipc_fields, batches) + Ok((schema, ipc_fields, batches)) } fn main() -> Result<()> { @@ -108,7 +112,7 @@ fn main() -> Result<()> { .collect::>() }); - let (schema, _, batches) = read_gzip_json("1.0.0-littleendian", json_file); + let (schema, _, batches) = read_gzip_json("1.0.0-littleendian", json_file)?; let schema = if let Some(projection) = &projection { let fields = schema @@ -144,7 +148,7 @@ fn main() -> Result<()> { } }) .collect(); - RecordBatch::try_new(Arc::new(schema.clone()), columns).unwrap() + Chunk::try_new(columns).unwrap() }) .collect::>() } else { diff --git a/benches/filter_kernels.rs b/benches/filter_kernels.rs index be29dd58ae9..fb9376b0d66 100644 --- a/benches/filter_kernels.rs +++ b/benches/filter_kernels.rs @@ -19,9 +19,9 @@ use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::*; -use arrow2::compute::filter::{build_filter, filter, filter_record_batch, Filter}; +use arrow2::chunk::Chunk; +use arrow2::compute::filter::{build_filter, filter, filter_chunk, Filter}; use arrow2::datatypes::{DataType, Field, Schema}; -use arrow2::record_batch::RecordBatch; use arrow2::util::bench_util::{create_boolean_array, create_primitive_array, create_string_array}; fn bench_filter(data_array: &dyn Array, filter_array: &BooleanArray) { @@ -125,13 +125,10 @@ fn add_benchmark(c: &mut Criterion) { let data_array = create_primitive_array::(size, 0.0); - let field = Field::new("c1", data_array.data_type().clone(), true); - let schema = Schema::new(vec![field]); - - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(data_array)]).unwrap(); + let columns = Chunk::try_new(vec![Arc::new(data_array)]).unwrap(); c.bench_function("filter single record batch", |b| { - b.iter(|| filter_record_batch(&batch, &filter_array)) + b.iter(|| filter_record_batch(&columns, &filter_array)) }); } diff --git a/benches/read_parquet.rs b/benches/read_parquet.rs index 82fef98dba2..e4c3fa12022 100644 --- a/benches/read_parquet.rs +++ b/benches/read_parquet.rs @@ -37,9 +37,9 @@ fn read_batch(buffer: &[u8], size: usize, column: usize) -> Result<()> { let reader = read::RecordReader::try_new(file, Some(vec![column]), None, None, None)?; - for maybe_batch in reader { - let batch = maybe_batch?; - assert_eq!(batch.num_rows(), size); + for maybe_chunk in reader { + let columns = maybe_chunk?; + assert_eq!(columns.len(), size); } Ok(()) } diff --git a/benches/write_csv.rs b/benches/write_csv.rs index 3afa5952aae..d88afdc16f9 100644 --- a/benches/write_csv.rs +++ b/benches/write_csv.rs @@ -3,22 +3,23 @@ use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::*; +use arrow2::chunk::Chunk; use arrow2::error::Result; use arrow2::io::csv::write; -use arrow2::record_batch::RecordBatch; use arrow2::util::bench_util::*; -fn write_batch(batch: &RecordBatch) -> Result<()> { +fn write_batch(columns: &Chunk) -> Result<()> { let writer = &mut write::WriterBuilder::new().from_writer(vec![]); - write::write_header(writer, batch.schema())?; + assert_eq!(columns.arrays().len(), 1); + write::write_header(writer, &["a"])?; let options = write::SerializeOptions::default(); write::write_batch(writer, batch, &options) } -fn make_batch(array: impl Array + 'static) -> RecordBatch { - RecordBatch::try_from_iter([("a", Arc::new(array) as Arc)]).unwrap() +fn make_chunk(array: impl Array + 'static) -> Chunk> { + Chunk::new(vec![Arc::new(array)]) } fn add_benchmark(c: &mut Criterion) { @@ -26,21 +27,21 @@ fn add_benchmark(c: &mut Criterion) { let size = 2usize.pow(log2_size); let array = create_primitive_array::(size, 0.1); - let batch = make_batch(array); + let batch = make_chunk(array); c.bench_function(&format!("csv write i32 2^{}", log2_size), |b| { b.iter(|| write_batch(&batch)) }); let array = create_string_array::(size, 100, 0.1, 42); - let batch = make_batch(array); + let batch = make_chunk(array); c.bench_function(&format!("csv write utf8 2^{}", log2_size), |b| { b.iter(|| write_batch(&batch)) }); let array = create_primitive_array::(size, 0.1); - let batch = make_batch(array); + let batch = make_chunk(array); c.bench_function(&format!("csv write f64 2^{}", log2_size), |b| { b.iter(|| write_batch(&batch)) diff --git a/benches/write_ipc.rs b/benches/write_ipc.rs index 77b1ab95905..2ec35a5b9a4 100644 --- a/benches/write_ipc.rs +++ b/benches/write_ipc.rs @@ -4,21 +4,21 @@ use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::*; +use arrow2::chunk::Chunk; use arrow2::datatypes::{Field, Schema}; use arrow2::error::Result; use arrow2::io::ipc::write::*; -use arrow2::record_batch::RecordBatch; use arrow2::util::bench_util::{create_boolean_array, create_primitive_array, create_string_array}; fn write(array: &dyn Array) -> Result<()> { let field = Field::new("c1", array.data_type().clone(), true); let schema = Schema::new(vec![field]); - let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![clone(array).into()])?; + let columns = Chunk::try_new(vec![clone(array).into()])?; let writer = Cursor::new(vec![]); - let mut writer = FileWriter::try_new(writer, &schema, Default::default())?; + let mut writer = FileWriter::try_new(writer, &schema, None, Default::default())?; - writer.write(&batch) + writer.write(&columns, None) } fn add_benchmark(c: &mut Criterion) { diff --git a/benches/write_json.rs b/benches/write_json.rs index 354ebfab0e0..11f550c90f5 100644 --- a/benches/write_json.rs +++ b/benches/write_json.rs @@ -3,19 +3,19 @@ use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::*; +use arrow2::chunk::Chunk; use arrow2::error::Result; use arrow2::io::json::write; -use arrow2::record_batch::RecordBatch; use arrow2::util::bench_util::*; -fn write_batch(batch: &RecordBatch) -> Result<()> { +fn write_batch(columns: &Chunk>) -> Result<()> { let mut writer = vec![]; let format = write::JsonArray::default(); - let batches = vec![Ok(batch.clone())].into_iter(); + let batches = vec![Ok(columns.clone())].into_iter(); // Advancing this iterator serializes the next batch to its internal buffer (i.e. CPU-bounded) - let blocks = write::Serializer::new(batches, vec![], format); + let blocks = write::Serializer::new(batches, vec!["c1".to_string()], vec![], format); // the operation of writing is IO-bounded. write::write(&mut writer, format, blocks)?; @@ -23,8 +23,8 @@ fn write_batch(batch: &RecordBatch) -> Result<()> { Ok(()) } -fn make_batch(array: impl Array + 'static) -> RecordBatch { - RecordBatch::try_from_iter([("a", Arc::new(array) as Arc)]).unwrap() +fn make_chunk(array: impl Array + 'static) -> Chunk> { + Chunk::new(vec![Arc::new(array) as Arc]) } fn add_benchmark(c: &mut Criterion) { @@ -32,24 +32,24 @@ fn add_benchmark(c: &mut Criterion) { let size = 2usize.pow(log2_size); let array = create_primitive_array::(size, 0.1); - let batch = make_batch(array); + let columns = make_chunk(array); c.bench_function(&format!("json write i32 2^{}", log2_size), |b| { - b.iter(|| write_batch(&batch)) + b.iter(|| write_batch(&columns)) }); let array = create_string_array::(size, 100, 0.1, 42); - let batch = make_batch(array); + let columns = make_chunk(array); c.bench_function(&format!("json write utf8 2^{}", log2_size), |b| { - b.iter(|| write_batch(&batch)) + b.iter(|| write_batch(&columns)) }); let array = create_primitive_array::(size, 0.1); - let batch = make_batch(array); + let columns = make_chunk(array); c.bench_function(&format!("json write f64 2^{}", log2_size), |b| { - b.iter(|| write_batch(&batch)) + b.iter(|| write_batch(&columns)) }); }); } diff --git a/benches/write_parquet.rs b/benches/write_parquet.rs index df89c36d777..e8dc072e24f 100644 --- a/benches/write_parquet.rs +++ b/benches/write_parquet.rs @@ -3,13 +3,13 @@ use std::io::Cursor; use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::{clone, Array}; +use arrow2::chunk::Chunk; use arrow2::error::Result; use arrow2::io::parquet::write::*; -use arrow2::record_batch::RecordBatch; use arrow2::util::bench_util::{create_boolean_array, create_primitive_array, create_string_array}; fn write(array: &dyn Array, encoding: Encoding) -> Result<()> { - let batch = RecordBatch::try_from_iter([("c1", clone(array).into())])?; + let columns = Chunk::new(vec![clone(array).into()]); let schema = batch.schema().clone(); let options = WriteOptions { @@ -19,7 +19,7 @@ fn write(array: &dyn Array, encoding: Encoding) -> Result<()> { }; let row_groups = RowGroupIterator::try_new( - vec![Ok(batch)].into_iter(), + vec![Ok(columns)].into_iter(), &schema, options, vec![encoding], diff --git a/examples/avro_read.rs b/examples/avro_read.rs index 33a1e717b47..6d7722ac1d4 100644 --- a/examples/avro_read.rs +++ b/examples/avro_read.rs @@ -1,6 +1,5 @@ use std::fs::File; use std::io::BufReader; -use std::sync::Arc; use arrow2::error::Result; use arrow2::io::avro::read; @@ -20,12 +19,12 @@ fn main() -> Result<()> { let reader = read::Reader::new( read::Decompressor::new(read::BlockStreamIterator::new(file, file_marker), codec), avro_schema, - Arc::new(schema), + schema.fields, ); - for batch in reader { - let batch = batch?; - assert!(batch.num_rows() > 0); + for maybe_chunk in reader { + let columns = maybe_chunk?; + assert!(!columns.is_empty()); } Ok(()) } diff --git a/examples/avro_read_async.rs b/examples/avro_read_async.rs index adf67855e2d..d3c5940b398 100644 --- a/examples/avro_read_async.rs +++ b/examples/avro_read_async.rs @@ -19,7 +19,6 @@ async fn main() -> Result<()> { let mut reader = File::open(file_path).await?.compat(); let (avro_schemas, schema, compression, marker) = read_metadata(&mut reader).await?; - let schema = Arc::new(schema); let avro_schemas = Arc::new(avro_schemas); let blocks = block_stream(&mut reader, marker).await; @@ -32,10 +31,10 @@ async fn main() -> Result<()> { let handle = tokio::task::spawn_blocking(move || { let mut decompressed = Block::new(0, vec![]); decompress_block(&mut block, &mut decompressed, compression)?; - deserialize(&decompressed, schema, &avro_schemas) + deserialize(&decompressed, schema.fields(), &avro_schemas) }); let batch = handle.await.unwrap()?; - assert!(batch.num_rows() > 0); + assert!(!batch.is_empty()); } Ok(()) diff --git a/examples/csv_read.rs b/examples/csv_read.rs index 2999f0d8b5a..6d0319c13c4 100644 --- a/examples/csv_read.rs +++ b/examples/csv_read.rs @@ -1,8 +1,11 @@ +use std::sync::Arc; + +use arrow2::array::Array; +use arrow2::chunk::Chunk; use arrow2::error::Result; use arrow2::io::csv::read; -use arrow2::record_batch::RecordBatch; -fn read_path(path: &str, projection: Option<&[usize]>) -> Result { +fn read_path(path: &str, projection: Option<&[usize]>) -> Result>> { // Create a CSV reader. This is typically created on the thread that reads the file and // thus owns the read head. let mut reader = read::ReaderBuilder::new().from_path(path)?; diff --git a/examples/csv_read_async.rs b/examples/csv_read_async.rs index 7f89999e900..94b6b072111 100644 --- a/examples/csv_read_async.rs +++ b/examples/csv_read_async.rs @@ -22,13 +22,13 @@ async fn main() -> Result<()> { let mut rows = vec![ByteRecord::default(); 100]; let rows_read = read_rows(&mut reader, 0, &mut rows).await?; - let batch = deserialize_batch( + let columns = deserialize_batch( &rows[..rows_read], schema.fields(), None, 0, deserialize_column, )?; - println!("{:?}", batch.column(0)); + println!("{:?}", columns.arrays()[0]); Ok(()) } diff --git a/examples/csv_read_parallel.rs b/examples/csv_read_parallel.rs index f40a6d7fa18..ad9744b64c5 100644 --- a/examples/csv_read_parallel.rs +++ b/examples/csv_read_parallel.rs @@ -4,9 +4,11 @@ use std::sync::Arc; use std::thread; use std::time::SystemTime; -use arrow2::{error::Result, io::csv::read, record_batch::RecordBatch}; +use arrow2::array::Array; +use arrow2::chunk::Chunk; +use arrow2::{error::Result, io::csv::read}; -fn parallel_read(path: &str) -> Result> { +fn parallel_read(path: &str) -> Result>>> { let batch_size = 100; let has_header = true; let projection = None; @@ -78,7 +80,7 @@ fn main() -> Result<()> { let batches = parallel_read(file_path)?; for batch in batches { - println!("{}", batch.num_rows()) + println!("{}", batch.len()) } Ok(()) } diff --git a/examples/csv_write.rs b/examples/csv_write.rs index 805aa0a0949..f9f73cfef53 100644 --- a/examples/csv_write.rs +++ b/examples/csv_write.rs @@ -1,22 +1,19 @@ -use std::sync::Arc; - use arrow2::{ array::{Array, Int32Array}, - datatypes::{Field, Schema}, + chunk::Chunk, error::Result, io::csv::write, - record_batch::RecordBatch, }; -fn write_batch(path: &str, batches: &[RecordBatch]) -> Result<()> { +fn write_batch>(path: &str, columns: &[Chunk]) -> Result<()> { let writer = &mut write::WriterBuilder::new().from_path(path)?; - write::write_header(writer, batches[0].schema())?; + write::write_header(writer, &["c1"])?; let options = write::SerializeOptions::default(); - batches + columns .iter() - .try_for_each(|batch| write::write_batch(writer, batch, &options)) + .try_for_each(|batch| write::write_chunk(writer, batch, &options)) } fn main() -> Result<()> { @@ -29,9 +26,7 @@ fn main() -> Result<()> { Some(5), Some(6), ]); - let field = Field::new("c1", array.data_type().clone(), true); - let schema = Schema::new(vec![field]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)])?; + let batch = Chunk::try_new(vec![&array as &dyn Array])?; write_batch("example.csv", &[batch]) } diff --git a/examples/csv_write_parallel.rs b/examples/csv_write_parallel.rs index 54e8453fcfa..f616a1b8292 100644 --- a/examples/csv_write_parallel.rs +++ b/examples/csv_write_parallel.rs @@ -4,19 +4,18 @@ use std::sync::Arc; use std::thread; use arrow2::{ - array::Int32Array, - datatypes::{Field, Schema}, + array::{Array, Int32Array}, + chunk::Chunk, error::Result, io::csv::write, - record_batch::RecordBatch, }; -fn parallel_write(path: &str, batches: [RecordBatch; 2]) -> Result<()> { +fn parallel_write(path: &str, batches: [Chunk>; 2]) -> Result<()> { let options = write::SerializeOptions::default(); // write a header let writer = &mut write::WriterBuilder::new().from_path(path)?; - write::write_header(writer, batches[0].schema())?; + write::write_header(writer, &["c1"])?; // prepare a channel to send serialized records from threads let (tx, rx): (Sender<_>, Receiver<_>) = mpsc::channel(); @@ -61,9 +60,7 @@ fn main() -> Result<()> { Some(5), Some(6), ]); - let field = Field::new("c1", array.data_type().clone(), true); - let schema = Schema::new(vec![field]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)])?; + let columns = Chunk::new(vec![Arc::new(array) as Arc]); - parallel_write("example.csv", [batch.clone(), batch]) + parallel_write("example.csv", [columns.clone(), columns]) } diff --git a/examples/extension.rs b/examples/extension.rs index a9659a821ae..24e390a4c1d 100644 --- a/examples/extension.rs +++ b/examples/extension.rs @@ -2,11 +2,11 @@ use std::io::{Cursor, Seek, Write}; use std::sync::Arc; use arrow2::array::*; +use arrow2::chunk::Chunk; use arrow2::datatypes::*; use arrow2::error::Result; use arrow2::io::ipc::read; use arrow2::io::ipc::write; -use arrow2::record_batch::RecordBatch; fn main() -> Result<()> { // declare an extension. @@ -40,14 +40,14 @@ fn write_ipc(writer: W, array: impl Array + 'static) -> Result< let options = write::WriteOptions { compression: None }; let mut writer = write::FileWriter::try_new(writer, &schema, None, options)?; - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)])?; + let batch = Chunk::try_new(vec![Arc::new(array) as Arc])?; writer.write(&batch, None)?; Ok(writer.into_inner()) } -fn read_ipc(buf: &[u8]) -> Result { +fn read_ipc(buf: &[u8]) -> Result>> { let mut cursor = Cursor::new(buf); let metadata = read::read_file_metadata(&mut cursor)?; let mut reader = read::FileReader::new(cursor, metadata, None); diff --git a/examples/ipc_file_read.rs b/examples/ipc_file_read.rs index a1792f10399..a88e18f25c1 100644 --- a/examples/ipc_file_read.rs +++ b/examples/ipc_file_read.rs @@ -1,20 +1,26 @@ use std::fs::File; +use std::sync::Arc; +use arrow2::array::Array; +use arrow2::chunk::Chunk; +use arrow2::datatypes::Schema; use arrow2::error::Result; use arrow2::io::ipc::read::{read_file_metadata, FileReader}; use arrow2::io::print; -use arrow2::record_batch::RecordBatch; -fn read_batches(path: &str) -> Result> { +fn read_batches(path: &str) -> Result<(Schema, Vec>>)> { let mut file = File::open(path)?; // read the files' metadata. At this point, we can distribute the read whatever we like. let metadata = read_file_metadata(&mut file)?; + let schema = metadata.schema.clone(); + // Simplest way: use the reader, an iterator over batches. let reader = FileReader::new(file, metadata, None); - reader.collect() + let columns = reader.collect::>>()?; + Ok((schema, columns)) } fn main() -> Result<()> { @@ -23,7 +29,8 @@ fn main() -> Result<()> { let file_path = &args[1]; - let batches = read_batches(file_path)?; - print::print(&batches); + let (schema, batches) = read_batches(file_path)?; + let names = schema.fields().iter().map(|f| f.name()).collect::>(); + println!("{}", print::write(&batches, &names)); Ok(()) } diff --git a/examples/ipc_file_write.rs b/examples/ipc_file_write.rs index 757622b8ced..219d6d9d548 100644 --- a/examples/ipc_file_write.rs +++ b/examples/ipc_file_write.rs @@ -1,20 +1,20 @@ use std::fs::File; use std::sync::Arc; -use arrow2::array::{Int32Array, Utf8Array}; +use arrow2::array::{Array, Int32Array, Utf8Array}; +use arrow2::chunk::Chunk; use arrow2::datatypes::{DataType, Field, Schema}; use arrow2::error::Result; use arrow2::io::ipc::write; -use arrow2::record_batch::RecordBatch; -fn write_batches(path: &str, schema: &Schema, batches: &[RecordBatch]) -> Result<()> { +fn write_batches(path: &str, schema: &Schema, columns: &[Chunk>]) -> Result<()> { let file = File::create(path)?; let options = write::WriteOptions { compression: None }; let mut writer = write::FileWriter::try_new(file, schema, None, options)?; - for batch in batches { - writer.write(batch, None)? + for columns in columns { + writer.write(columns, None)? } writer.finish() } @@ -34,7 +34,7 @@ fn main() -> Result<()> { let a = Int32Array::from_slice(&[1, 2, 3, 4, 5]); let b = Utf8Array::::from_slice(&["a", "b", "c", "d", "e"]); - let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(a), Arc::new(b)])?; + let batch = Chunk::try_new(vec![Arc::new(a) as Arc, Arc::new(b)])?; // write it write_batches(file_path, &schema, &[batch])?; diff --git a/examples/json_read.rs b/examples/json_read.rs index 9c975f2067a..a3eb2f33ef4 100644 --- a/examples/json_read.rs +++ b/examples/json_read.rs @@ -1,11 +1,13 @@ use std::fs::File; use std::io::BufReader; +use std::sync::Arc; +use arrow2::array::Array; +use arrow2::chunk::Chunk; use arrow2::error::Result; use arrow2::io::json::read; -use arrow2::record_batch::RecordBatch; -fn read_path(path: &str, projection: Option>) -> Result { +fn read_path(path: &str, projection: Option>) -> Result>> { // Example of reading a JSON file. let mut reader = BufReader::new(File::open(path)?); @@ -29,9 +31,9 @@ fn read_path(path: &str, projection: Option>) -> Result { let read = read::read_rows(&mut reader, &mut rows)?; let rows = &rows[..read]; - // deserialize `rows` into a `RecordBatch`. This is CPU-intensive, has no IO, + // deserialize `rows` into `Chunk`. This is CPU-intensive, has no IO, // and can be performed on a different thread pool via a channel. - read::deserialize(rows, fields) + read::deserialize(rows, &fields) } fn main() -> Result<()> { diff --git a/examples/json_write.rs b/examples/json_write.rs index e0679e6f1ba..07213fc3ddc 100644 --- a/examples/json_write.rs +++ b/examples/json_write.rs @@ -2,21 +2,20 @@ use std::fs::File; use std::sync::Arc; use arrow2::{ - array::Int32Array, - datatypes::{Field, Schema}, + array::{Array, Int32Array}, + chunk::Chunk, error::Result, io::json::write, - record_batch::RecordBatch, }; -fn write_batches(path: &str, batches: &[RecordBatch]) -> Result<()> { +fn write_batches(path: &str, names: Vec, batches: &[Chunk>]) -> Result<()> { let mut writer = File::create(path)?; let format = write::JsonArray::default(); let batches = batches.iter().cloned().map(Ok); // Advancing this iterator serializes the next batch to its internal buffer (i.e. CPU-bounded) - let blocks = write::Serializer::new(batches, vec![], format); + let blocks = write::Serializer::new(batches, names, vec![], format); // the operation of writing is IO-bounded. write::write(&mut writer, format, blocks)?; @@ -25,10 +24,19 @@ fn write_batches(path: &str, batches: &[RecordBatch]) -> Result<()> { } fn main() -> Result<()> { - let array = Int32Array::from(&[Some(0), None, Some(2), Some(3), Some(4), Some(5), Some(6)]); - let field = Field::new("c1", array.data_type().clone(), true); - let schema = Schema::new(vec![field]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)])?; - - write_batches("example.json", &[batch.clone(), batch]) + let array = Arc::new(Int32Array::from(&[ + Some(0), + None, + Some(2), + Some(3), + Some(4), + Some(5), + Some(6), + ])) as Arc; + + write_batches( + "example.json", + vec!["c1".to_string()], + &[Chunk::new(vec![array.clone()]), Chunk::new(vec![array])], + ) } diff --git a/examples/metadata.rs b/examples/metadata.rs index bbfc830df5a..b4990817b61 100644 --- a/examples/metadata.rs +++ b/examples/metadata.rs @@ -1,4 +1,4 @@ -use std::collections::{BTreeMap, HashMap}; +use std::collections::HashMap; use arrow2::datatypes::{DataType, Field, Metadata, Schema}; diff --git a/examples/parquet_read_parallel.rs b/examples/parquet_read_parallel.rs index fd474201c19..f3d206ac6b1 100644 --- a/examples/parquet_read_parallel.rs +++ b/examples/parquet_read_parallel.rs @@ -6,11 +6,11 @@ use std::time::SystemTime; use crossbeam_channel::unbounded; use arrow2::{ - array::Array, error::Result, io::parquet::read, io::parquet::read::MutStreamingIterator, - record_batch::RecordBatch, + array::Array, chunk::Chunk, error::Result, io::parquet::read, + io::parquet::read::MutStreamingIterator, }; -fn parallel_read(path: &str, row_group: usize) -> Result { +fn parallel_read(path: &str, row_group: usize) -> Result>> { // prepare a channel to send compressed pages across threads. let (tx, rx) = unbounded(); @@ -99,7 +99,7 @@ fn parallel_read(path: &str, row_group: usize) -> Result { let columns = columns.into_iter().map(|x| x.1.into()).collect(); println!("Finished - {:?}", start.elapsed().unwrap()); - RecordBatch::try_new(arrow_schema, columns) + Chunk::try_new(columns) } fn main() -> Result<()> { @@ -109,7 +109,7 @@ fn main() -> Result<()> { let start = SystemTime::now(); let batch = parallel_read(file_path, 0)?; - assert!(batch.num_rows() > 0); + assert!(!batch.is_empty()); println!("took: {} ms", start.elapsed().unwrap().as_millis()); Ok(()) } diff --git a/examples/parquet_read_parallel/src/main.rs b/examples/parquet_read_parallel/src/main.rs index aa12fcd5fb9..827343ee0f2 100644 --- a/examples/parquet_read_parallel/src/main.rs +++ b/examples/parquet_read_parallel/src/main.rs @@ -7,14 +7,14 @@ use std::time::SystemTime; use rayon::prelude::*; use arrow2::{ - error::Result, io::parquet::read, io::parquet::read::MutStreamingIterator, - record_batch::RecordBatch, + array::Array, chunk::Chunk, error::Result, io::parquet::read, + io::parquet::read::MutStreamingIterator, }; -fn parallel_read(path: &str, row_group: usize) -> Result { +fn parallel_read(path: &str, row_group: usize) -> Result>> { let mut file = BufReader::new(File::open(path)?); let file_metadata = read::read_metadata(&mut file)?; - let arrow_schema = Arc::new(read::get_schema(&file_metadata)?); + let schema = read::get_schema(&file_metadata)?; // IO-bounded let columns = file_metadata @@ -58,13 +58,13 @@ fn parallel_read(path: &str, row_group: usize) -> Result { .into_par_iter() .map(|(field_i, parquet_field, column_chunks)| { let columns = read::ReadColumnIterator::new(parquet_field, column_chunks); - let field = &arrow_schema.fields()[field_i]; + let field = &schema.fields()[field_i]; read::column_iter_to_array(columns, field, vec![]).map(|x| x.0.into()) }) .collect::>>()?; - RecordBatch::try_new(arrow_schema, columns) + Chunk::try_new(columns) } fn main() -> Result<()> { @@ -75,7 +75,7 @@ fn main() -> Result<()> { let start = SystemTime::now(); let batch = parallel_read(file_path, row_group)?; - assert!(batch.num_rows() > 0); + assert!(!batch.is_empty()); println!("took: {} ms", start.elapsed().unwrap().as_millis()); Ok(()) diff --git a/examples/parquet_read_record.rs b/examples/parquet_read_record.rs index 6f9e1db4d6f..9cb2f65ff91 100644 --- a/examples/parquet_read_record.rs +++ b/examples/parquet_read_record.rs @@ -14,9 +14,9 @@ fn main() -> Result<()> { let reader = read::RecordReader::try_new(reader, None, None, None, None)?; let start = SystemTime::now(); - for maybe_batch in reader { - let batch = maybe_batch?; - assert!(batch.num_rows() > 0); + for maybe_chunk in reader { + let columns = maybe_chunk?; + assert!(!columns.is_empty()); } println!("took: {} ms", start.elapsed().unwrap().as_millis()); Ok(()) diff --git a/examples/parquet_write_parallel/src/main.rs b/examples/parquet_write_parallel/src/main.rs index feef774decd..915bbe98233 100644 --- a/examples/parquet_write_parallel/src/main.rs +++ b/examples/parquet_write_parallel/src/main.rs @@ -6,12 +6,14 @@ use rayon::prelude::*; use arrow2::{ array::*, - datatypes::PhysicalType, + chunk::Chunk as AChunk, + datatypes::*, error::{ArrowError, Result}, io::parquet::write::*, - record_batch::RecordBatch, }; +type Chunk = AChunk>; + struct Bla { columns: VecDeque, current: Option, @@ -40,13 +42,16 @@ impl FallibleStreamingIterator for Bla { } } -fn parallel_write(path: &str, batches: &[RecordBatch]) -> Result<()> { +fn parallel_write(path: &str, schema: &Schema, batches: &[Chunk]) -> Result<()> { + // declare the options let options = WriteOptions { write_statistics: true, compression: Compression::Snappy, version: Version::V2, }; - let encodings = batches[0].schema().fields().par_iter().map(|field| { + + // declare encodings + let encodings = schema.fields().par_iter().map(|field| { match field.data_type().to_physical_type() { // let's be fancy and use delta-encoding for binary fields PhysicalType::Binary @@ -58,7 +63,8 @@ fn parallel_write(path: &str, batches: &[RecordBatch]) -> Result<()> { } }); - let parquet_schema = to_parquet_schema(batches[0].schema())?; + // derive the parquet schema (physical types) from arrow's schema. + let parquet_schema = to_parquet_schema(schema)?; let a = parquet_schema.clone(); let row_groups = batches.iter().map(|batch| { @@ -89,19 +95,12 @@ fn parallel_write(path: &str, batches: &[RecordBatch]) -> Result<()> { let mut file = std::fs::File::create(path)?; // Write the file. - let _file_size = write_file( - &mut file, - row_groups, - batches[0].schema(), - parquet_schema, - options, - None, - )?; + let _file_size = write_file(&mut file, row_groups, schema, parquet_schema, options, None)?; Ok(()) } -fn create_batch(size: usize) -> Result { +fn create_batch(size: usize) -> Result { let c1: Int32Array = (0..size) .map(|x| if x % 9 == 0 { None } else { Some(x as i32) }) .collect(); @@ -115,14 +114,21 @@ fn create_batch(size: usize) -> Result { }) .collect(); - RecordBatch::try_from_iter([ - ("c1", Arc::new(c1) as Arc), - ("c2", Arc::new(c2) as Arc), + Chunk::try_new(vec![ + Arc::new(c1) as Arc, + Arc::new(c2) as Arc, ]) } fn main() -> Result<()> { + let schema = Schema { + fields: vec![ + Field::new("c1", DataType::Int32, true), + Field::new("c1", DataType::Utf8, true), + ], + metadata: Default::default(), + }; let batch = create_batch(5_000_000)?; - parallel_write("example.parquet", &[batch.clone(), batch]) + parallel_write("example.parquet", &schema, &[batch.clone(), batch]) } diff --git a/examples/parquet_write_record.rs b/examples/parquet_write_record.rs index 290e20ab102..ace66402540 100644 --- a/examples/parquet_write_record.rs +++ b/examples/parquet_write_record.rs @@ -2,25 +2,23 @@ use std::fs::File; use std::sync::Arc; use arrow2::{ - array::Int32Array, + array::{Array, Int32Array}, + chunk::Chunk, datatypes::{Field, Schema}, error::Result, io::parquet::write::{ write_file, Compression, Encoding, RowGroupIterator, Version, WriteOptions, }, - record_batch::RecordBatch, }; -fn write_batch(path: &str, batch: RecordBatch) -> Result<()> { - let schema = batch.schema().clone(); - +fn write_batch(path: &str, schema: Schema, columns: Chunk>) -> Result<()> { let options = WriteOptions { write_statistics: true, compression: Compression::Uncompressed, version: Version::V2, }; - let iter = vec![Ok(batch)]; + let iter = vec![Ok(columns)]; let row_groups = RowGroupIterator::try_new(iter.into_iter(), &schema, options, vec![Encoding::Plain])?; @@ -53,7 +51,7 @@ fn main() -> Result<()> { ]); let field = Field::new("c1", array.data_type().clone(), true); let schema = Schema::new(vec![field]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)])?; + let columns = Chunk::new(vec![Arc::new(array) as Arc]); - write_batch("test.parquet", batch) + write_batch("test.parquet", schema, columns) } diff --git a/guide/src/arrow.md b/guide/src/arrow.md index d59166205fb..d1abb5196f3 100644 --- a/guide/src/arrow.md +++ b/guide/src/arrow.md @@ -89,7 +89,7 @@ sharing can be done regarding the language that is used. And thanks to this standardization the data can also be shared with processes that don't share the same memory. By creating a data server, packets of data -with known structure (RecordBatch) can be sent across computers (or pods) and +with known structure (Chunk) can be sent across computers (or pods) and the receiving process doesn't need to spend time coding and decoding the data to a known format. The data is ready to be used once its being received. diff --git a/guide/src/io/csv_reader.md b/guide/src/io/csv_reader.md index 6b9d3161209..b0ee579d232 100644 --- a/guide/src/io/csv_reader.md +++ b/guide/src/io/csv_reader.md @@ -7,7 +7,7 @@ This crate makes minimal assumptions on how you want to read a CSV, and offers a There are two CPU-intensive tasks in reading a CSV file: * split the CSV file into rows, which includes parsing quotes and delimiters, and is necessary to `seek` to a given row. -* parse a set of CSV rows (bytes) into a `RecordBatch`. +* parse a set of CSV rows (bytes) into a `Array`s. Parsing bytes into values is more expensive than interpreting lines. As such, it is generally advantageous to have multiple readers of a single file that scan different parts of the file (within IO constraints). diff --git a/guide/src/io/ipc_read.md b/guide/src/io/ipc_read.md index 5ecaf6d3792..bd8b39d54a8 100644 --- a/guide/src/io/ipc_read.md +++ b/guide/src/io/ipc_read.md @@ -2,13 +2,13 @@ When compiled with feature `io_ipc`, this crate can be used to read Arrow files. -An Arrow file is composed by a header, a footer, and blocks of `RecordBatch`es. +An Arrow file is composed by a header, a footer, and blocks of `Array`s. Reading it generally consists of: 1. read metadata, containing the block positions in the file 2. seek to each block and read it -The example below shows how to read them into `RecordBatch`es: +The example below shows how to read them into `Chunk`es: ```rust {{#include ../../../examples/ipc_file_read.rs}} diff --git a/guide/src/io/parquet_read.md b/guide/src/io/parquet_read.md index 48ba8651ced..7ced401c849 100644 --- a/guide/src/io/parquet_read.md +++ b/guide/src/io/parquet_read.md @@ -6,9 +6,9 @@ It makes minimal assumptions on how you to decompose CPU and IO intensive tasks. First, some notation: -* `page`: part of a column (e.g. similar of a slice of an `Array`) -* `column chunk`: composed of multiple pages (similar of an `Array`) -* `row group`: a group of columns with the same length (similar of a `RecordBatch` in Arrow) +* `page`: part of a column (e.g. similar to a slice of an `Array`) +* `column chunk`: composed of multiple pages (similar to an `Array`) +* `row group`: a group of columns with the same length (similar to a `Chunk`) Here is how to read a single column chunk from a single row group: diff --git a/guide/src/io/parquet_write.md b/guide/src/io/parquet_write.md index 4e9cf75036e..a0272bd9709 100644 --- a/guide/src/io/parquet_write.md +++ b/guide/src/io/parquet_write.md @@ -7,9 +7,9 @@ as an higher-level API to abstract away some of this work into an easy to use AP First, some notation: -* `page`: part of a column (e.g. similar of a slice of an `Array`) -* `column chunk`: composed of multiple pages (similar of an `Array`) -* `row group`: a group of columns with the same length (similar of a `RecordBatch` in Arrow) +* `page`: part of a column (e.g. similar to a slice of an `Array`) +* `column chunk`: composed of multiple pages (similar to an `Array`) +* `row group`: a group of columns with the same length (similar to a `Chunk` in Arrow) ## Single threaded @@ -20,7 +20,7 @@ Here is an example of how to write a single column chunk into a single row group ``` For single-threaded writing, this crate offers an API that encapsulates the above logic. It -assumes that a `RecordBatch` is mapped to a single row group with a single page per column. +assumes that a `Chunk` is mapped to a single row group with a single page per column. ```rust {{#include ../../../examples/parquet_write_record.rs}} diff --git a/integration-testing/src/bin/arrow-json-integration-test.rs b/integration-testing/src/bin/arrow-json-integration-test.rs index 180896a9418..6cbb0766f4f 100644 --- a/integration-testing/src/bin/arrow-json-integration-test.rs +++ b/integration-testing/src/bin/arrow-json-integration-test.rs @@ -91,10 +91,17 @@ fn arrow_to_json(arrow_name: &str, json_name: &str, verbose: bool) -> Result<()> let metadata = read::read_file_metadata(&mut arrow_file)?; let reader = read::FileReader::new(arrow_file, metadata.clone(), None); + let names = metadata + .schema + .fields + .iter() + .map(|f| f.name()) + .collect::>(); + let schema = json_write::serialize_schema(&metadata.schema, &metadata.ipc_schema.fields); let batches = reader - .map(|batch| Ok(json_write::from_record_batch(&batch?))) + .map(|batch| Ok(json_write::serialize_chunk(&batch?, &names))) .collect::>>()?; let arrow_json = ArrowJson { @@ -121,10 +128,10 @@ fn validate(arrow_name: &str, json_name: &str, verbose: bool) -> Result<()> { let mut arrow_file = File::open(arrow_name)?; let metadata = read::read_file_metadata(&mut arrow_file)?; let reader = read::FileReader::new(arrow_file, metadata, None); - let arrow_schema = reader.schema().as_ref().to_owned(); + let arrow_schema = reader.schema(); // compare schemas - if json_file.schema != arrow_schema { + if &json_file.schema != arrow_schema { return Err(ArrowError::InvalidArgumentError(format!( "Schemas do not match. JSON: {:?}. Arrow: {:?}", json_file.schema, arrow_schema diff --git a/integration-testing/src/flight_client_scenarios/integration_test.rs b/integration-testing/src/flight_client_scenarios/integration_test.rs index 461659069dc..049be88f772 100644 --- a/integration-testing/src/flight_client_scenarios/integration_test.rs +++ b/integration-testing/src/flight_client_scenarios/integration_test.rs @@ -18,6 +18,8 @@ use crate::{read_json_file, ArrowFile}; use arrow2::{ + array::Array, + chunk::Chunk, datatypes::*, io::ipc::{ read::{self, Dictionaries}, @@ -27,7 +29,6 @@ use arrow2::{ flight::{self, deserialize_batch, serialize_batch}, ipc::IpcField, }, - record_batch::RecordBatch, }; use arrow_format::flight::data::{ flight_descriptor::DescriptorType, FlightData, FlightDescriptor, Location, Ticket, @@ -45,6 +46,8 @@ type Result = std::result::Result; type Client = FlightServiceClient; +type ChunkArc = Chunk>; + pub async fn run_scenario(host: &str, port: &str, path: &str) -> Result { let url = format!("http://{}:{}", host, port); @@ -75,7 +78,7 @@ pub async fn run_scenario(host: &str, port: &str, path: &str) -> Result { batches.clone(), ) .await?; - verify_data(client, descriptor, schema, &ipc_schema, &batches).await?; + verify_data(client, descriptor, &schema, &ipc_schema, &batches).await?; Ok(()) } @@ -85,7 +88,7 @@ async fn upload_data( schema: &Schema, fields: &[IpcField], descriptor: FlightDescriptor, - original_data: Vec, + original_data: Vec, ) -> Result { let (mut upload_tx, upload_rx) = mpsc::channel(10); @@ -140,7 +143,7 @@ async fn upload_data( async fn send_batch( upload_tx: &mut mpsc::Sender, metadata: &[u8], - batch: &RecordBatch, + batch: &ChunkArc, fields: &[IpcField], options: &write::WriteOptions, ) -> Result { @@ -159,9 +162,9 @@ async fn send_batch( async fn verify_data( mut client: Client, descriptor: FlightDescriptor, - expected_schema: SchemaRef, + expected_schema: &Schema, ipc_schema: &IpcSchema, - expected_data: &[RecordBatch], + expected_data: &[ChunkArc], ) -> Result { let resp = client.get_flight_info(Request::new(descriptor)).await?; let info = resp.into_inner(); @@ -184,7 +187,7 @@ async fn verify_data( location, ticket.clone(), expected_data, - expected_schema.clone(), + expected_schema, ipc_schema, ) .await?; @@ -197,8 +200,8 @@ async fn verify_data( async fn consume_flight_location( location: Location, ticket: Ticket, - expected_data: &[RecordBatch], - schema: SchemaRef, + expected_data: &[ChunkArc], + schema: &Schema, ipc_schema: &IpcSchema, ) -> Result { let mut location = location; @@ -231,21 +234,20 @@ async fn consume_flight_location( let metadata = counter.to_string().into_bytes(); assert_eq!(metadata, data.app_metadata); - let actual_batch = deserialize_batch(&data, schema.clone(), ipc_schema, &dictionaries) + let actual_batch = deserialize_batch(&data, schema.fields(), ipc_schema, &dictionaries) .expect("Unable to convert flight data to Arrow batch"); - assert_eq!(expected_batch.schema(), actual_batch.schema()); - assert_eq!(expected_batch.num_columns(), actual_batch.num_columns()); - assert_eq!(expected_batch.num_rows(), actual_batch.num_rows()); - let schema = expected_batch.schema(); - for i in 0..expected_batch.num_columns() { + assert_eq!(expected_batch.columns().len(), actual_batch.columns().len()); + assert_eq!(expected_batch.len(), actual_batch.len()); + for (i, (expected, actual)) in expected_batch + .columns() + .iter() + .zip(actual_batch.columns().iter()) + .enumerate() + { let field = schema.field(i); let field_name = field.name(); - - let expected_data = expected_batch.column(i); - let actual_data = actual_batch.column(i); - - assert_eq!(expected_data, actual_data, "Data for field {}", field_name); + assert_eq!(expected, actual, "Data for field {}", field_name); } } diff --git a/integration-testing/src/flight_server_scenarios/integration_test.rs b/integration-testing/src/flight_server_scenarios/integration_test.rs index 82e99f503d7..bd0d2532107 100644 --- a/integration-testing/src/flight_server_scenarios/integration_test.rs +++ b/integration-testing/src/flight_server_scenarios/integration_test.rs @@ -19,6 +19,8 @@ use std::collections::HashMap; use std::pin::Pin; use std::sync::Arc; +use arrow2::array::Array; +use arrow2::chunk::Chunk; use arrow2::io::flight::{deserialize_schemas, serialize_batch, serialize_schema}; use arrow2::io::ipc::read::Dictionaries; use arrow2::io::ipc::IpcSchema; @@ -28,9 +30,7 @@ use arrow_format::flight::service::flight_service_server::*; use arrow_format::ipc::Message::{root_as_message, Message, MessageHeader}; use arrow_format::ipc::Schema as ArrowSchema; -use arrow2::{ - datatypes::*, io::flight::serialize_schema_to_info, io::ipc, record_batch::RecordBatch, -}; +use arrow2::{datatypes::*, io::flight::serialize_schema_to_info, io::ipc}; use futures::{channel::mpsc, sink::SinkExt, Stream, StreamExt}; use tokio::sync::Mutex; @@ -62,7 +62,7 @@ pub async fn scenario_setup(port: &str) -> Result { struct IntegrationDataset { schema: Schema, ipc_schema: IpcSchema, - chunks: Vec, + chunks: Vec>>, } #[derive(Clone, Default)] @@ -173,7 +173,7 @@ impl FlightService for FlightServiceImpl { let endpoint = self.endpoint_from_path(&path[0]); - let total_records: usize = flight.chunks.iter().map(|chunk| chunk.num_rows()).sum(); + let total_records: usize = flight.chunks.iter().map(|chunk| chunk.len()).sum(); let schema = serialize_schema_to_info(&flight.schema, &flight.ipc_schema.fields) .expect( @@ -218,7 +218,6 @@ impl FlightService for FlightServiceImpl { let (schema, ipc_schema) = deserialize_schemas(&flight_data.data_header) .map_err(|e| Status::invalid_argument(format!("Invalid schema: {:?}", e)))?; - let schema_ref = Arc::new(schema.clone()); let (response_tx, response_rx) = mpsc::channel(10); @@ -228,11 +227,10 @@ impl FlightService for FlightServiceImpl { let mut error_tx = response_tx.clone(); if let Err(e) = save_uploaded_chunks( uploaded_chunks, - schema_ref, + schema, ipc_schema, input_stream, response_tx, - schema, key, ) .await @@ -280,10 +278,10 @@ async fn send_app_metadata( async fn record_batch_from_message( message: Message<'_>, data_body: &[u8], - schema_ref: Arc, + fields: &[Field], ipc_schema: &IpcSchema, dictionaries: &mut Dictionaries, -) -> Result { +) -> Result>, Status> { let ipc_batch = message .header_as_record_batch() .ok_or_else(|| Status::internal("Could not parse message header as record batch"))?; @@ -292,7 +290,7 @@ async fn record_batch_from_message( let arrow_batch_result = ipc::read::read_record_batch( ipc_batch, - schema_ref, + fields, ipc_schema, None, dictionaries, @@ -302,7 +300,7 @@ async fn record_batch_from_message( ); arrow_batch_result - .map_err(|e| Status::internal(format!("Could not convert to RecordBatch: {:?}", e))) + .map_err(|e| Status::internal(format!("Could not convert to Chunk: {:?}", e))) } async fn dictionary_from_message( @@ -326,11 +324,10 @@ async fn dictionary_from_message( async fn save_uploaded_chunks( uploaded_chunks: Arc>>, - schema_ref: Arc, + schema: Schema, ipc_schema: IpcSchema, mut input_stream: Streaming, mut response_tx: mpsc::Sender>, - schema: Schema, key: String, ) -> Result<(), Status> { let mut chunks = vec![]; @@ -354,7 +351,7 @@ async fn save_uploaded_chunks( let batch = record_batch_from_message( message, &data.data_body, - schema_ref.clone(), + schema.fields(), &ipc_schema, &mut dictionaries, ) @@ -366,7 +363,7 @@ async fn save_uploaded_chunks( dictionary_from_message( message, &data.data_body, - schema_ref.fields(), + schema.fields(), &ipc_schema, &mut dictionaries, ) diff --git a/integration-testing/src/lib.rs b/integration-testing/src/lib.rs index 6da3ff17081..0fa3f9e1e0b 100644 --- a/integration-testing/src/lib.rs +++ b/integration-testing/src/lib.rs @@ -17,17 +17,19 @@ //! Common code used in the integration test binaries +use arrow2::array::Array; use arrow2::io::ipc::IpcField; use serde_json::Value; +use arrow2::chunk::Chunk; use arrow2::datatypes::*; use arrow2::error::Result; use arrow2::io::json_integration::{read, ArrowJsonBatch, ArrowJsonDictionaryBatch}; -use arrow2::record_batch::RecordBatch; use std::collections::HashMap; use std::fs::File; use std::io::BufReader; +use std::sync::Arc; /// The expected username for the basic auth integration test. pub const AUTH_USERNAME: &str = "arrow"; @@ -43,7 +45,7 @@ pub struct ArrowFile { // we can evolve this into a concrete Arrow type // this is temporarily not being read from pub _dictionaries: HashMap, - pub batches: Vec, + pub batches: Vec>>, } pub fn read_json_file(json_name: &str) -> Result { @@ -66,12 +68,15 @@ pub fn read_json_file(json_name: &str) -> Result { } } - let mut batches = vec![]; - for b in arrow_json["batches"].as_array().unwrap() { - let json_batch: ArrowJsonBatch = serde_json::from_value(b.clone()).unwrap(); - let batch = read::to_record_batch(&schema, &fields, &json_batch, &dictionaries)?; - batches.push(batch); - } + let batches = arrow_json["batches"] + .as_array() + .unwrap() + .iter() + .map(|b| { + let json_batch: ArrowJsonBatch = serde_json::from_value(b.clone()).unwrap(); + read::deserialize_chunk(&schema, &fields, &json_batch, &dictionaries) + }) + .collect::>()?; Ok(ArrowFile { schema, fields, diff --git a/src/array/dictionary/mutable.rs b/src/array/dictionary/mutable.rs index bf152bd693b..c322ed63d49 100644 --- a/src/array/dictionary/mutable.rs +++ b/src/array/dictionary/mutable.rs @@ -101,12 +101,18 @@ impl MutableDictionaryArray { &self.values } - /// converts itself into `Arc` + /// converts itself into [`Arc`] pub fn into_arc(self) -> Arc { let a: DictionaryArray = self.into(); Arc::new(a) } + /// converts itself into [`Box`] + pub fn into_box(self) -> Box { + let a: DictionaryArray = self.into(); + Box::new(a) + } + /// Shrinks the capacity of the [`MutableDictionaryArray`] to fit its current length. pub fn shrink_to_fit(&mut self) { self.values.shrink_to_fit(); diff --git a/src/array/list/mutable.rs b/src/array/list/mutable.rs index 52bb9ec5704..23e4aae5121 100644 --- a/src/array/list/mutable.rs +++ b/src/array/list/mutable.rs @@ -183,6 +183,12 @@ impl MutableListArray { Arc::new(a) } + /// converts itself into [`Box`] + pub fn into_box(self) -> Box { + let a: ListArray = self.into(); + Box::new(a) + } + /// Shrinks the capacity of the [`MutableListArray`] to fit its current length. pub fn shrink_to_fit(&mut self) { self.values.shrink_to_fit(); diff --git a/src/array/mod.rs b/src/array/mod.rs index 06dc571d427..b9a83d87225 100644 --- a/src/array/mod.rs +++ b/src/array/mod.rs @@ -338,6 +338,14 @@ pub fn clone(array: &dyn Array) -> Box { } } +// see https://users.rust-lang.org/t/generic-for-dyn-a-or-box-dyn-a-or-arc-dyn-a/69430/3 +// for details +impl<'a> AsRef<(dyn Array + 'a)> for dyn Array { + fn as_ref(&self) -> &(dyn Array + 'a) { + self + } +} + mod binary; mod boolean; mod dictionary; diff --git a/src/columns.rs b/src/chunk.rs similarity index 58% rename from src/columns.rs rename to src/chunk.rs index 653f3448f7c..9162a6251d2 100644 --- a/src/columns.rs +++ b/src/chunk.rs @@ -1,26 +1,25 @@ -//! Contains [`Columns`], a container [`Array`] where all arrays have the +//! Contains [`Chunk`], a container of [`Array`] where every array has the //! same length. -use std::sync::Arc; use crate::array::Array; use crate::error::{ArrowError, Result}; -use crate::record_batch::RecordBatch; -/// A vector of [`Array`] where every array has the same length. +/// A vector of trait objects of [`Array`] where every item has +/// the same length, [`Chunk::len`]. #[derive(Debug, Clone, PartialEq)] -pub struct Columns> { +pub struct Chunk> { arrays: Vec, } -impl> Columns { - /// Creates a new [`Columns`]. +impl> Chunk { + /// Creates a new [`Chunk`]. /// # Panic /// Iff the arrays do not have the same length pub fn new(arrays: Vec) -> Self { Self::try_new(arrays).unwrap() } - /// Creates a new [`Columns`]. + /// Creates a new [`Chunk`]. /// # Error /// Iff the arrays do not have the same length pub fn try_new(arrays: Vec) -> Result { @@ -32,19 +31,24 @@ impl> Columns { .any(|array| array.len() != len) { return Err(ArrowError::InvalidArgumentError( - "Columns require all its arrays to have an equal number of rows".to_string(), + "Chunk require all its arrays to have an equal number of rows".to_string(), )); } } Ok(Self { arrays }) } - /// returns the [`Array`]s in [`Columns`]. + /// returns the [`Array`]s in [`Chunk`] pub fn arrays(&self) -> &[A] { &self.arrays } - /// returns the length (number of rows) + /// returns the [`Array`]s in [`Chunk`] + pub fn columns(&self) -> &[A] { + &self.arrays + } + + /// returns the number of rows of every array pub fn len(&self) -> usize { self.arrays .first() @@ -52,20 +56,25 @@ impl> Columns { .unwrap_or_default() } - /// Consumes [`Columns`] into its underlying arrays. + /// returns whether the columns have any rows + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Consumes [`Chunk`] into its underlying arrays. /// The arrays are guaranteed to have the same length pub fn into_arrays(self) -> Vec { self.arrays } } -impl> From> for Vec { - fn from(c: Columns) -> Self { +impl> From> for Vec { + fn from(c: Chunk) -> Self { c.into_arrays() } } -impl> std::ops::Deref for Columns { +impl> std::ops::Deref for Chunk { type Target = [A]; #[inline] @@ -73,11 +82,3 @@ impl> std::ops::Deref for Columns { self.arrays() } } - -impl From for Columns> { - fn from(batch: RecordBatch) -> Self { - Self { - arrays: batch.into_inner().0, - } - } -} diff --git a/src/compute/filter.rs b/src/compute/filter.rs index af0a6ef8e04..4366da4f2d2 100644 --- a/src/compute/filter.rs +++ b/src/compute/filter.rs @@ -1,9 +1,9 @@ //! Contains operators to filter arrays such as [`filter`]. use crate::array::growable::{make_growable, Growable}; use crate::bitmap::{utils::SlicesIterator, Bitmap, MutableBitmap}; +use crate::chunk::Chunk; use crate::datatypes::DataType; use crate::error::Result; -use crate::record_batch::RecordBatch; use crate::{array::*, types::NativeType}; /// Function that can filter arbitrary arrays @@ -66,9 +66,7 @@ fn filter_growable<'a>(growable: &mut impl Growable<'a>, chunks: &[(usize, usize /// Returns a prepared function optimized to filter multiple arrays. /// Creating this function requires time, but using it is faster than [filter] when the -/// same filter needs to be applied to multiple arrays (e.g. a multi-column `RecordBatch`). -/// WARNING: the nulls of `filter` are ignored and the value on its slot is considered. -/// Therefore, it is considered undefined behavior to pass `filter` with null values. +/// same filter needs to be applied to multiple arrays (e.g. a multiple columns). pub fn build_filter(filter: &BooleanArray) -> Result { let iter = SlicesIterator::new(filter.values()); let filter_count = iter.slots(); @@ -154,27 +152,24 @@ pub fn filter(array: &dyn Array, filter: &BooleanArray) -> Result } } -/// Returns a new [RecordBatch] with arrays containing only values matching the filter. -/// WARNING: the nulls of `filter` are ignored and the value on its slot is considered. -/// Therefore, it is considered undefined behavior to pass `filter` with null values. -pub fn filter_record_batch( - record_batch: &RecordBatch, +/// Returns a new [Chunk] with arrays containing only values matching the filter. +/// This is a convenience function: filter multiple columns is embarassingly parallel. +pub fn filter_chunk>( + columns: &Chunk, filter_values: &BooleanArray, -) -> Result { - let num_colums = record_batch.columns().len(); +) -> Result>> { + let arrays = columns.arrays(); + + let num_colums = arrays.len(); let filtered_arrays = match num_colums { 1 => { - vec![filter(record_batch.columns()[0].as_ref(), filter_values)?.into()] + vec![filter(columns.arrays()[0].as_ref(), filter_values)?] } _ => { let filter = build_filter(filter_values)?; - record_batch - .columns() - .iter() - .map(|a| filter(a.as_ref()).into()) - .collect() + arrays.iter().map(|a| filter(a.as_ref())).collect() } }; - RecordBatch::try_new(record_batch.schema().clone(), filtered_arrays) + Chunk::try_new(filtered_arrays) } diff --git a/src/compute/merge_sort/mod.rs b/src/compute/merge_sort/mod.rs index 238e06d7447..c56a82d3e8d 100644 --- a/src/compute/merge_sort/mod.rs +++ b/src/compute/merge_sort/mod.rs @@ -24,7 +24,7 @@ //! //! The main advantage of merge-sort over `sort` is that it can be parallelized. //! For example, given a set of arrays `[a0, a1, a2, a3]` representing the same field, -//! e.g. over 4 `RecordBatch`es, they can be sorted in parallel as follows (pseudo-code): +//! e.g. over 4 batches of arrays, they can be sorted in parallel as follows (pseudo-code): //! //! ```rust,ignore //! // in parallel @@ -42,7 +42,7 @@ //! ``` //! //! A common operation in query engines is to merge multiple fields based on the -//! same sorting field (e.g. merge-sort multiple `RecordBatch`es). +//! same sorting field (e.g. merge-sort multiple batches of arrays). //! To perform this, use the same idea as above, but use `take_arrays` over //! each independent field (which can again be parallelized): //! @@ -209,7 +209,7 @@ pub fn slices(pairs: &[(&[&dyn Array], &SortOptions)]) -> Result /// recursively sort-merges multiple `slices` representing slices of sorted arrays according /// to a comparison function between those arrays. /// Note that `slices` is an array of arrays, `slices[i][j]`. The index `i` represents -/// the set of arrays `i` (schema-unaware version of a [`RecordBatch`]), while the index `j` represents +/// the set of arrays `i`, while the index `j` represents /// the array `j` within that set. /// Note that this does not split to the smallest element as arrays: the smallest unit is a `slice` fn recursive_merge_sort(slices: &[&[MergeSlice]], comparator: &Comparator) -> Vec { diff --git a/src/compute/sort/lex_sort.rs b/src/compute/sort/lex_sort.rs index cab5e3dcec5..695269c92d2 100644 --- a/src/compute/sort/lex_sort.rs +++ b/src/compute/sort/lex_sort.rs @@ -39,7 +39,7 @@ pub struct SortColumn<'a> { /// let int64 = Int64Array::from(&[None, Some(-2), Some(89), Some(-64), Some(101)]); /// let utf8 = Utf8Array::::from(&vec![Some("hello"), Some("world"), Some(","), Some("foobar"), Some("!")]); /// -/// let sorted_columns = lexsort::(&vec![ +/// let sorted_chunk = lexsort::(&vec![ /// SortColumn { /// values: &int64, /// options: None, @@ -53,7 +53,7 @@ pub struct SortColumn<'a> { /// }, /// ], None).unwrap(); /// -/// let sorted = sorted_columns[0].as_any().downcast_ref::().unwrap(); +/// let sorted = sorted_chunk[0].as_any().downcast_ref::().unwrap(); /// assert_eq!(sorted.value(1), -64); /// assert!(sorted.is_null(0)); /// ``` diff --git a/src/datatypes/mod.rs b/src/datatypes/mod.rs index 8d9de83d583..886921d05f2 100644 --- a/src/datatypes/mod.rs +++ b/src/datatypes/mod.rs @@ -207,30 +207,6 @@ pub enum IntervalUnit { } impl DataType { - /// Compares the datatype with another, ignoring nested field names - /// and metadata. - pub(crate) fn equals_datatype(&self, other: &DataType) -> bool { - match (&self, other) { - (DataType::List(a), DataType::List(b)) - | (DataType::LargeList(a), DataType::LargeList(b)) => { - a.is_nullable() == b.is_nullable() && a.data_type().equals_datatype(b.data_type()) - } - (DataType::FixedSizeList(a, a_size), DataType::FixedSizeList(b, b_size)) => { - a_size == b_size - && a.is_nullable() == b.is_nullable() - && a.data_type().equals_datatype(b.data_type()) - } - (DataType::Struct(a), DataType::Struct(b)) => { - a.len() == b.len() - && a.iter().zip(b).all(|(a, b)| { - a.is_nullable() == b.is_nullable() - && a.data_type().equals_datatype(b.data_type()) - }) - } - _ => self == other, - } - } - /// the [`PhysicalType`] of this [`DataType`]. pub fn to_physical_type(&self) -> PhysicalType { use DataType::*; diff --git a/src/doc/lib.md b/src/doc/lib.md index b9879f60879..24964aa4300 100644 --- a/src/doc/lib.md +++ b/src/doc/lib.md @@ -12,10 +12,11 @@ Below is an example of some of the things you can do with it: use std::sync::Arc; use arrow2::array::*; +use arrow2::datatypes::{Field, DataType, Schema}; use arrow2::compute::arithmetics; use arrow2::error::Result; use arrow2::io::parquet::write::*; -use arrow2::record_batch::RecordBatch; +use arrow2::chunk::Chunk; fn main() -> Result<()> { // declare arrays @@ -26,16 +27,19 @@ fn main() -> Result<()> { let c = arithmetics::basic::mul_scalar(&a, &2); assert_eq!(c, b); - // declare records - let batch = RecordBatch::try_from_iter([ - ("c1", Arc::new(a) as Arc), - ("c2", Arc::new(b) as Arc), - ])?; - // with metadata - println!("{:?}", batch.schema()); + // declare a schema with fields + let schema = Schema::new(vec![ + Field::new("c1", DataType::Int32, true), + Field::new("c2", DataType::Int32, true), + ]); + + // declare chunk + let chunk = Chunk::new(vec![ + Arc::new(a) as Arc, + Arc::new(b) as Arc, + ]); // write to parquet (probably the fastest implementation of writing to parquet out there) - let schema = batch.schema().clone(); let options = WriteOptions { write_statistics: true, @@ -44,7 +48,7 @@ fn main() -> Result<()> { }; let row_groups = RowGroupIterator::try_new( - vec![Ok(batch)].into_iter(), + vec![Ok(chunk)].into_iter(), &schema, options, vec![Encoding::Plain, Encoding::Plain], diff --git a/src/io/avro/read/deserialize.rs b/src/io/avro/read/deserialize.rs index dda3fb4cb7e..f3b37ec1546 100644 --- a/src/io/avro/read/deserialize.rs +++ b/src/io/avro/read/deserialize.rs @@ -4,10 +4,10 @@ use std::sync::Arc; use avro_schema::{Enum, Schema as AvroSchema}; use crate::array::*; +use crate::chunk::Chunk; use crate::datatypes::*; use crate::error::ArrowError; use crate::error::Result; -use crate::record_batch::RecordBatch; use crate::types::months_days_ns; use super::super::Block; @@ -242,18 +242,17 @@ fn deserialize_value<'a>( Ok(block) } -/// Deserializes a [`Block`] into a [`RecordBatch`]. +/// Deserializes a [`Block`] into [`Chunk`]. pub fn deserialize( block: &Block, - schema: Arc, + fields: &[Field], avro_schemas: &[AvroSchema], -) -> Result { +) -> Result>> { let rows = block.number_of_rows; let mut block = block.data.as_ref(); // create mutables, one per field - let mut arrays: Vec> = schema - .fields() + let mut arrays: Vec> = fields .iter() .zip(avro_schemas.iter()) .map(|(field, avro_schema)| { @@ -266,13 +265,11 @@ pub fn deserialize( for _ in 0..rows { for ((array, field), avro_field) in arrays .iter_mut() - .zip(schema.fields().iter()) + .zip(fields.iter()) .zip(avro_schemas.iter()) { block = deserialize_item(array.as_mut(), field.is_nullable(), avro_field, block)? } } - let columns = arrays.iter_mut().map(|array| array.as_arc()).collect(); - - RecordBatch::try_new(schema, columns) + Chunk::try_new(arrays.iter_mut().map(|array| array.as_arc()).collect()) } diff --git a/src/io/avro/read/mod.rs b/src/io/avro/read/mod.rs index b203884de18..b48302d7b27 100644 --- a/src/io/avro/read/mod.rs +++ b/src/io/avro/read/mod.rs @@ -19,9 +19,10 @@ mod util; pub(super) use header::deserialize_header; pub(super) use schema::convert_schema; -use crate::datatypes::Schema; +use crate::array::Array; +use crate::chunk::Chunk; +use crate::datatypes::{Field, Schema}; use crate::error::Result; -use crate::record_batch::RecordBatch; use super::Compression; @@ -42,20 +43,20 @@ pub fn read_metadata( Ok((avro_schema, schema, codec, marker)) } -/// Single threaded, blocking reader of Avro; [`Iterator`] of [`RecordBatch`]es. +/// Single threaded, blocking reader of Avro; [`Iterator`] of [`Chunk`]. pub struct Reader { iter: Decompressor, - schema: Arc, avro_schemas: Vec, + fields: Vec, } impl Reader { /// Creates a new [`Reader`]. - pub fn new(iter: Decompressor, avro_schemas: Vec, schema: Arc) -> Self { + pub fn new(iter: Decompressor, avro_schemas: Vec, fields: Vec) -> Self { Self { iter, avro_schemas, - schema, + fields, } } @@ -66,15 +67,15 @@ impl Reader { } impl Iterator for Reader { - type Item = Result; + type Item = Result>>; fn next(&mut self) -> Option { - let schema = self.schema.clone(); + let fields = &self.fields[..]; let avro_schemas = &self.avro_schemas; self.iter .next() .transpose() - .map(|maybe_block| deserialize(maybe_block?, schema, avro_schemas)) + .map(|maybe_block| deserialize(maybe_block?, fields, avro_schemas)) } } diff --git a/src/io/csv/read/deserialize.rs b/src/io/csv/read/deserialize.rs index 66d77a907c3..2a86e97c167 100644 --- a/src/io/csv/read/deserialize.rs +++ b/src/io/csv/read/deserialize.rs @@ -4,9 +4,9 @@ use csv::ByteRecord; use crate::{ array::Array, + chunk::Chunk, datatypes::{DataType, Field}, error::Result, - record_batch::RecordBatch, }; use super::super::read_utils::{ @@ -31,7 +31,7 @@ pub fn deserialize_column( deserialize_column_gen(rows, column, datatype, line_number) } -/// Deserializes rows [`ByteRecord`] into a [`RecordBatch`]. +/// Deserializes rows [`ByteRecord`] into a [`Chunk`]. /// Note that this is a convenience function: column deserialization /// is trivially parallelizable (e.g. rayon). pub fn deserialize_batch( @@ -40,7 +40,7 @@ pub fn deserialize_batch( projection: Option<&[usize]>, line_number: usize, deserialize_column: F, -) -> Result +) -> Result>> where F: Fn(&[ByteRecord], usize, DataType, usize) -> Result>, { diff --git a/src/io/csv/read_async/deserialize.rs b/src/io/csv/read_async/deserialize.rs index aa4ce96097b..be64b7398b3 100644 --- a/src/io/csv/read_async/deserialize.rs +++ b/src/io/csv/read_async/deserialize.rs @@ -4,9 +4,9 @@ use csv_async::ByteRecord; use crate::{ array::Array, + chunk::Chunk, datatypes::{DataType, Field}, error::Result, - record_batch::RecordBatch, }; use super::super::read_utils::{ @@ -31,7 +31,7 @@ pub fn deserialize_column( deserialize_column_gen(rows, column, datatype, line_number) } -/// Deserializes rows [`ByteRecord`] into a [`RecordBatch`]. +/// Deserializes rows [`ByteRecord`] into [`Chunk`]. /// Note that this is a convenience function: column deserialization /// is trivially parallelizable (e.g. rayon). pub fn deserialize_batch( @@ -40,7 +40,7 @@ pub fn deserialize_batch( projection: Option<&[usize]>, line_number: usize, deserialize_column: F, -) -> Result +) -> Result>> where F: Fn(&[ByteRecord], usize, DataType, usize) -> Result>, { diff --git a/src/io/csv/read_utils.rs b/src/io/csv/read_utils.rs index c414f22e355..5ecdc0e8d5f 100644 --- a/src/io/csv/read_utils.rs +++ b/src/io/csv/read_utils.rs @@ -11,9 +11,9 @@ pub(crate) trait ByteRecordGeneric { use crate::{ array::*, + chunk::Chunk, datatypes::*, error::{ArrowError, Result}, - record_batch::RecordBatch, temporal_conversions, types::NativeType, }; @@ -257,16 +257,16 @@ pub(crate) fn deserialize_column( }) } -/// Deserializes rows [`ByteRecord`] into a [`RecordBatch`]. +/// Deserializes rows [`ByteRecord`] into [`Chunk`]. /// Note that this is a convenience function: column deserialization -/// is trivially parallelizable (e.g. rayon). +/// is embarassingly parallel (e.g. rayon). pub(crate) fn deserialize_batch( rows: &[B], fields: &[Field], projection: Option<&[usize]>, line_number: usize, deserialize_column: F, -) -> Result +) -> Result>> where F: Fn(&[B], usize, DataType, usize) -> Result>, { @@ -274,15 +274,12 @@ where Some(v) => v.to_vec(), None => fields.iter().enumerate().map(|(i, _)| i).collect(), }; - let projected_fields: Vec = projection.iter().map(|i| fields[*i].clone()).collect(); - - let schema = Arc::new(Schema::new(projected_fields)); if rows.is_empty() { - return Ok(RecordBatch::new_empty(schema)); + return Ok(Chunk::new(vec![])); } - let columns = projection + projection .iter() .map(|column| { let column = *column; @@ -290,7 +287,6 @@ where let data_type = field.data_type(); deserialize_column(rows, column, data_type.clone(), line_number) }) - .collect::>>()?; - - RecordBatch::try_new(schema, columns) + .collect::>>() + .and_then(Chunk::try_new) } diff --git a/src/io/csv/write/mod.rs b/src/io/csv/write/mod.rs index 5aa44c91d31..37d926f482e 100644 --- a/src/io/csv/write/mod.rs +++ b/src/io/csv/write/mod.rs @@ -10,53 +10,58 @@ pub use csv::{ByteRecord, Writer, WriterBuilder}; pub use serialize::*; -use crate::record_batch::RecordBatch; -use crate::{datatypes::Schema, error::Result}; +use crate::array::Array; +use crate::chunk::Chunk; +use crate::error::Result; -/// Creates serializers that iterate over each column of `batch` and serialize each item according +/// Creates serializers that iterate over each column that serializes each item according /// to `options`. -fn new_serializers<'a>( - batch: &'a RecordBatch, +fn new_serializers<'a, A: AsRef>( + columns: &'a [A], options: &'a SerializeOptions, ) -> Result + 'a>>> { - batch - .columns() + columns .iter() .map(|column| new_serializer(column.as_ref(), options)) .collect() } -/// Serializes a [`RecordBatch`] as vector of `ByteRecord`. -/// The vector is guaranteed to have `batch.num_rows()` entries. -/// Each `ByteRecord` is guaranteed to have `batch.num_columns()` fields. -pub fn serialize(batch: &RecordBatch, options: &SerializeOptions) -> Result> { - let mut serializers = new_serializers(batch, options)?; +/// Serializes [`Chunk`] to a vector of `ByteRecord`. +/// The vector is guaranteed to have `columns.len()` entries. +/// Each `ByteRecord` is guaranteed to have `columns.array().len()` fields. +pub fn serialize>( + columns: &Chunk, + options: &SerializeOptions, +) -> Result> { + let mut serializers = new_serializers(columns, options)?; - let mut records = vec![ByteRecord::with_capacity(0, batch.num_columns()); batch.num_rows()]; + let rows = columns.len(); + let mut records = vec![ByteRecord::with_capacity(0, columns.arrays().len()); rows]; records.iter_mut().for_each(|record| { serializers .iter_mut() - // `unwrap` is infalible because `array.len()` equals `num_rows` on a `RecordBatch` + // `unwrap` is infalible because `array.len()` equals `len` in `Chunk::len` .for_each(|iter| record.push_field(iter.next().unwrap())); }); Ok(records) } -/// Writes the data in a `RecordBatch` to `writer` according to the serialization options `options`. -pub fn write_batch( +/// Writes [`Chunk`] to `writer` according to the serialization options `options`. +pub fn write_chunk>( writer: &mut Writer, - batch: &RecordBatch, + columns: &Chunk, options: &SerializeOptions, ) -> Result<()> { - let mut serializers = new_serializers(batch, options)?; + let mut serializers = new_serializers(columns.arrays(), options)?; - let mut record = ByteRecord::with_capacity(0, batch.num_columns()); + let rows = columns.len(); + let mut record = ByteRecord::with_capacity(0, columns.arrays().len()); // this is where the (expensive) transposition happens: the outer loop is on rows, the inner on columns - (0..batch.num_rows()).try_for_each(|_| { + (0..rows).try_for_each(|_| { serializers .iter_mut() - // `unwrap` is infalible because `array.len()` equals `num_rows` on a `RecordBatch` + // `unwrap` is infalible because `array.len()` equals `Chunk::len` .for_each(|iter| record.push_field(iter.next().unwrap())); writer.write_byte_record(&record)?; record.clear(); @@ -65,13 +70,11 @@ pub fn write_batch( Ok(()) } -/// Writes a header to `writer` according to `schema` -pub fn write_header(writer: &mut Writer, schema: &Schema) -> Result<()> { - let fields = schema - .fields() - .iter() - .map(|field| field.name().to_string()) - .collect::>(); - writer.write_record(&fields)?; +/// Writes a CSV header to `writer` +pub fn write_header(writer: &mut Writer, names: &[T]) -> Result<()> +where + T: AsRef, +{ + writer.write_record(names.iter().map(|x| x.as_ref().as_bytes()))?; Ok(()) } diff --git a/src/io/flight/mod.rs b/src/io/flight/mod.rs index 31d794d7196..d1e35d32920 100644 --- a/src/io/flight/mod.rs +++ b/src/io/flight/mod.rs @@ -4,28 +4,28 @@ use arrow_format::flight::data::{FlightData, SchemaResult}; use arrow_format::ipc; use crate::{ + array::Array, + chunk::Chunk, datatypes::*, error::{ArrowError, Result}, io::ipc::read, io::ipc::write, - io::ipc::write::common::{encode_columns, DictionaryTracker, EncodedData, WriteOptions}, - record_batch::RecordBatch, + io::ipc::write::common::{encode_chunk, DictionaryTracker, EncodedData, WriteOptions}, }; use super::ipc::{IpcField, IpcSchema}; -/// Serializes a [`RecordBatch`] to a vector of [`FlightData`] representing the serialized dictionaries +/// Serializes [`Chunk`] to a vector of [`FlightData`] representing the serialized dictionaries /// and a [`FlightData`] representing the batch. pub fn serialize_batch( - batch: &RecordBatch, + columns: &Chunk>, fields: &[IpcField], options: &WriteOptions, ) -> (Vec, FlightData) { let mut dictionary_tracker = DictionaryTracker::new(false); - let columns = batch.clone().into(); let (encoded_dictionaries, encoded_batch) = - encode_columns(&columns, fields, &mut dictionary_tracker, options) + encode_chunk(columns, fields, &mut dictionary_tracker, options) .expect("DictionaryTracker configured above to not error on replacement"); let flight_dictionaries = encoded_dictionaries.into_iter().map(Into::into).collect(); @@ -99,13 +99,13 @@ pub fn deserialize_schemas(bytes: &[u8]) -> Result<(Schema, IpcSchema)> { } } -/// Deserializes [`FlightData`] to a [`RecordBatch`]. +/// Deserializes [`FlightData`] to [`Chunk`]. pub fn deserialize_batch( data: &FlightData, - schema: Arc, + fields: &[Field], ipc_schema: &IpcSchema, dictionaries: &read::Dictionaries, -) -> Result { +) -> Result>> { // check that the data_header is a record batch message let message = ipc::Message::root_as_message(&data.data_header[..]).map_err(|err| { ArrowError::OutOfSpec(format!("Unable to get root as message: {:?}", err)) @@ -123,7 +123,7 @@ pub fn deserialize_batch( .map(|batch| { read::read_record_batch( batch, - schema.clone(), + fields, ipc_schema, None, dictionaries, diff --git a/src/io/ipc/mod.rs b/src/io/ipc/mod.rs index dd4b93590af..37bcec2ccc8 100644 --- a/src/io/ipc/mod.rs +++ b/src/io/ipc/mod.rs @@ -10,9 +10,8 @@ //! using the more integrated approach that is exposed in this module. //! //! [Arrow's IPC protocol](https://arrow.apache.org/docs/format/Columnar.html#serialization-and-interprocess-communication-ipc) -//! allows only [`RecordBatch`](crate::record_batch::RecordBatch)es or -//! [`DictionaryBatch`](gen::Message::DictionaryBatch) to be passed -//! around due to its reliance on a pre-defined data scheme. This limitation +//! allows only batch or dictionary columns to be passed +//! around due to its reliance on a pre-defined data scheme. This constraint //! provides a large performance gain because serialized data will always have a //! known structutre, i.e. the same fields and datatypes, with the only variance //! being the number of rows and the actual data inside the Batch. This dramatically @@ -34,8 +33,8 @@ //! # use std::fs::File; //! # use std::sync::Arc; //! # use arrow2::datatypes::{Field, Schema, DataType}; -//! # use arrow2::array::Int32Array; -//! # use arrow2::record_batch::RecordBatch; +//! # use arrow2::array::{Int32Array, Array}; +//! # use arrow2::chunk::Chunk; //! # use arrow2::error::ArrowError; //! // Setup the writer //! let path = "example.arrow".to_string(); @@ -49,14 +48,13 @@ //! // Setup the data //! let x_data = Int32Array::from_slice([-1i32, 1]); //! let y_data = Int32Array::from_slice([1i32, -1]); -//! let batch = RecordBatch::try_new( -//! Arc::new(schema), -//! vec![Arc::new(x_data), Arc::new(y_data)] -//! )?; +//! let chunk = Chunk::try_new( +//! vec![Arc::new(x_data) as Arc, Arc::new(y_data)] +//! )?; //! //! // Write the messages and finalize the stream //! for _ in 0..5 { -//! writer.write(&batch, None); +//! writer.write(&chunk, None); //! } //! writer.finish(); //! diff --git a/src/io/ipc/read/common.rs b/src/io/ipc/read/common.rs index 623b4a9219a..24b023cfeec 100644 --- a/src/io/ipc/read/common.rs +++ b/src/io/ipc/read/common.rs @@ -1,4 +1,4 @@ -use std::collections::{HashMap, VecDeque}; +use std::collections::VecDeque; use std::io::{Read, Seek}; use std::sync::Arc; @@ -6,10 +6,10 @@ use arrow_format::ipc; use arrow_format::ipc::Schema::MetadataVersion; use crate::array::*; -use crate::datatypes::{DataType, Field, Schema}; +use crate::chunk::Chunk; +use crate::datatypes::{DataType, Field}; use crate::error::{ArrowError, Result}; use crate::io::ipc::{IpcField, IpcSchema}; -use crate::record_batch::RecordBatch; use super::deserialize::{read, skip}; use super::Dictionaries; @@ -80,18 +80,18 @@ impl<'a, A, I: Iterator> Iterator for ProjectionIter<'a, A, I> { #[allow(clippy::too_many_arguments)] pub fn read_record_batch( batch: ipc::Message::RecordBatch, - schema: Arc, + fields: &[Field], ipc_schema: &IpcSchema, - projection: Option<(&[usize], Arc)>, + projection: Option<&[usize]>, dictionaries: &Dictionaries, version: MetadataVersion, reader: &mut R, block_offset: u64, -) -> Result { - assert_eq!(schema.fields().len(), ipc_schema.fields.len()); +) -> Result>> { + assert_eq!(fields.len(), ipc_schema.fields.len()); let buffers = batch .buffers() - .ok_or_else(|| ArrowError::oos("IPC RecordBatch must contain buffers"))?; + .ok_or_else(|| ArrowError::oos("Unable to get buffers from IPC RecordBatch"))?; let mut buffers: VecDeque<&ipc::Schema::Buffer> = buffers.iter().collect(); let field_nodes = batch @@ -99,15 +99,11 @@ pub fn read_record_batch( .ok_or_else(|| ArrowError::oos("IPC RecordBatch must contain field nodes"))?; let mut field_nodes = field_nodes.iter().collect::>(); - let (schema, columns) = if let Some(projection) = projection { - let projected_schema = projection.1.clone(); - - let projection = ProjectionIter::new( - projection.0, - schema.fields().iter().zip(ipc_schema.fields.iter()), - ); + let columns = if let Some(projection) = projection { + let projection = + ProjectionIter::new(projection, fields.iter().zip(ipc_schema.fields.iter())); - let arrays = projection + projection .map(|maybe_field| match maybe_field { ProjectionResult::Selected((field, ipc_field)) => Ok(Some(read( &mut field_nodes, @@ -128,11 +124,9 @@ pub fn read_record_batch( }) .map(|x| x.transpose()) .flatten() - .collect::>>()?; - (projected_schema, arrays) + .collect::>>()? } else { - let arrays = schema - .fields() + fields .iter() .zip(ipc_schema.fields.iter()) .map(|(field, ipc_field)| { @@ -149,10 +143,9 @@ pub fn read_record_batch( version, ) }) - .collect::>>()?; - (schema.clone(), arrays) + .collect::>>()? }; - RecordBatch::try_new(schema, columns) + Chunk::try_new(columns) } fn find_first_dict_field_d<'a>( @@ -233,21 +226,16 @@ pub fn read_dictionary( let dictionary_values: ArrayRef = match first_field.data_type() { DataType::Dictionary(_, ref value_type, _) => { // Make a fake schema for the dictionary batch. - let schema = Arc::new(Schema { - fields: vec![Field::new("", value_type.as_ref().clone(), false)], - metadata: HashMap::new(), - }); + let fields = vec![Field::new("", value_type.as_ref().clone(), false)]; let ipc_schema = IpcSchema { fields: vec![first_ipc_field.clone()], is_little_endian: ipc_schema.is_little_endian, }; - assert_eq!(ipc_schema.fields.len(), schema.fields().len()); - // Read a single column - let record_batch = read_record_batch( + let columns = read_record_batch( batch .data() .ok_or_else(|| ArrowError::oos("The dictionary batch must have data."))?, - schema, + &fields, &ipc_schema, None, dictionaries, @@ -255,7 +243,8 @@ pub fn read_dictionary( reader, block_offset, )?; - Some(record_batch.column(0).clone()) + let mut arrays = columns.into_arrays(); + Some(arrays.pop().unwrap()) } _ => None, } diff --git a/src/io/ipc/read/reader.rs b/src/io/ipc/read/reader.rs index 59592a646e9..86bee1644b5 100644 --- a/src/io/ipc/read/reader.rs +++ b/src/io/ipc/read/reader.rs @@ -5,10 +5,11 @@ use arrow_format::ipc; use arrow_format::ipc::flatbuffers::VerifierOptions; use arrow_format::ipc::File::Block; +use crate::array::Array; +use crate::chunk::Chunk; use crate::datatypes::Schema; use crate::error::{ArrowError, Result}; use crate::io::ipc::IpcSchema; -use crate::record_batch::RecordBatch; use super::super::{ARROW_MAGIC, CONTINUATION_MARKER}; use super::common::*; @@ -18,7 +19,7 @@ use super::Dictionaries; #[derive(Debug, Clone)] pub struct FileMetadata { /// The schema that is read from the file footer - pub schema: Arc, + pub schema: Schema, /// The files' [`IpcSchema`] pub ipc_schema: IpcSchema, @@ -35,19 +36,12 @@ pub struct FileMetadata { version: ipc::Schema::MetadataVersion, } -impl FileMetadata { - /// Returns the schema. - pub fn schema(&self) -> &Arc { - &self.schema - } -} - /// Arrow File reader pub struct FileReader { reader: R, metadata: FileMetadata, current_block: usize, - projection: Option<(Vec, Arc)>, + projection: Option<(Vec, Schema)>, buffer: Vec, } @@ -167,7 +161,6 @@ pub fn read_file_metadata(reader: &mut R) -> Result( pub fn read_batch( reader: &mut R, metadata: &FileMetadata, - projection: Option<(&[usize], Arc)>, + projection: Option<&[usize]>, block: usize, block_data: &mut Vec, -) -> Result { +) -> Result>> { let block = metadata.blocks[block]; // read length @@ -236,7 +229,7 @@ pub fn read_batch( read_record_batch( batch, - metadata.schema.clone(), + metadata.schema.fields(), &metadata.ipc_schema, projection, &metadata.dictionaries, @@ -260,12 +253,12 @@ impl FileReader { }); } let projection = projection.map(|projection| { - let fields = metadata.schema().fields(); + let fields = metadata.schema.fields(); let fields = projection.iter().map(|x| fields[*x].clone()).collect(); - let schema = Arc::new(Schema { + let schema = Schema { fields, - metadata: metadata.schema().metadata().clone(), - }); + metadata: metadata.schema.metadata().clone(), + }; (projection, schema) }); Self { @@ -278,7 +271,7 @@ impl FileReader { } /// Return the schema of the file - pub fn schema(&self) -> &Arc { + pub fn schema(&self) -> &Schema { self.projection .as_ref() .map(|x| &x.1) @@ -297,7 +290,7 @@ impl FileReader { } impl Iterator for FileReader { - type Item = Result; + type Item = Result>>; fn next(&mut self) -> Option { // get current block @@ -307,9 +300,7 @@ impl Iterator for FileReader { Some(read_batch( &mut self.reader, &self.metadata, - self.projection - .as_ref() - .map(|x| (x.0.as_ref(), x.1.clone())), + self.projection.as_ref().map(|x| x.0.as_ref()), block, &mut self.buffer, )) diff --git a/src/io/ipc/read/stream.rs b/src/io/ipc/read/stream.rs index a975eb9d3aa..57058d30733 100644 --- a/src/io/ipc/read/stream.rs +++ b/src/io/ipc/read/stream.rs @@ -4,10 +4,11 @@ use std::sync::Arc; use arrow_format::ipc; use arrow_format::ipc::Schema::MetadataVersion; +use crate::array::Array; +use crate::chunk::Chunk; use crate::datatypes::Schema; use crate::error::{ArrowError, Result}; use crate::io::ipc::IpcSchema; -use crate::record_batch::RecordBatch; use super::super::CONTINUATION_MARKER; use super::common::*; @@ -17,7 +18,7 @@ use super::Dictionaries; #[derive(Debug, Clone)] pub struct StreamMetadata { /// The schema that is read from the stream's first message - pub schema: Arc, + pub schema: Schema, pub version: MetadataVersion, @@ -50,7 +51,6 @@ pub fn read_stream_metadata(reader: &mut R) -> Result { .header_as_schema() .ok_or_else(|| ArrowError::OutOfSpec("Unable to read IPC message as schema".to_string()))?; let (schema, ipc_schema) = fb_to_schema(ipc_schema)?; - let schema = Arc::new(schema); Ok(StreamMetadata { schema, @@ -64,15 +64,15 @@ pub fn read_stream_metadata(reader: &mut R) -> Result { /// A stream is an iterator, and an iterator returns `Option`. The `Item` /// type in the [`StreamReader`] case is `StreamState`, which means that an Arrow /// stream may yield one of three values: (1) `None`, which signals that the stream -/// is done; (2) `Some(StreamState::Some(RecordBatch))`, which signals that there was +/// is done; (2) [`StreamState::Some`], which signals that there was /// data waiting in the stream and we read it; and finally (3) -/// `Some(StreamState::Waiting)`, which means that the stream is still "live", it +/// [`Some(StreamState::Waiting)`], which means that the stream is still "live", it /// just doesn't hold any data right now. pub enum StreamState { /// A live stream without data Waiting, /// Next item in the stream - Some(RecordBatch), + Some(Chunk>), } impl StreamState { @@ -81,7 +81,7 @@ impl StreamState { /// # Panics /// /// If the `StreamState` was `Waiting`. - pub fn unwrap(self) -> RecordBatch { + pub fn unwrap(self) -> Chunk> { if let StreamState::Some(batch) = self { batch } else { @@ -155,7 +155,7 @@ fn read_next( read_record_batch( batch, - metadata.schema.clone(), + metadata.schema.fields(), &metadata.ipc_schema, None, dictionaries, @@ -184,7 +184,7 @@ fn read_next( 0, )?; - // read the next message until we encounter a RecordBatch + // read the next message until we encounter a RecordBatch message read_next(reader, metadata, dictionaries, message_buffer, data_buffer) } ipc::Message::MessageHeader::NONE => Ok(Some(StreamState::Waiting)), diff --git a/src/io/ipc/write/common.rs b/src/io/ipc/write/common.rs index 20890c93538..54af2ec1f7a 100644 --- a/src/io/ipc/write/common.rs +++ b/src/io/ipc/write/common.rs @@ -5,7 +5,7 @@ use arrow_format::ipc::flatbuffers::FlatBufferBuilder; use arrow_format::ipc::Message::CompressionType; use crate::array::*; -use crate::columns::Columns; +use crate::chunk::Chunk; use crate::datatypes::*; use crate::error::{ArrowError, Result}; use crate::io::ipc::endianess::is_native_little_endian; @@ -172,8 +172,8 @@ fn encode_dictionary( } } -pub fn encode_columns( - columns: &Columns>, +pub fn encode_chunk( + columns: &Chunk>, fields: &[IpcField], dictionary_tracker: &mut DictionaryTracker, options: &WriteOptions, @@ -195,9 +195,9 @@ pub fn encode_columns( Ok((encoded_dictionaries, encoded_message)) } -/// Write a `RecordBatch` into two sets of bytes, one for the header (ipc::Schema::Message) and the +/// Write [`Chunk`] into two sets of bytes, one for the header (ipc::Schema::Message) and the /// other for the batch's data -fn columns_to_bytes(columns: &Columns>, options: &WriteOptions) -> EncodedData { +fn columns_to_bytes(columns: &Chunk>, options: &WriteOptions) -> EncodedData { let mut fbb = FlatBufferBuilder::new(); let mut nodes: Vec = vec![]; diff --git a/src/io/ipc/write/stream.rs b/src/io/ipc/write/stream.rs index b19a76feeb8..da145d7f1c6 100644 --- a/src/io/ipc/write/stream.rs +++ b/src/io/ipc/write/stream.rs @@ -4,15 +4,17 @@ //! however the `FileWriter` expects a reader that supports `Seek`ing use std::io::Write; +use std::sync::Arc; use super::super::IpcField; -use super::common::{encode_columns, DictionaryTracker, EncodedData, WriteOptions}; +use super::common::{encode_chunk, DictionaryTracker, EncodedData, WriteOptions}; use super::common_sync::{write_continuation, write_message}; use super::schema_to_bytes; +use crate::array::Array; +use crate::chunk::Chunk; use crate::datatypes::*; use crate::error::{ArrowError, Result}; -use crate::record_batch::RecordBatch; /// Arrow stream writer /// @@ -52,8 +54,8 @@ impl StreamWriter { Ok(()) } - /// Writes [`RecordBatch`] to the stream - pub fn write(&mut self, batch: &RecordBatch, fields: &[IpcField]) -> Result<()> { + /// Writes [`Chunk`] to the stream + pub fn write(&mut self, columns: &Chunk>, fields: &[IpcField]) -> Result<()> { if self.finished { return Err(ArrowError::Io(std::io::Error::new( std::io::ErrorKind::UnexpectedEof, @@ -61,9 +63,8 @@ impl StreamWriter { ))); } - let columns = batch.clone().into(); - let (encoded_dictionaries, encoded_message) = encode_columns( - &columns, + let (encoded_dictionaries, encoded_message) = encode_chunk( + columns, fields, &mut self.dictionary_tracker, &self.write_options, diff --git a/src/io/ipc/write/stream_async.rs b/src/io/ipc/write/stream_async.rs index e84df52342c..fa665fa7c4b 100644 --- a/src/io/ipc/write/stream_async.rs +++ b/src/io/ipc/write/stream_async.rs @@ -1,16 +1,18 @@ //! `async` writing of arrow streams +use std::sync::Arc; use futures::AsyncWrite; use super::super::IpcField; pub use super::common::WriteOptions; -use super::common::{encode_columns, DictionaryTracker, EncodedData}; +use super::common::{encode_chunk, DictionaryTracker, EncodedData}; use super::common_async::{write_continuation, write_message}; use super::{default_ipc_fields, schema_to_bytes}; +use crate::array::Array; +use crate::chunk::Chunk; use crate::datatypes::*; use crate::error::{ArrowError, Result}; -use crate::record_batch::RecordBatch; /// An `async` writer to the Apache Arrow stream format. pub struct StreamWriter { @@ -53,10 +55,11 @@ impl StreamWriter { Ok(()) } - /// Writes [`RecordBatch`] to the stream + /// Writes [`Chunk`] to the stream pub async fn write( &mut self, - batch: &RecordBatch, + columns: &Chunk>, + schema: &Schema, ipc_fields: Option<&[IpcField]>, ) -> Result<()> { if self.finished { @@ -67,18 +70,16 @@ impl StreamWriter { } let (encoded_dictionaries, encoded_message) = if let Some(ipc_fields) = ipc_fields { - let columns = batch.clone().into(); - encode_columns( - &columns, + encode_chunk( + columns, ipc_fields, &mut self.dictionary_tracker, &self.write_options, )? } else { - let ipc_fields = default_ipc_fields(batch.schema().fields()); - let columns = batch.clone().into(); - encode_columns( - &columns, + let ipc_fields = default_ipc_fields(schema.fields()); + encode_chunk( + columns, &ipc_fields, &mut self.dictionary_tracker, &self.write_options, diff --git a/src/io/ipc/write/writer.rs b/src/io/ipc/write/writer.rs index 4591f9a3953..9f818b5f7f1 100644 --- a/src/io/ipc/write/writer.rs +++ b/src/io/ipc/write/writer.rs @@ -1,4 +1,4 @@ -use std::io::Write; +use std::{io::Write, sync::Arc}; use arrow_format::ipc; use arrow_format::ipc::flatbuffers::FlatBufferBuilder; @@ -6,14 +6,15 @@ use arrow_format::ipc::flatbuffers::FlatBufferBuilder; use super::{ super::IpcField, super::ARROW_MAGIC, - common::{encode_columns, DictionaryTracker, EncodedData, WriteOptions}, + common::{encode_chunk, DictionaryTracker, EncodedData, WriteOptions}, common_sync::{write_continuation, write_message}, default_ipc_fields, schema, schema_to_bytes, }; +use crate::array::Array; +use crate::chunk::Chunk; use crate::datatypes::*; use crate::error::{ArrowError, Result}; -use crate::record_batch::RecordBatch; /// Arrow file writer pub struct FileWriter { @@ -78,8 +79,12 @@ impl FileWriter { self.writer } - /// Writes [`RecordBatch`] to the file - pub fn write(&mut self, batch: &RecordBatch, ipc_fields: Option<&[IpcField]>) -> Result<()> { + /// Writes [`Chunk`] to the file + pub fn write( + &mut self, + columns: &Chunk>, + ipc_fields: Option<&[IpcField]>, + ) -> Result<()> { if self.finished { return Err(ArrowError::Io(std::io::Error::new( std::io::ErrorKind::UnexpectedEof, @@ -93,9 +98,8 @@ impl FileWriter { self.ipc_fields.as_ref() }; - let columns = batch.clone().into(); - let (encoded_dictionaries, encoded_message) = encode_columns( - &columns, + let (encoded_dictionaries, encoded_message) = encode_chunk( + columns, ipc_fields, &mut self.dictionary_tracker, &self.options, diff --git a/src/io/json/read/deserialize.rs b/src/io/json/read/deserialize.rs index a2a4f5fab86..d08b520fe80 100644 --- a/src/io/json/read/deserialize.rs +++ b/src/io/json/read/deserialize.rs @@ -7,13 +7,12 @@ use indexmap::map::IndexMap as HashMap; use num_traits::NumCast; use serde_json::Value; -use crate::datatypes::{Field, Schema}; -use crate::error::ArrowError; -use crate::record_batch::RecordBatch; use crate::{ array::*, bitmap::MutableBitmap, - datatypes::{DataType, IntervalUnit}, + chunk::Chunk, + datatypes::{DataType, Field, IntervalUnit}, + error::ArrowError, types::NativeType, }; @@ -252,13 +251,13 @@ fn _deserialize>(rows: &[A], data_type: DataType) -> Arc>( rows: &[A], - fields: Vec, -) -> Result { - let data_type = DataType::Struct(fields); + fields: &[Field], +) -> Result>, ArrowError> { + let data_type = DataType::Struct(fields.to_vec()); // convert rows to `Value` let rows = rows @@ -269,6 +268,6 @@ pub fn deserialize>( }) .collect::, ArrowError>>()?; - let (fields, columns, _) = deserialize_struct(&rows, data_type).into_data(); - RecordBatch::try_new(Arc::new(Schema::new(fields)), columns) + let (_, columns, _) = deserialize_struct(&rows, data_type).into_data(); + Ok(Chunk::new(columns)) } diff --git a/src/io/json/write/mod.rs b/src/io/json/write/mod.rs index 726825ed09f..fcf1c435148 100644 --- a/src/io/json/write/mod.rs +++ b/src/io/json/write/mod.rs @@ -6,8 +6,9 @@ pub use format::*; pub use serialize::serialize; use crate::{ + array::Array, + chunk::Chunk, error::{ArrowError, Result}, - record_batch::RecordBatch, }; /// Writes blocks of JSON-encoded data into `writer`, ensuring that the written @@ -29,27 +30,42 @@ where Ok(()) } -/// [`FallibleStreamingIterator`] that serializes a [`RecordBatch`] to bytes. +/// [`FallibleStreamingIterator`] that serializes a [`Chunk`] to bytes. /// Advancing it is CPU-bounded -pub struct Serializer>> { - iter: I, +pub struct Serializer +where + F: JsonFormat, + A: AsRef, + I: Iterator>>, +{ + batches: I, + names: Vec, buffer: Vec, format: F, } -impl>> Serializer { +impl Serializer +where + F: JsonFormat, + A: AsRef, + I: Iterator>>, +{ /// Creates a new [`Serializer`]. - pub fn new(iter: I, buffer: Vec, format: F) -> Self { + pub fn new(batches: I, names: Vec, buffer: Vec, format: F) -> Self { Self { - iter, + batches, + names, buffer, format, } } } -impl>> FallibleStreamingIterator - for Serializer +impl FallibleStreamingIterator for Serializer +where + F: JsonFormat, + A: AsRef, + I: Iterator>>, { type Item = [u8]; @@ -57,18 +73,11 @@ impl>> FallibleStreamingIt fn advance(&mut self) -> Result<()> { self.buffer.clear(); - self.iter + self.batches .next() - .map(|maybe_batch| { - maybe_batch.map(|batch| { - let names = batch - .schema() - .fields() - .iter() - .map(|f| f.name().as_str()) - .collect::>(); - serialize(&names, batch.columns(), self.format, &mut self.buffer) - }) + .map(|maybe_chunk| { + maybe_chunk + .map(|columns| serialize(&self.names, &columns, self.format, &mut self.buffer)) }) .transpose()?; Ok(()) diff --git a/src/io/json/write/serialize.rs b/src/io/json/write/serialize.rs index 769d1e28551..39af9e8871b 100644 --- a/src/io/json/write/serialize.rs +++ b/src/io/json/write/serialize.rs @@ -3,6 +3,7 @@ use serde_json::Value; use streaming_iterator::StreamingIterator; use crate::bitmap::utils::zip_validity; +use crate::chunk::Chunk; use crate::io::iterator::BufStreamingIterator; use crate::util::lexical_to_bytes_mut; use crate::{array::*, datatypes::DataType, types::NativeType}; @@ -82,7 +83,7 @@ fn struct_serializer<'a>( serializers .iter_mut() .zip(names) - // `unwrap` is infalible because `array.len()` equals `num_rows` on a `RecordBatch` + // `unwrap` is infalible because `array.len()` equals `len` on `Chunk` .for_each(|(iter, name)| { let item = iter.next().unwrap(); record.push((name, item)); @@ -198,15 +199,16 @@ fn serialize_item( /// Serializes a (name, array) to a valid JSON to `buffer` /// This is CPU-bounded -pub fn serialize(names: &[N], arrays: &[A], format: F, buffer: &mut Vec) +pub fn serialize(names: &[N], columns: &Chunk, format: F, buffer: &mut Vec) where N: AsRef, A: AsRef, F: JsonFormat, { - let num_rows = arrays[0].as_ref().len(); + let num_rows = columns.len(); - let mut serializers: Vec<_> = arrays + let mut serializers: Vec<_> = columns + .arrays() .iter() .map(|array| new_serializer(array.as_ref())) .collect(); @@ -217,7 +219,7 @@ where serializers .iter_mut() .zip(names.iter()) - // `unwrap` is infalible because `array.len()` equals `num_rows` on a `RecordBatch` + // `unwrap` is infalible because `array.len()` equals `len` on `Chunk` .for_each(|(iter, name)| { let item = iter.next().unwrap(); record.push((name.as_ref(), item)); diff --git a/src/io/json_integration/read/array.rs b/src/io/json_integration/read/array.rs index 6579d7f815c..78dcd863823 100644 --- a/src/io/json_integration/read/array.rs +++ b/src/io/json_integration/read/array.rs @@ -7,10 +7,10 @@ use crate::{ array::*, bitmap::{Bitmap, MutableBitmap}, buffer::Buffer, + chunk::Chunk, datatypes::{DataType, PhysicalType, PrimitiveType, Schema}, error::{ArrowError, Result}, io::ipc::IpcField, - record_batch::RecordBatch, types::{days_ms, months_days_ns, NativeType}, }; @@ -410,13 +410,13 @@ pub fn to_array( } } -pub fn to_record_batch( +pub fn deserialize_chunk( schema: &Schema, ipc_fields: &[IpcField], json_batch: &ArrowJsonBatch, json_dictionaries: &HashMap, -) -> Result { - let columns = schema +) -> Result>> { + let arrays = schema .fields() .iter() .zip(&json_batch.columns) @@ -429,7 +429,7 @@ pub fn to_record_batch( json_dictionaries, ) }) - .collect::>>()?; + .collect::>()?; - RecordBatch::try_new(Arc::new(schema.clone()), columns) + Chunk::try_new(arrays) } diff --git a/src/io/json_integration/write/array.rs b/src/io/json_integration/write/array.rs index ea0bc1c7669..0d317b9a7e4 100644 --- a/src/io/json_integration/write/array.rs +++ b/src/io/json_integration/write/array.rs @@ -1,19 +1,27 @@ -use crate::record_batch::RecordBatch; -use crate::{array::PrimitiveArray, datatypes::DataType}; +use std::sync::Arc; + +use crate::{ + array::{Array, PrimitiveArray}, + chunk::Chunk, + datatypes::DataType, +}; use super::super::{ArrowJsonBatch, ArrowJsonColumn}; -/// Serializes a [`RecordBatch`] to [`ArrowJsonBatch`]. -pub fn from_record_batch(batch: &RecordBatch) -> ArrowJsonBatch { - let mut json_batch = ArrowJsonBatch { - count: batch.num_rows(), - columns: Vec::with_capacity(batch.num_columns()), - }; +/// Serializes a [`Chunk`] to [`ArrowJsonBatch`]. +pub fn serialize_chunk( + columns: &Chunk>, + names: &[A], +) -> ArrowJsonBatch { + let count = columns.len(); - for (col, field) in batch.columns().iter().zip(batch.schema().fields.iter()) { - let json_col = match field.data_type() { + let columns = columns + .arrays() + .iter() + .zip(names.iter()) + .map(|(array, name)| match array.data_type() { DataType::Int8 => { - let array = col.as_any().downcast_ref::>().unwrap(); + let array = array.as_any().downcast_ref::>().unwrap(); let (validity, data) = array .iter() @@ -21,8 +29,8 @@ pub fn from_record_batch(batch: &RecordBatch) -> ArrowJsonBatch { .unzip(); ArrowJsonColumn { - name: field.name().clone(), - count: col.len(), + name: name.to_string(), + count: array.len(), validity: Some(validity), data: Some(data), offset: None, @@ -31,18 +39,16 @@ pub fn from_record_batch(batch: &RecordBatch) -> ArrowJsonBatch { } } _ => ArrowJsonColumn { - name: field.name().clone(), - count: col.len(), + name: name.to_string(), + count: array.len(), validity: None, data: None, offset: None, type_id: None, children: None, }, - }; - - json_batch.columns.push(json_col); - } + }) + .collect(); - json_batch + ArrowJsonBatch { count, columns } } diff --git a/src/io/parquet/read/mod.rs b/src/io/parquet/read/mod.rs index dd7eb3035e3..414ab1c2c84 100644 --- a/src/io/parquet/read/mod.rs +++ b/src/io/parquet/read/mod.rs @@ -224,17 +224,17 @@ fn column_datatype(data_type: &DataType, column: usize) -> DataType { | LargeUtf8 | Dictionary(_) | List | LargeList | FixedSizeList => data_type.clone(), Struct => { if let DataType::Struct(fields) = data_type.to_logical_type() { - let mut total_columns = 0; + let mut total_chunk = 0; let mut total_fields = 0; for f in fields { - let field_columns = column_offset(f.data_type()); - if column < total_columns + field_columns { - return column_datatype(f.data_type(), column + total_columns); + let field_chunk = column_offset(f.data_type()); + if column < total_chunk + field_chunk { + return column_datatype(f.data_type(), column + total_chunk); } - total_fields += (field_columns > 0) as usize; - total_columns += field_columns; + total_fields += (field_chunk > 0) as usize; + total_chunk += field_chunk; } - fields[column + total_fields - total_columns] + fields[column + total_fields - total_chunk] .data_type() .clone() } else { diff --git a/src/io/parquet/read/record_batch.rs b/src/io/parquet/read/record_batch.rs index a168b2981b7..f42f380592e 100644 --- a/src/io/parquet/read/record_batch.rs +++ b/src/io/parquet/read/record_batch.rs @@ -4,9 +4,10 @@ use std::{ }; use crate::{ + array::Array, + chunk::Chunk, datatypes::{Field, Schema}, error::{ArrowError, Result}, - record_batch::RecordBatch, }; use super::{ @@ -16,7 +17,7 @@ use super::{ type GroupFilter = Arc bool>; -/// Single threaded iterator of [`RecordBatch`] from a parquet file. +/// Single threaded iterator of a paquet file. pub struct RecordReader { reader: R, schema: Arc, @@ -107,7 +108,7 @@ impl RecordReader { } impl Iterator for RecordReader { - type Item = Result; + type Item = Result>>; fn next(&mut self) -> Option { if self.schema.fields().is_empty() { @@ -158,19 +159,18 @@ impl Iterator for RecordReader { array }; - columns.push(array.into()); + let column: Arc = array.into(); + columns.push(column); Result::Ok((b1, b2, columns)) }, ); self.current_group += 1; - Some(a.and_then(|(b1, b2, columns)| { + Some(a.map(|(b1, b2, columns)| { self.buffer = b1; self.decompress_buffer = b2; - RecordBatch::try_new(self.schema.clone(), columns).map(|batch| { - self.remaining_rows -= batch.num_rows(); - batch - }) + self.remaining_rows -= columns[0].len(); + Chunk::new(columns) })) } } diff --git a/src/io/parquet/write/record_batch.rs b/src/io/parquet/write/record_batch.rs index b6cb940799c..3fa34b4e3b9 100644 --- a/src/io/parquet/write/record_batch.rs +++ b/src/io/parquet/write/record_batch.rs @@ -6,23 +6,24 @@ use super::{ SchemaDescriptor, WriteOptions, }; use crate::{ + array::Array, + chunk::Chunk, datatypes::Schema, error::{ArrowError, Result}, - record_batch::RecordBatch, }; -/// An iterator adapter that converts an iterator over [`RecordBatch`] into an iterator +/// An iterator adapter that converts an iterator over [`Chunk`] into an iterator /// of row groups. /// Use it to create an iterator consumable by the parquet's API. -pub struct RowGroupIterator>> { +pub struct RowGroupIterator + 'static, I: Iterator>>> { iter: I, options: WriteOptions, parquet_schema: SchemaDescriptor, encodings: Vec, } -impl<'a, I: Iterator>> RowGroupIterator { - /// Creates a new [`RowGroupIterator`] from an iterator over [`RecordBatch`]. +impl + 'static, I: Iterator>>> RowGroupIterator { + /// Creates a new [`RowGroupIterator`] from an iterator over [`Chunk`]. pub fn try_new( iter: I, schema: &Schema, @@ -47,18 +48,20 @@ impl<'a, I: Iterator>> RowGroupIterator { } } -impl>> Iterator for RowGroupIterator { +impl + 'static, I: Iterator>>> Iterator + for RowGroupIterator +{ type Item = Result>; fn next(&mut self) -> Option { let options = self.options; - self.iter.next().map(|batch| { - let batch = batch?; - let columns = batch.columns().to_vec(); + self.iter.next().map(|maybe_chunk| { + let columns = maybe_chunk?; let encodings = self.encodings.clone(); Ok(DynIter::new( columns + .into_arrays() .into_iter() .zip(self.parquet_schema.columns().to_vec().into_iter()) .zip(encodings.into_iter()) diff --git a/src/io/print.rs b/src/io/print.rs index 4cc46f31e73..1f5e8c1b9c3 100644 --- a/src/io/print.rs +++ b/src/io/print.rs @@ -1,51 +1,39 @@ -//! APIs to represent [`RecordBatch`] as a formatted table. +//! APIs to represent [`Chunk`] as a formatted table. -use crate::{array::get_display, record_batch::RecordBatch}; +use crate::{ + array::{get_display, Array}, + chunk::Chunk, +}; use comfy_table::{Cell, Table}; -/// Returns a visual representation of multiple [`RecordBatch`]es. -pub fn write(batches: &[RecordBatch]) -> String { - create_table(batches).to_string() -} - -/// Prints a visual representation of record batches to stdout -pub fn print(results: &[RecordBatch]) { - println!("{}", create_table(results)) -} - -/// Convert a series of record batches into a table -fn create_table(results: &[RecordBatch]) -> Table { +/// Returns a visual representation of [`Chunk`] +pub fn write, N: AsRef>(batches: &[Chunk], names: &[N]) -> String { let mut table = Table::new(); table.load_preset("||--+-++| ++++++"); - if results.is_empty() { - return table; + if batches.is_empty() { + return table.to_string(); } - let schema = results[0].schema(); - - let mut header = Vec::new(); - for field in schema.fields() { - header.push(Cell::new(field.name())); - } + let header = names.iter().map(|name| Cell::new(name.as_ref())); table.set_header(header); - for batch in results { + for batch in batches { let displayes = batch - .columns() + .arrays() .iter() .map(|array| get_display(array.as_ref())) .collect::>(); - for row in 0..batch.num_rows() { + for row in 0..batch.len() { let mut cells = Vec::new(); - (0..batch.num_columns()).for_each(|col| { + (0..batch.arrays().len()).for_each(|col| { let string = displayes[col](row); cells.push(Cell::new(&string)); }); table.add_row(cells); } } - table + table.to_string() } diff --git a/src/lib.rs b/src/lib.rs index d215753026f..608bef98120 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -9,7 +9,7 @@ pub mod array; pub mod bitmap; pub mod buffer; -pub(crate) mod columns; +pub mod chunk; pub mod error; pub mod scalar; pub mod trusted_len; @@ -17,7 +17,7 @@ pub mod types; pub mod compute; pub mod io; -pub mod record_batch; +//pub mod record_batch; pub mod temporal_conversions; pub mod datatypes; diff --git a/src/record_batch.rs b/src/record_batch.rs deleted file mode 100644 index 369e74a0481..00000000000 --- a/src/record_batch.rs +++ /dev/null @@ -1,340 +0,0 @@ -//! Contains [`RecordBatch`]. -use std::sync::Arc; - -use crate::array::*; -use crate::datatypes::*; -use crate::error::{ArrowError, Result}; - -/// A two-dimensional dataset with a number of -/// columns ([`Array`]) and rows and defined [`Schema`](crate::datatypes::Schema). -/// # Implementation -/// Cloning is `O(C)` where `C` is the number of columns. -#[derive(Clone, Debug, PartialEq)] -pub struct RecordBatch { - schema: Arc, - columns: Vec>, -} - -impl RecordBatch { - /// Creates a [`RecordBatch`] from a schema and columns. - /// # Errors - /// This function errors iff - /// * `columns` is empty - /// * the schema and column data types do not match - /// * `columns` have a different length - /// # Example - /// - /// ``` - /// # use std::sync::Arc; - /// # use arrow2::array::PrimitiveArray; - /// # use arrow2::datatypes::{Schema, Field, DataType}; - /// # use arrow2::record_batch::RecordBatch; - /// # fn main() -> arrow2::error::Result<()> { - /// let id_array = PrimitiveArray::from_slice([1i32, 2, 3, 4, 5]); - /// let schema = Arc::new(Schema::new(vec![ - /// Field::new("id", DataType::Int32, false) - /// ])); - /// - /// let batch = RecordBatch::try_new( - /// schema, - /// vec![Arc::new(id_array)] - /// )?; - /// # Ok(()) - /// # } - /// ``` - pub fn try_new(schema: Arc, columns: Vec>) -> Result { - let options = RecordBatchOptions::default(); - Self::validate_new_batch(&schema, columns.as_slice(), &options)?; - Ok(RecordBatch { schema, columns }) - } - - /// Creates a [`RecordBatch`] from a schema and columns, with additional options, - /// such as whether to strictly validate field names. - /// - /// See [`Self::try_new()`] for the expected conditions. - pub fn try_new_with_options( - schema: Arc, - columns: Vec>, - options: &RecordBatchOptions, - ) -> Result { - Self::validate_new_batch(&schema, &columns, options)?; - Ok(RecordBatch { schema, columns }) - } - - /// Creates a new empty [`RecordBatch`]. - pub fn new_empty(schema: Arc) -> Self { - let columns = schema - .fields() - .iter() - .map(|field| new_empty_array(field.data_type().clone()).into()) - .collect(); - RecordBatch { schema, columns } - } - - /// Validate the schema and columns using [`RecordBatchOptions`]. Returns an error - /// if any validation check fails. - fn validate_new_batch( - schema: &Schema, - columns: &[Arc], - options: &RecordBatchOptions, - ) -> Result<()> { - // check that there are some columns - if columns.is_empty() { - return Err(ArrowError::InvalidArgumentError( - "at least one column must be defined to create a record batch".to_string(), - )); - } - // check that number of fields in schema match column length - if schema.fields().len() != columns.len() { - return Err(ArrowError::InvalidArgumentError(format!( - "number of columns({}) must match number of fields({}) in schema", - columns.len(), - schema.fields().len(), - ))); - } - // check that all columns have the same row count, and match the schema - let len = columns[0].len(); - - // This is a bit repetitive, but it is better to check the condition outside the loop - if options.match_field_names { - for (i, column) in columns.iter().enumerate() { - if column.len() != len { - return Err(ArrowError::InvalidArgumentError( - "all columns in a record batch must have the same length".to_string(), - )); - } - if column.data_type() != schema.field(i).data_type() { - return Err(ArrowError::InvalidArgumentError(format!( - "column types must match schema types, expected {:?} but found {:?} at column index {}", - schema.field(i).data_type(), - column.data_type(), - i))); - } - } - } else { - for (i, column) in columns.iter().enumerate() { - if column.len() != len { - return Err(ArrowError::InvalidArgumentError( - "all columns in a record batch must have the same length".to_string(), - )); - } - if !column - .data_type() - .equals_datatype(schema.field(i).data_type()) - { - return Err(ArrowError::InvalidArgumentError(format!( - "column types must match schema types, expected {:?} but found {:?} at column index {}", - schema.field(i).data_type(), - column.data_type(), - i))); - } - } - } - - Ok(()) - } - - /// Returns the [`Schema`](crate::datatypes::Schema) of the record batch. - pub fn schema(&self) -> &Arc { - &self.schema - } - - /// Returns the number of columns in the record batch. - /// - /// # Example - /// - /// ``` - /// # use std::sync::Arc; - /// # use arrow2::array::PrimitiveArray; - /// # use arrow2::datatypes::{Schema, Field, DataType}; - /// # use arrow2::record_batch::RecordBatch; - /// # fn main() -> arrow2::error::Result<()> { - /// let id_array = PrimitiveArray::from_slice([1i32, 2, 3, 4, 5]); - /// let schema = Arc::new(Schema::new(vec![ - /// Field::new("id", DataType::Int32, false) - /// ])); - /// - /// let batch = RecordBatch::try_new(schema, vec![Arc::new(id_array)])?; - /// - /// assert_eq!(batch.num_columns(), 1); - /// # Ok(()) - /// # } - /// ``` - pub fn num_columns(&self) -> usize { - self.columns.len() - } - - /// Returns the number of rows in each column. - /// - /// # Panics - /// - /// Panics if the `RecordBatch` contains no columns. - /// - /// # Example - /// - /// ``` - /// # use std::sync::Arc; - /// # use arrow2::array::PrimitiveArray; - /// # use arrow2::datatypes::{Schema, Field, DataType}; - /// # use arrow2::record_batch::RecordBatch; - /// # fn main() -> arrow2::error::Result<()> { - /// let id_array = PrimitiveArray::from_slice([1i32, 2, 3, 4, 5]); - /// let schema = Arc::new(Schema::new(vec![ - /// Field::new("id", DataType::Int32, false) - /// ])); - /// - /// let batch = RecordBatch::try_new(schema, vec![Arc::new(id_array)])?; - /// - /// assert_eq!(batch.num_rows(), 5); - /// # Ok(()) - /// # } - /// ``` - pub fn num_rows(&self) -> usize { - self.columns[0].len() - } - - /// Get a reference to a column's array by index. - /// - /// # Panics - /// - /// Panics if `index` is outside of `0..num_columns`. - pub fn column(&self, index: usize) -> &Arc { - &self.columns[index] - } - - /// Get a reference to all columns in the record batch. - pub fn columns(&self) -> &[Arc] { - &self.columns[..] - } - - /// Create a `RecordBatch` from an iterable list of pairs of the - /// form `(field_name, array)`, with the same requirements on - /// fields and arrays as [`RecordBatch::try_new`]. This method is - /// often used to create a single `RecordBatch` from arrays, - /// e.g. for testing. - /// - /// The resulting schema is marked as nullable for each column if - /// the array for that column is has any nulls. To explicitly - /// specify nullibility, use [`RecordBatch::try_from_iter_with_nullable`] - /// - /// Example: - /// ``` - /// use std::sync::Arc; - /// use arrow2::array::*; - /// use arrow2::datatypes::DataType; - /// use arrow2::record_batch::RecordBatch; - /// - /// let a: Arc = Arc::new(Int32Array::from_slice(&[1, 2])); - /// let b: Arc = Arc::new(Utf8Array::::from_slice(&["a", "b"])); - /// - /// let record_batch = RecordBatch::try_from_iter(vec![ - /// ("a", a), - /// ("b", b), - /// ]); - /// ``` - pub fn try_from_iter(value: I) -> Result - where - I: IntoIterator)>, - F: AsRef, - { - // TODO: implement `TryFrom` trait, once - // https://github.com/rust-lang/rust/issues/50133 is no longer an - // issue - let iter = value.into_iter().map(|(field_name, array)| { - let nullable = array.null_count() > 0; - (field_name, array, nullable) - }); - - Self::try_from_iter_with_nullable(iter) - } - - /// Create a `RecordBatch` from an iterable list of tuples of the - /// form `(field_name, array, nullable)`, with the same requirements on - /// fields and arrays as [`RecordBatch::try_new`]. This method is often - /// used to create a single `RecordBatch` from arrays, e.g. for - /// testing. - /// - /// Example: - /// ``` - /// use std::sync::Arc; - /// use arrow2::array::*; - /// use arrow2::datatypes::DataType; - /// use arrow2::record_batch::RecordBatch; - /// - /// let a: Arc = Arc::new(Int32Array::from_slice(&[1, 2])); - /// let b: Arc = Arc::new(Utf8Array::::from_slice(&["a", "b"])); - /// - /// // Note neither `a` nor `b` has any actual nulls, but we mark - /// // b an nullable - /// let record_batch = RecordBatch::try_from_iter_with_nullable(vec![ - /// ("a", a, false), - /// ("b", b, true), - /// ]); - /// ``` - pub fn try_from_iter_with_nullable(value: I) -> Result - where - I: IntoIterator, bool)>, - F: AsRef, - { - // TODO: implement `TryFrom` trait, once - // https://github.com/rust-lang/rust/issues/50133 is no longer an - // issue - let (fields, columns) = value - .into_iter() - .map(|(field_name, array, nullable)| { - let field_name = field_name.as_ref(); - let field = Field::new(field_name, array.data_type().clone(), nullable); - (field, array) - }) - .unzip(); - - let schema = Arc::new(Schema::new(fields)); - RecordBatch::try_new(schema, columns) - } - - /// Deconstructs itself into its internal components - pub fn into_inner(self) -> (Vec>, Arc) { - let Self { columns, schema } = self; - (columns, schema) - } -} - -/// Options that control the behaviour used when creating a [`RecordBatch`]. -#[derive(Debug)] -pub struct RecordBatchOptions { - /// Match field names of structs and lists. If set to `true`, the names must match. - pub match_field_names: bool, -} - -impl Default for RecordBatchOptions { - fn default() -> Self { - Self { - match_field_names: true, - } - } -} - -impl From for RecordBatch { - /// # Panics iff the null count of the array is not null. - fn from(array: StructArray) -> Self { - assert!(array.null_count() == 0); - let (fields, values, _) = array.into_data(); - RecordBatch { - schema: Arc::new(Schema::new(fields)), - columns: values, - } - } -} - -impl From for StructArray { - fn from(batch: RecordBatch) -> Self { - let (fields, values) = batch - .schema - .fields - .iter() - .zip(batch.columns.iter()) - .map(|t| (t.0.clone(), t.1.clone())) - .unzip(); - StructArray::from_data(DataType::Struct(fields), values, None) - } -} diff --git a/tests/it/io/avro/read.rs b/tests/it/io/avro/read.rs index bd75048efc4..bd22bbe153c 100644 --- a/tests/it/io/avro/read.rs +++ b/tests/it/io/avro/read.rs @@ -1,5 +1,6 @@ use std::sync::Arc; +use arrow2::chunk::Chunk; use avro_rs::types::{Record, Value}; use avro_rs::{Codec, Writer}; use avro_rs::{Days, Duration, Millis, Months, Schema as AvroSchema}; @@ -8,7 +9,6 @@ use arrow2::array::*; use arrow2::datatypes::*; use arrow2::error::Result; use arrow2::io::avro::read; -use arrow2::record_batch::RecordBatch; pub(super) fn schema() -> (AvroSchema, Schema) { let raw_schema = r#" @@ -69,7 +69,7 @@ pub(super) fn schema() -> (AvroSchema, Schema) { (AvroSchema::parse_str(raw_schema).unwrap(), schema) } -pub(super) fn data() -> RecordBatch { +pub(super) fn data() -> Chunk> { let data = vec![ Some(vec![Some(1i32), None, Some(3)]), Some(vec![Some(1i32), None, Some(3)]), @@ -80,21 +80,21 @@ pub(super) fn data() -> RecordBatch { let columns = vec![ Arc::new(Int64Array::from_slice([27, 47])) as Arc, - Arc::new(Utf8Array::::from_slice(["foo", "bar"])) as Arc, - Arc::new(Int32Array::from_slice([1, 1])) as Arc, - Arc::new(Int32Array::from_slice([1, 2]).to(DataType::Date32)) as Arc, - Arc::new(BinaryArray::::from_slice([b"foo", b"bar"])) as Arc, - Arc::new(PrimitiveArray::::from_slice([1.0, 2.0])) as Arc, - Arc::new(BooleanArray::from_slice([true, false])) as Arc, - Arc::new(Utf8Array::::from([Some("foo"), None])) as Arc, + Arc::new(Utf8Array::::from_slice(["foo", "bar"])), + Arc::new(Int32Array::from_slice([1, 1])), + Arc::new(Int32Array::from_slice([1, 2]).to(DataType::Date32)), + Arc::new(BinaryArray::::from_slice([b"foo", b"bar"])), + Arc::new(PrimitiveArray::::from_slice([1.0, 2.0])), + Arc::new(BooleanArray::from_slice([true, false])), + Arc::new(Utf8Array::::from([Some("foo"), None])), array.into_arc(), Arc::new(DictionaryArray::::from_data( Int32Array::from_slice([1, 0]), Arc::new(Utf8Array::::from_slice(["SPADES", "HEARTS"])), - )) as Arc, + )), ]; - RecordBatch::try_new(Arc::new(schema().1), columns).unwrap() + Chunk::try_new(columns).unwrap() } pub(super) fn write_avro(codec: Codec) -> std::result::Result, avro_rs::Error> { @@ -149,7 +149,7 @@ pub(super) fn write_avro(codec: Codec) -> std::result::Result, avro_rs:: Ok(writer.into_inner().unwrap()) } -pub(super) fn read_avro(mut avro: &[u8]) -> Result { +pub(super) fn read_avro(mut avro: &[u8]) -> Result<(Chunk>, Schema)> { let file = &mut avro; let (avro_schema, schema, codec, file_marker) = read::read_metadata(file)?; @@ -157,18 +157,20 @@ pub(super) fn read_avro(mut avro: &[u8]) -> Result { let mut reader = read::Reader::new( read::Decompressor::new(read::BlockStreamIterator::new(file, file_marker), codec), avro_schema, - Arc::new(schema), + schema.fields.clone(), ); - reader.next().unwrap() + reader.next().unwrap().map(|x| (x, schema)) } fn test(codec: Codec) -> Result<()> { let avro = write_avro(codec).unwrap(); let expected = data(); + let (_, expected_schema) = schema(); - let result = read_avro(&avro)?; + let (result, schema) = read_avro(&avro)?; + assert_eq!(schema, expected_schema); assert_eq!(result, expected); Ok(()) } diff --git a/tests/it/io/avro/read_async.rs b/tests/it/io/avro/read_async.rs index 5d0c688b7c7..3dd04cdb479 100644 --- a/tests/it/io/avro/read_async.rs +++ b/tests/it/io/avro/read_async.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use avro_rs::Codec; use futures::pin_mut; @@ -8,18 +6,17 @@ use futures::StreamExt; use arrow2::error::Result; use arrow2::io::avro::read_async::*; -use super::read::{data, write_avro}; +use super::read::{schema, write_avro}; async fn test(codec: Codec) -> Result<()> { let avro_data = write_avro(codec).unwrap(); - let expected = data(); + let (_, expected_schema) = schema(); let mut reader = &mut &avro_data[..]; let (_, schema, _, marker) = read_metadata(&mut reader).await?; - let schema = Arc::new(schema); - assert_eq!(schema.as_ref(), expected.schema().as_ref()); + assert_eq!(schema, expected_schema); let blocks = block_stream(&mut reader, marker).await; diff --git a/tests/it/io/avro/write.rs b/tests/it/io/avro/write.rs index 6e605828dab..01c33bbd77c 100644 --- a/tests/it/io/avro/write.rs +++ b/tests/it/io/avro/write.rs @@ -1,10 +1,10 @@ use std::sync::Arc; use arrow2::array::*; +use arrow2::chunk::Chunk; use arrow2::datatypes::*; use arrow2::error::Result; use arrow2::io::avro::write; -use arrow2::record_batch::RecordBatch; fn schema() -> Schema { Schema::new(vec![ @@ -19,7 +19,7 @@ fn schema() -> Schema { ]) } -fn data() -> RecordBatch { +fn data() -> Chunk> { let columns = vec![ Arc::new(Int64Array::from_slice([27, 47])) as Arc, Arc::new(Utf8Array::::from_slice(["foo", "bar"])) as Arc, @@ -31,25 +31,26 @@ fn data() -> RecordBatch { Arc::new(Utf8Array::::from([Some("foo"), None])) as Arc, ]; - RecordBatch::try_new(Arc::new(schema()), columns).unwrap() + Chunk::try_new(columns).unwrap() } use super::read::read_avro; fn write_avro>( - arrays: &[R], + columns: &Chunk, schema: &Schema, compression: Option, ) -> Result> { let avro_fields = write::to_avro_schema(schema)?; - let mut serializers = arrays + let mut serializers = columns + .arrays() .iter() .map(|x| x.as_ref()) .zip(avro_fields.iter()) .map(|(array, field)| write::new_serializer(array, &field.schema)) .collect::>(); - let mut block = write::Block::new(arrays[0].as_ref().len(), vec![]); + let mut block = write::Block::new(columns.len(), vec![]); write::serialize(&mut serializers, &mut block); @@ -68,19 +69,16 @@ fn write_avro>( fn roundtrip(compression: Option) -> Result<()> { let expected = data(); + let expected_schema = schema(); - let arrays = expected.columns(); - let schema = expected.schema(); + let data = write_avro(&expected, &expected_schema, compression)?; - let data = write_avro(arrays, schema, compression)?; + let (result, read_schema) = read_avro(&data)?; - let result = read_avro(&data)?; - - assert_eq!(result.schema(), expected.schema()); + assert_eq!(expected_schema, read_schema); for (c1, c2) in result.columns().iter().zip(expected.columns().iter()) { - assert_eq!(c1, c2); + assert_eq!(c1.as_ref(), c2.as_ref()); } - assert_eq!(result, expected); Ok(()) } diff --git a/tests/it/io/csv/read.rs b/tests/it/io/csv/read.rs index 40e5344ea3c..d7981230871 100644 --- a/tests/it/io/csv/read.rs +++ b/tests/it/io/csv/read.rs @@ -32,7 +32,7 @@ fn read() -> Result<()> { let mut rows = vec![ByteRecord::default(); 100]; let rows_read = read_rows(&mut reader, 0, &mut rows)?; - let batch = deserialize_batch( + let columns = deserialize_batch( &rows[..rows_read], schema.fields(), None, @@ -40,21 +40,16 @@ fn read() -> Result<()> { deserialize_column, )?; - let batch_schema = batch.schema(); + assert_eq!(14, columns.len()); + assert_eq!(3, columns.arrays().len()); - assert_eq!(&schema, batch_schema); - assert_eq!(14, batch.num_rows()); - assert_eq!(3, batch.num_columns()); - - let lat = batch - .column(1) + let lat = columns.arrays()[1] .as_any() .downcast_ref::() .unwrap(); assert!((57.653484 - lat.value(0)).abs() < f64::EPSILON); - let city = batch - .column(0) + let city = columns.arrays()[0] .as_any() .downcast_ref::>() .unwrap(); diff --git a/tests/it/io/csv/read_async.rs b/tests/it/io/csv/read_async.rs index 6d22004ad7c..73c2ad60078 100644 --- a/tests/it/io/csv/read_async.rs +++ b/tests/it/io/csv/read_async.rs @@ -1,5 +1,4 @@ use futures::io::Cursor; -use std::sync::Arc; use arrow2::array::*; use arrow2::error::Result; @@ -24,12 +23,12 @@ async fn read() -> Result<()> { "Aberdeen, Aberdeen City, UK",57.149651,-2.099075"#; let mut reader = AsyncReaderBuilder::new().create_reader(Cursor::new(data.as_bytes())); - let schema = Arc::new(infer_schema(&mut reader, None, true, &infer).await?); + let schema = infer_schema(&mut reader, None, true, &infer).await?; let mut rows = vec![ByteRecord::default(); 100]; let rows_read = read_rows(&mut reader, 0, &mut rows).await?; - let batch = deserialize_batch( + let columns = deserialize_batch( &rows[..rows_read], schema.fields(), None, @@ -37,21 +36,16 @@ async fn read() -> Result<()> { deserialize_column, )?; - let batch_schema = batch.schema(); + assert_eq!(14, columns.len()); + assert_eq!(3, columns.arrays().len()); - assert_eq!(&schema, batch_schema); - assert_eq!(14, batch.num_rows()); - assert_eq!(3, batch.num_columns()); - - let lat = batch - .column(1) + let lat = columns.arrays()[1] .as_any() .downcast_ref::() .unwrap(); assert!((57.653484 - lat.value(0)).abs() < f64::EPSILON); - let city = batch - .column(0) + let city = columns.arrays()[0] .as_any() .downcast_ref::>() .unwrap(); diff --git a/tests/it/io/csv/write.rs b/tests/it/io/csv/write.rs index c8a8fc0984f..3b86ed2a60a 100644 --- a/tests/it/io/csv/write.rs +++ b/tests/it/io/csv/write.rs @@ -2,12 +2,12 @@ use std::io::Cursor; use std::sync::Arc; use arrow2::array::*; +use arrow2::chunk::Chunk; use arrow2::datatypes::*; use arrow2::error::Result; use arrow2::io::csv::write::*; -use arrow2::record_batch::RecordBatch; -fn data() -> RecordBatch { +fn data() -> Chunk> { let c1 = Utf8Array::::from_slice(["a b", "c", "d"]); let c2 = Float64Array::from([Some(123.564532), None, Some(-556132.25)]); let c3 = UInt32Array::from_slice(&[3, 2, 1]); @@ -19,28 +19,27 @@ fn data() -> RecordBatch { let keys = UInt32Array::from_slice(&[2, 0, 1]); let c7 = DictionaryArray::from_data(keys, Arc::new(c1.clone())); - RecordBatch::try_from_iter(vec![ - ("c1", Arc::new(c1) as Arc), - ("c2", Arc::new(c2) as Arc), - ("c3", Arc::new(c3) as Arc), - ("c4", Arc::new(c4) as Arc), - ("c5", Arc::new(c5) as Arc), - ("c6", Arc::new(c6) as Arc), - ("c7", Arc::new(c7) as Arc), + Chunk::new(vec![ + Box::new(c1) as Box, + Box::new(c2), + Box::new(c3), + Box::new(c4), + Box::new(c5), + Box::new(c6), + Box::new(c7), ]) - .unwrap() } #[test] fn write_csv() -> Result<()> { - let batch = data(); + let columns = data(); let write = Cursor::new(Vec::::new()); let mut writer = WriterBuilder::new().from_writer(write); - write_header(&mut writer, batch.schema())?; + write_header(&mut writer, &["c1", "c2", "c3", "c4", "c5", "c6", "c7"])?; let options = SerializeOptions::default(); - write_batch(&mut writer, &batch, &options)?; + write_chunk(&mut writer, &columns, &options)?; // check let buffer = writer.into_inner().unwrap().into_inner(); @@ -68,7 +67,7 @@ fn write_csv_custom_options() -> Result<()> { time64_format: Some("%r".to_string()), ..Default::default() }; - write_batch(&mut writer, &batch, &options)?; + write_chunk(&mut writer, &batch, &options)?; // check let buffer = writer.into_inner().unwrap().into_inner(); @@ -83,7 +82,7 @@ d|-556132.25|1||2019-04-18 02:45:55.555|11:46:03 PM|c Ok(()) } -fn data_array(column: usize) -> (RecordBatch, Vec<&'static str>) { +fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { let (array, expected) = match column { 0 => ( Arc::new(Utf8Array::::from_slice(["a b", "c", "d"])) as Arc, @@ -208,21 +207,18 @@ fn data_array(column: usize) -> (RecordBatch, Vec<&'static str>) { _ => todo!(), }; - ( - RecordBatch::try_from_iter(vec![("c1", array)]).unwrap(), - expected, - ) + (Chunk::new(vec![array]), expected) } fn write_single(column: usize) -> Result<()> { - let (batch, data) = data_array(column); + let (columns, data) = data_array(column); let write = Cursor::new(Vec::::new()); let mut writer = WriterBuilder::new().delimiter(b'|').from_writer(write); - write_header(&mut writer, batch.schema())?; + write_header(&mut writer, &["c1"])?; let options = SerializeOptions::default(); - write_batch(&mut writer, &batch, &options)?; + write_chunk(&mut writer, &columns, &options)?; // check let buffer = writer.into_inner().unwrap().into_inner(); @@ -230,7 +226,7 @@ fn write_single(column: usize) -> Result<()> { let mut expected = "c1\n".to_owned(); expected.push_str(&data.join("\n")); expected.push('\n'); - assert_eq!(expected, String::from_utf8(buffer).unwrap(),); + assert_eq!(expected, String::from_utf8(buffer).unwrap()); Ok(()) } diff --git a/tests/it/io/ipc/common.rs b/tests/it/io/ipc/common.rs index 52a75e0052a..0f5270d76e1 100644 --- a/tests/it/io/ipc/common.rs +++ b/tests/it/io/ipc/common.rs @@ -1,18 +1,17 @@ -use std::{collections::HashMap, fs::File, io::Read}; +use std::{collections::HashMap, fs::File, io::Read, sync::Arc}; use arrow2::{ - datatypes::Schema, error::Result, io::ipc::read::read_stream_metadata, - io::ipc::read::StreamReader, io::ipc::IpcField, io::json_integration::read, - io::json_integration::ArrowJson, record_batch::RecordBatch, + array::Array, chunk::Chunk, datatypes::Schema, error::Result, + io::ipc::read::read_stream_metadata, io::ipc::read::StreamReader, io::ipc::IpcField, + io::json_integration::read, io::json_integration::ArrowJson, }; use flate2::read::GzDecoder; +type IpcRead = (Schema, Vec, Vec>>); + /// Read gzipped JSON file -pub fn read_gzip_json( - version: &str, - file_name: &str, -) -> Result<(Schema, Vec, Vec)> { +pub fn read_gzip_json(version: &str, file_name: &str) -> Result { let testdata = crate::test_util::arrow_test_data(); let file = File::open(format!( "{}/arrow-ipc-stream/integration/{}/{}.json.gz", @@ -41,16 +40,13 @@ pub fn read_gzip_json( let batches = arrow_json .batches .iter() - .map(|batch| read::to_record_batch(&schema, &ipc_fields, batch, &dictionaries)) + .map(|batch| read::deserialize_chunk(&schema, &ipc_fields, batch, &dictionaries)) .collect::>>()?; Ok((schema, ipc_fields, batches)) } -pub fn read_arrow_stream( - version: &str, - file_name: &str, -) -> (Schema, Vec, Vec) { +pub fn read_arrow_stream(version: &str, file_name: &str) -> IpcRead { let testdata = crate::test_util::arrow_test_data(); let mut file = File::open(format!( "{}/arrow-ipc-stream/integration/{}/{}.stream", @@ -61,7 +57,7 @@ pub fn read_arrow_stream( let metadata = read_stream_metadata(&mut file).unwrap(); let reader = StreamReader::new(file, metadata); - let schema = reader.metadata().schema.as_ref().clone(); + let schema = reader.metadata().schema.clone(); let ipc_fields = reader.metadata().ipc_schema.fields.clone(); ( diff --git a/tests/it/io/ipc/read/file.rs b/tests/it/io/ipc/read/file.rs index 8cce26b81c5..9331b2e673d 100644 --- a/tests/it/io/ipc/read/file.rs +++ b/tests/it/io/ipc/read/file.rs @@ -18,7 +18,7 @@ fn test_file(version: &str, file_name: &str) -> Result<()> { let metadata = read_file_metadata(&mut file)?; let reader = FileReader::new(file, metadata, None); - assert_eq!(&schema, reader.schema().as_ref()); + assert_eq!(&schema, reader.schema()); batches.iter().zip(reader).try_for_each(|(lhs, rhs)| { assert_eq!(lhs, &rhs?); @@ -168,7 +168,7 @@ fn test_projection(version: &str, file_name: &str, column: usize) -> Result<()> assert_eq!(reader.schema().fields().len(), 1); reader.try_for_each(|rhs| { - assert_eq!(rhs?.num_columns(), 1); + assert_eq!(rhs?.arrays().len(), 1); Result::Ok(()) })?; Ok(()) diff --git a/tests/it/io/ipc/read/stream.rs b/tests/it/io/ipc/read/stream.rs index 3c3fcf095d6..18e7ecd0756 100644 --- a/tests/it/io/ipc/read/stream.rs +++ b/tests/it/io/ipc/read/stream.rs @@ -18,7 +18,7 @@ fn test_file(version: &str, file_name: &str) -> Result<()> { // read expected JSON output let (schema, ipc_fields, batches) = read_gzip_json(version, file_name)?; - assert_eq!(&schema, reader.metadata().schema.as_ref()); + assert_eq!(&schema, &reader.metadata().schema); assert_eq!(&ipc_fields, &reader.metadata().ipc_schema.fields); batches diff --git a/tests/it/io/ipc/write/file.rs b/tests/it/io/ipc/write/file.rs index 979d970c150..2df0cda955b 100644 --- a/tests/it/io/ipc/write/file.rs +++ b/tests/it/io/ipc/write/file.rs @@ -1,16 +1,17 @@ use std::io::Cursor; +use std::sync::Arc; use arrow2::array::*; -use arrow2::datatypes::Schema; +use arrow2::chunk::Chunk; +use arrow2::datatypes::{Field, Schema}; use arrow2::error::Result; use arrow2::io::ipc::read::{read_file_metadata, FileReader}; use arrow2::io::ipc::{write::*, IpcField}; -use arrow2::record_batch::RecordBatch; use crate::io::ipc::common::read_gzip_json; fn write_( - batches: &[RecordBatch], + batches: &[Chunk>], schema: &Schema, ipc_fields: Option>, compression: Option, @@ -25,18 +26,26 @@ fn write_( Ok(writer.into_inner()) } -fn round_trip(batch: RecordBatch, ipc_fields: Option>) -> Result<()> { - let (expected_schema, expected_batches) = (batch.schema().clone(), vec![batch.clone()]); - - let schema = batch.schema().clone(); - let result = write_(&[batch], &schema, ipc_fields, Some(Compression::ZSTD))?; +fn round_trip( + columns: Chunk>, + schema: Schema, + ipc_fields: Option>, +) -> Result<()> { + let (expected_schema, expected_batches) = (schema.clone(), vec![columns]); + + let result = write_( + &expected_batches, + &schema, + ipc_fields, + Some(Compression::ZSTD), + )?; let mut reader = Cursor::new(result); let metadata = read_file_metadata(&mut reader)?; - let schema = metadata.schema().clone(); + let schema = metadata.schema.clone(); let reader = FileReader::new(reader, metadata, None); - assert_eq!(schema.as_ref(), expected_schema.as_ref()); + assert_eq!(schema, expected_schema); let batches = reader.collect::>>()?; @@ -56,7 +65,7 @@ fn test_file(version: &str, file_name: &str, compressed: bool) -> Result<()> { let result = write_(&batches, &schema, Some(ipc_fields), compression)?; let mut reader = Cursor::new(result); let metadata = read_file_metadata(&mut reader)?; - let schema = metadata.schema().clone(); + let schema = metadata.schema.clone(); let ipc_fields = metadata.ipc_schema.fields.clone(); let reader = FileReader::new(reader, metadata, None); @@ -65,7 +74,7 @@ fn test_file(version: &str, file_name: &str, compressed: bool) -> Result<()> { let (expected_schema, expected_ipc_fields, expected_batches) = read_gzip_json(version, file_name)?; - assert_eq!(schema.as_ref(), &expected_schema); + assert_eq!(schema, expected_schema); assert_eq!(ipc_fields, expected_ipc_fields); let batches = reader.collect::>>()?; @@ -329,8 +338,9 @@ fn write_boolean() -> Result<()> { None, Some(true), ])) as Arc; - let batch = RecordBatch::try_from_iter(vec![("a", array)])?; - round_trip(batch, None) + let schema = Schema::new(vec![Field::new("a", array.data_type().clone(), true)]); + let columns = Chunk::try_new(vec![array])?; + round_trip(columns, schema, None) } #[test] @@ -338,8 +348,9 @@ fn write_boolean() -> Result<()> { fn write_sliced_utf8() -> Result<()> { use std::sync::Arc; let array = Arc::new(Utf8Array::::from_slice(["aa", "bb"]).slice(1, 1)) as Arc; - let batch = RecordBatch::try_from_iter(vec![("a", array)])?; - round_trip(batch, None) + let schema = Schema::new(vec![Field::new("a", array.data_type().clone(), true)]); + let columns = Chunk::try_new(vec![array])?; + round_trip(columns, schema, None) } #[test] @@ -353,7 +364,9 @@ fn write_sliced_list() -> Result<()> { let mut array = MutableListArray::>::new(); array.try_extend(data).unwrap(); - let array = array.into_arc().slice(1, 2).into(); - let batch = RecordBatch::try_from_iter(vec![("a", array)]).unwrap(); - round_trip(batch, None) + let array: Arc = array.into_arc().slice(1, 2).into(); + + let schema = Schema::new(vec![Field::new("a", array.data_type().clone(), true)]); + let columns = Chunk::try_new(vec![array])?; + round_trip(columns, schema, None) } diff --git a/tests/it/io/ipc/write/stream.rs b/tests/it/io/ipc/write/stream.rs index 80fe5f505fe..8a20ef70b2e 100644 --- a/tests/it/io/ipc/write/stream.rs +++ b/tests/it/io/ipc/write/stream.rs @@ -1,17 +1,19 @@ use std::io::Cursor; +use std::sync::Arc; +use arrow2::array::Array; +use arrow2::chunk::Chunk; use arrow2::datatypes::Schema; use arrow2::error::Result; use arrow2::io::ipc::read::read_stream_metadata; use arrow2::io::ipc::read::StreamReader; use arrow2::io::ipc::write::{StreamWriter, WriteOptions}; use arrow2::io::ipc::IpcField; -use arrow2::record_batch::RecordBatch; use crate::io::ipc::common::read_arrow_stream; use crate::io::ipc::common::read_gzip_json; -fn write_(schema: &Schema, ipc_fields: &[IpcField], batches: &[RecordBatch]) -> Vec { +fn write_(schema: &Schema, ipc_fields: &[IpcField], batches: &[Chunk>]) -> Vec { let mut result = vec![]; let options = WriteOptions { compression: None }; @@ -40,7 +42,7 @@ fn test_file(version: &str, file_name: &str) { let (expected_schema, expected_ipc_fields, expected_batches) = read_gzip_json(version, file_name).unwrap(); - assert_eq!(schema.as_ref(), &expected_schema); + assert_eq!(schema, expected_schema); assert_eq!(ipc_fields, expected_ipc_fields); let batches = reader diff --git a/tests/it/io/ipc/write_async.rs b/tests/it/io/ipc/write_async.rs index e77558ce510..9f27eea8808 100644 --- a/tests/it/io/ipc/write_async.rs +++ b/tests/it/io/ipc/write_async.rs @@ -1,11 +1,13 @@ use std::io::Cursor; +use std::sync::Arc; +use arrow2::array::Array; +use arrow2::chunk::Chunk; use arrow2::datatypes::Schema; use arrow2::error::Result; use arrow2::io::ipc::read; use arrow2::io::ipc::write::stream_async; use arrow2::io::ipc::IpcField; -use arrow2::record_batch::RecordBatch; use futures::io::Cursor as AsyncCursor; use crate::io::ipc::common::read_arrow_stream; @@ -14,7 +16,7 @@ use crate::io::ipc::common::read_gzip_json; async fn write_( schema: &Schema, ipc_fields: &[IpcField], - batches: &[RecordBatch], + batches: &[Chunk>], ) -> Result> { let mut result = AsyncCursor::new(vec![]); @@ -22,7 +24,7 @@ async fn write_( let mut writer = stream_async::StreamWriter::new(&mut result, options); writer.start(schema, Some(ipc_fields)).await?; for batch in batches { - writer.write(batch, Some(ipc_fields)).await?; + writer.write(batch, schema, Some(ipc_fields)).await?; } writer.finish().await?; Ok(result.into_inner()) @@ -37,7 +39,7 @@ async fn test_file(version: &str, file_name: &str) -> Result<()> { let metadata = read::read_stream_metadata(&mut reader)?; let reader = read::StreamReader::new(reader, metadata); - let schema = reader.metadata().schema.as_ref(); + let schema = &reader.metadata().schema; let ipc_fields = reader.metadata().ipc_schema.fields.clone(); // read expected JSON output diff --git a/tests/it/io/json/mod.rs b/tests/it/io/json/mod.rs index e50abab78c3..661bfb0d3e1 100644 --- a/tests/it/io/json/mod.rs +++ b/tests/it/io/json/mod.rs @@ -4,16 +4,14 @@ mod write; use std::io::Cursor; use std::sync::Arc; -use serde_json::Value; - use arrow2::array::*; +use arrow2::chunk::Chunk; use arrow2::datatypes::*; use arrow2::error::Result; use arrow2::io::json::read as json_read; use arrow2::io::json::write as json_write; -use arrow2::record_batch::RecordBatch; -fn read_batch(data: String, fields: Vec) -> Result { +fn read_batch(data: String, fields: &[Field]) -> Result>> { let mut reader = Cursor::new(data); let mut rows = vec![String::default(); 1024]; @@ -22,10 +20,14 @@ fn read_batch(data: String, fields: Vec) -> Result { json_read::deserialize(rows, fields) } -fn write_batch(batch: RecordBatch, format: F) -> Result> { +fn write_batch>( + batch: Chunk, + names: Vec, + format: F, +) -> Result> { let batches = vec![Ok(batch)].into_iter(); - let blocks = json_write::Serializer::new(batches, vec![], format); + let blocks = json_write::Serializer::new(batches, names, vec![], format); let mut buf = Vec::new(); json_write::write(&mut buf, format, blocks)?; @@ -37,27 +39,17 @@ fn round_trip(data: String) -> Result<()> { let fields = json_read::infer(&mut reader, None)?; let data = reader.into_inner(); - let batch = read_batch(data.clone(), fields)?; - - let buf = write_batch(batch, json_write::LineDelimited::default())?; - - let result = String::from_utf8(buf).unwrap(); - println!("{}", result); - for (r, e) in result.lines().zip(data.lines()) { - let mut result_json = serde_json::from_str::(r).unwrap(); - let expected_json = serde_json::from_str::(e).unwrap(); - if let Value::Object(e) = &expected_json { - // remove null value from object to make comparison consistent: - if let Value::Object(r) = result_json { - result_json = Value::Object( - r.into_iter() - .filter(|(k, v)| e.contains_key(k) || *v != Value::Null) - .collect(), - ); - } - assert_eq!(result_json, expected_json); - } - } + let columns = read_batch(data, &fields)?; + + let buf = write_batch( + columns.clone(), + fields.iter().map(|x| x.name().to_string()).collect(), + json_write::LineDelimited::default(), + )?; + + let new_chunk = read_batch(String::from_utf8(buf).unwrap(), &fields)?; + + assert_eq!(columns, new_chunk); Ok(()) } diff --git a/tests/it/io/json/read.rs b/tests/it/io/json/read.rs index c0aa763032b..d8eccb931a9 100644 --- a/tests/it/io/json/read.rs +++ b/tests/it/io/json/read.rs @@ -11,9 +11,7 @@ use super::*; fn basic() -> Result<()> { let (data, schema, columns) = case_basics(); - let batch = read_batch(data, schema.fields.clone())?; - - assert_eq!(&schema, batch.schema().as_ref()); + let batch = read_batch(data, &schema.fields)?; columns .iter() @@ -26,9 +24,7 @@ fn basic() -> Result<()> { fn basic_projection() -> Result<()> { let (data, schema, columns) = case_basics_schema(); - let batch = read_batch(data, schema.fields.clone())?; - - assert_eq!(&schema, batch.schema().as_ref()); + let batch = read_batch(data, &schema.fields)?; columns .iter() @@ -41,9 +37,7 @@ fn basic_projection() -> Result<()> { fn lists() -> Result<()> { let (data, schema, columns) = case_list(); - let batch = read_batch(data, schema.fields.clone())?; - - assert_eq!(&schema, batch.schema().as_ref()); + let batch = read_batch(data, &schema.fields)?; columns .iter() @@ -60,10 +54,7 @@ fn line_break_in_values() -> Result<()> { {"a":null} "#; - let batch = read_batch( - data.to_string(), - vec![Field::new("a", DataType::Utf8, true)], - )?; + let batch = read_batch(data.to_string(), &[Field::new("a", DataType::Utf8, true)])?; let expected = Utf8Array::::from(&[Some("aa\n\n"), Some("aa\n"), None]); @@ -88,7 +79,7 @@ fn invalid_read_record() -> Result<()> { DataType::Struct(vec![Field::new("a", DataType::Utf8, true)]), true, )]; - let batch = read_batch("city,lat,lng".to_string(), fields); + let batch = read_batch("city,lat,lng".to_string(), &fields); assert_eq!( batch.err().unwrap().to_string(), @@ -101,9 +92,7 @@ fn invalid_read_record() -> Result<()> { fn nested_struct_arrays() -> Result<()> { let (data, schema, columns) = case_struct(); - let batch = read_batch(data, schema.fields.clone())?; - - assert_eq!(&schema, batch.schema().as_ref()); + let batch = read_batch(data, &schema.fields)?; columns .iter() @@ -124,7 +113,6 @@ fn nested_list_arrays() -> Result<()> { ); let a_list_data_type = DataType::List(Box::new(a_struct_field)); let a_field = Field::new("a", a_list_data_type.clone(), true); - let fields = vec![a_field]; let data = r#" {"a": [{"b": true, "c": {"d": "a_text"}}, {"b": false, "c": {"d": "b_text"}}]} @@ -134,7 +122,7 @@ fn nested_list_arrays() -> Result<()> { {"a": []} "#; - let batch = read_batch(data.to_string(), fields)?; + let batch = read_batch(data.to_string(), &[a_field])?; // build expected output let d = Utf8Array::::from(&vec![ @@ -168,7 +156,7 @@ fn nested_list_arrays() -> Result<()> { Some(Bitmap::from_u8_slice([0b00010111], 5)), ); - assert_eq!(expected, batch.column(0).as_ref()); + assert_eq!(expected, batch.columns()[0].as_ref()); Ok(()) } @@ -181,14 +169,10 @@ fn skip_empty_lines() { {\"a\": 3}"; - let batch = read_batch( - data.to_string(), - vec![Field::new("a", DataType::Int64, true)], - ) - .unwrap(); + let batch = read_batch(data.to_string(), &[Field::new("a", DataType::Int64, true)]).unwrap(); - assert_eq!(1, batch.num_columns()); - assert_eq!(3, batch.num_rows()); + assert_eq!(1, batch.arrays().len()); + assert_eq!(3, batch.len()); } #[test] @@ -208,9 +192,7 @@ fn row_type_validation() { fn list_of_string_dictionary_from_with_nulls() -> Result<()> { let (data, schema, columns) = case_dict(); - let batch = read_batch(data, schema.fields.clone())?; - - assert_eq!(&schema, batch.schema().as_ref()); + let batch = read_batch(data, &schema.fields)?; assert_eq!(columns[0].as_ref(), batch.columns()[0].as_ref()); Ok(()) diff --git a/tests/it/io/json/write.rs b/tests/it/io/json/write.rs index c7fa9cc3771..51997c1d367 100644 --- a/tests/it/io/json/write.rs +++ b/tests/it/io/json/write.rs @@ -4,26 +4,24 @@ use arrow2::{ array::*, bitmap::Bitmap, buffer::Buffer, - datatypes::{DataType, Field, Schema}, + datatypes::{DataType, Field}, error::Result, - record_batch::RecordBatch, }; use super::*; #[test] fn write_simple_rows() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("c1", DataType::Int32, false), - Field::new("c2", DataType::Utf8, false), - ]); - let a = Int32Array::from([Some(1), Some(2), Some(3), None, Some(5)]); let b = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c"), Some("d"), None]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)]).unwrap(); + let batch = Chunk::try_new(vec![&a as &dyn Array, &b]).unwrap(); - let buf = write_batch(batch, json_write::LineDelimited::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string(), "c2".to_string()], + json_write::LineDelimited::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap(), @@ -39,17 +37,16 @@ fn write_simple_rows() -> Result<()> { #[test] fn write_simple_rows_array() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("c1", DataType::Int32, false), - Field::new("c2", DataType::Utf8, false), - ]); - let a = Int32Array::from([Some(1), Some(2), Some(3), None, Some(5)]); let b = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c"), Some("d"), None]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)]).unwrap(); + let batch = Chunk::try_new(vec![&a as &dyn Array, &b]).unwrap(); - let buf = write_batch(batch, json_write::JsonArray::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string(), "c2".to_string()], + json_write::JsonArray::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap(), @@ -68,10 +65,6 @@ fn write_nested_struct_with_validity() -> Result<()> { Field::new("c11", DataType::Int32, false), Field::new("c12", DataType::Struct(inner.clone()), false), ]; - let schema = Schema::new(vec![ - Field::new("c1", DataType::Struct(fields.clone()), false), - Field::new("c2", DataType::Utf8, false), - ]); let c1 = StructArray::from_data( DataType::Struct(fields), @@ -90,9 +83,13 @@ fn write_nested_struct_with_validity() -> Result<()> { ); let c2 = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c")]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(c1), Arc::new(c2)]).unwrap(); + let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); - let buf = write_batch(batch, json_write::LineDelimited::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string(), "c2".to_string()], + json_write::LineDelimited::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap(), @@ -111,10 +108,6 @@ fn write_nested_structs() -> Result<()> { Field::new("c11", DataType::Int32, false), Field::new("c12", DataType::Struct(vec![c121.clone()]), false), ]; - let schema = Schema::new(vec![ - Field::new("c1", DataType::Struct(fields.clone()), false), - Field::new("c2", DataType::Utf8, false), - ]); let c1 = StructArray::from_data( DataType::Struct(fields), @@ -135,9 +128,13 @@ fn write_nested_structs() -> Result<()> { let c2 = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c")]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(c1), Arc::new(c2)]).unwrap(); + let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); - let buf = write_batch(batch, json_write::LineDelimited::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string(), "c2".to_string()], + json_write::LineDelimited::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap(), @@ -151,11 +148,6 @@ fn write_nested_structs() -> Result<()> { #[test] fn write_struct_with_list_field() -> Result<()> { - let list_datatype = DataType::List(Box::new(Field::new("c_list", DataType::Utf8, false))); - let field_c1 = Field::new("c1", list_datatype, false); - let field_c2 = Field::new("c2", DataType::Int32, false); - let schema = Schema::new(vec![field_c1, field_c2]); - let iter = vec![vec!["a", "a1"], vec!["b"], vec!["c"], vec!["d"], vec!["e"]]; let iter = iter @@ -172,9 +164,13 @@ fn write_struct_with_list_field() -> Result<()> { let b = PrimitiveArray::from_slice([1, 2, 3, 4, 5]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)]).unwrap(); + let batch = Chunk::try_new(vec![&a as &dyn Array, &b]).unwrap(); - let buf = write_batch(batch, json_write::LineDelimited::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string(), "c2".to_string()], + json_write::LineDelimited::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap(), @@ -190,12 +186,6 @@ fn write_struct_with_list_field() -> Result<()> { #[test] fn write_nested_list() -> Result<()> { - let list_inner = DataType::List(Box::new(Field::new("b", DataType::Int32, false))); - let list_datatype = DataType::List(Box::new(Field::new("a", list_inner, false))); - let field_c1 = Field::new("c1", list_datatype, true); - let field_c2 = Field::new("c2", DataType::Utf8, true); - let schema = Schema::new(vec![field_c1, field_c2]); - let iter = vec![ vec![Some(vec![Some(1), Some(2)]), Some(vec![Some(3)])], vec![], @@ -218,9 +208,13 @@ fn write_nested_list() -> Result<()> { let c2 = Utf8Array::::from(&vec![Some("foo"), Some("bar"), None]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(c1), Arc::new(c2)]).unwrap(); + let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); - let buf = write_batch(batch, json_write::LineDelimited::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string(), "c2".to_string()], + json_write::LineDelimited::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap(), @@ -244,9 +238,6 @@ fn write_list_of_struct() -> Result<()> { DataType::Struct(fields.clone()), false, ))); - let field_c1 = Field::new("c1", c1_datatype.clone(), true); - let field_c2 = Field::new("c2", DataType::Int32, false); - let schema = Schema::new(vec![field_c1, field_c2]); let s = StructArray::from_data( DataType::Struct(fields), @@ -278,9 +269,13 @@ fn write_list_of_struct() -> Result<()> { let c2 = Int32Array::from_slice(&[1, 2, 3]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(c1), Arc::new(c2)]).unwrap(); + let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); - let buf = write_batch(batch, json_write::LineDelimited::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string(), "c2".to_string()], + json_write::LineDelimited::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap(), @@ -294,12 +289,15 @@ fn write_list_of_struct() -> Result<()> { #[test] fn write_escaped_utf8() -> Result<()> { - let schema = Schema::new(vec![Field::new("c1", DataType::Utf8, false)]); let a = Utf8Array::::from(&vec![Some("a\na"), None]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap(); + let batch = Chunk::try_new(vec![&a as &dyn Array]).unwrap(); - let buf = write_batch(batch, json_write::LineDelimited::default())?; + let buf = write_batch( + batch, + vec!["c1".to_string()], + json_write::LineDelimited::default(), + )?; assert_eq!( String::from_utf8(buf).unwrap().as_bytes(), diff --git a/tests/it/io/parquet/mod.rs b/tests/it/io/parquet/mod.rs index ada15b32615..4fc1a9a4781 100644 --- a/tests/it/io/parquet/mod.rs +++ b/tests/it/io/parquet/mod.rs @@ -3,9 +3,8 @@ use std::sync::Arc; use arrow2::error::ArrowError; use arrow2::{ - array::*, bitmap::Bitmap, buffer::Buffer, datatypes::*, error::Result, + array::*, bitmap::Bitmap, buffer::Buffer, chunk::Chunk, datatypes::*, error::Result, io::parquet::read::statistics::*, io::parquet::read::*, io::parquet::write::*, - record_batch::RecordBatch, }; use crate::io::ipc::read_gzip_json; @@ -621,7 +620,7 @@ pub fn pyarrow_struct_statistics(column: usize) -> Option> { } /// Round-trip with parquet using the same integration files used for IPC integration tests. -fn integration_write(schema: &Schema, batches: &[RecordBatch]) -> Result> { +fn integration_write(schema: &Schema, batches: &[Chunk>]) -> Result> { let options = WriteOptions { write_statistics: true, compression: Compression::Uncompressed, @@ -668,7 +667,9 @@ fn integration_write(schema: &Schema, batches: &[RecordBatch]) -> Result Ok(writer.into_inner()) } -fn integration_read(data: &[u8]) -> Result<(Arc, Vec)> { +type IntegrationRead = (Arc, Vec>>); + +fn integration_read(data: &[u8]) -> Result { let reader = Cursor::new(data); let reader = RecordReader::try_new(reader, None, None, None, None)?; let schema = reader.schema().clone(); @@ -719,10 +720,7 @@ fn arrow_type() -> Result<()> { Field::new("a1", dt1, true), Field::new("a2", array2.data_type().clone(), true), ]); - let batch = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(array), Arc::new(array2)], - )?; + let batch = Chunk::try_new(vec![Arc::new(array) as Arc, Arc::new(array2)])?; let r = integration_write(&schema, &[batch.clone()])?; diff --git a/tests/it/io/parquet/read.rs b/tests/it/io/parquet/read.rs index 800770248ec..ee4b7723b20 100644 --- a/tests/it/io/parquet/read.rs +++ b/tests/it/io/parquet/read.rs @@ -373,15 +373,13 @@ fn all_types() -> Result<()> { let batches = reader.collect::>>()?; assert_eq!(batches.len(), 1); - let result = batches[0] - .column(0) + let result = batches[0].columns()[0] .as_any() .downcast_ref::() .unwrap(); assert_eq!(result, &Int32Array::from_slice([4, 5, 6, 7, 2, 3, 0, 1])); - let result = batches[0] - .column(6) + let result = batches[0].columns()[6] .as_any() .downcast_ref::() .unwrap(); @@ -390,8 +388,7 @@ fn all_types() -> Result<()> { &Float32Array::from_slice([0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]) ); - let result = batches[0] - .column(9) + let result = batches[0].columns()[9] .as_any() .downcast_ref::>() .unwrap(); diff --git a/tests/it/io/parquet/write.rs b/tests/it/io/parquet/write.rs index 4feb8405d52..a6da48c41b9 100644 --- a/tests/it/io/parquet/write.rs +++ b/tests/it/io/parquet/write.rs @@ -1,7 +1,7 @@ use std::io::Cursor; +use arrow2::error::Result; use arrow2::io::parquet::write::*; -use arrow2::{error::Result, record_batch::RecordBatch}; use super::*; @@ -42,10 +42,7 @@ fn round_trip( let parquet_schema = to_parquet_schema(&schema)?; - let iter = vec![RecordBatch::try_new( - Arc::new(schema.clone()), - vec![array.clone()], - )]; + let iter = vec![Chunk::try_new(vec![array.clone()])]; let row_groups = RowGroupIterator::try_new(iter.into_iter(), &schema, options, vec![encoding])?; diff --git a/tests/it/io/print.rs b/tests/it/io/print.rs index b1fd5f5040c..68eccdcc32b 100644 --- a/tests/it/io/print.rs +++ b/tests/it/io/print.rs @@ -1,33 +1,23 @@ use std::sync::Arc; use arrow2::{ - array::*, bitmap::Bitmap, buffer::Buffer, datatypes::*, error::Result, io::print::*, - record_batch::RecordBatch, + array::*, + bitmap::Bitmap, + buffer::Buffer, + chunk::Chunk, + datatypes::{DataType, Field, TimeUnit, UnionMode}, + error::Result, + io::print::*, }; #[test] fn write_basics() -> Result<()> { - // define a schema. - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, true), - Field::new("b", DataType::Int32, true), - ])); - - // define data. - let batch = RecordBatch::try_new( - schema, - vec![ - Arc::new(Utf8Array::::from(vec![ - Some("a"), - Some("b"), - None, - Some("d"), - ])), - Arc::new(Int32Array::from(vec![Some(1), None, Some(10), Some(100)])), - ], - )?; - - let table = write(&[batch]); + let a = Utf8Array::::from(vec![Some("a"), Some("b"), None, Some("d")]); + let b = Int32Array::from(vec![Some(1), None, Some(10), Some(100)]); + + let batch = Chunk::try_new(vec![&a as &dyn Array, &b])?; + + let table = write(&[batch], &["a".to_string(), "b".to_string()]); let expected = vec![ "+---+-----+", @@ -49,23 +39,16 @@ fn write_basics() -> Result<()> { #[test] fn write_null() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Null, true), - ])); - let num_rows = 4; - let arrays = schema - .fields() + let arrays = [DataType::Utf8, DataType::Int32, DataType::Null] .iter() - .map(|f| new_null_array(f.data_type().clone(), num_rows).into()) + .map(|dt| new_null_array(dt.clone(), num_rows)) .collect(); // define data (null) - let batch = RecordBatch::try_new(schema, arrays)?; + let columns = Chunk::try_new(arrays)?; - let table = write(&[batch]); + let table = write(&[columns], &["a", "b", "c"]); let expected = vec![ "+---+---+---+", @@ -86,18 +69,14 @@ fn write_null() -> Result<()> { #[test] fn write_dictionary() -> Result<()> { - // define a schema. - let field_type = DataType::Dictionary(i32::KEY_TYPE, Box::new(DataType::Utf8), false); - let schema = Arc::new(Schema::new(vec![Field::new("d1", field_type, true)])); - let mut array = MutableDictionaryArray::>::new(); array.try_extend(vec![Some("one"), None, Some("three")])?; - let array = array.into_arc(); + let array = array.into_box(); - let batch = RecordBatch::try_new(schema, vec![array])?; + let batch = Chunk::new(vec![array]); - let table = write(&[batch]); + let table = write(&[batch], &["d1"]); let expected = vec![ "+-------+", @@ -118,17 +97,13 @@ fn write_dictionary() -> Result<()> { #[test] fn dictionary_validities() -> Result<()> { - // define a schema. - let field_type = DataType::Dictionary(i32::KEY_TYPE, Box::new(DataType::Int32), false); - let schema = Arc::new(Schema::new(vec![Field::new("d1", field_type, true)])); - let keys = PrimitiveArray::::from([Some(1), None, Some(0)]); let values = PrimitiveArray::::from([None, Some(10)]); let array = DictionaryArray::::from_data(keys, Arc::new(values)); - let batch = RecordBatch::try_new(schema, vec![Arc::new(array)])?; + let columns = Chunk::new(vec![&array as &dyn Array]); - let table = write(&[batch]); + let table = write(&[columns], &["d1"]); let expected = vec![ "+----+", "| d1 |", "+----+", "| 10 |", "| |", "| |", "+----+", @@ -146,16 +121,10 @@ fn dictionary_validities() -> Result<()> { /// formatting that array with `write` macro_rules! check_datetime { ($ty:ty, $datatype:expr, $value:expr, $EXPECTED_RESULT:expr) => { - let array = Arc::new(PrimitiveArray::<$ty>::from(&[Some($value), None]).to($datatype)); - - let schema = Arc::new(Schema::new(vec![Field::new( - "f", - array.data_type().clone(), - true, - )])); - let batch = RecordBatch::try_new(schema, vec![array]).unwrap(); + let array = PrimitiveArray::<$ty>::from(&[Some($value), None]).to($datatype); + let batch = Chunk::new(vec![&array as &dyn Array]); - let table = write(&[batch]); + let table = write(&[batch], &["f"]); let expected = $EXPECTED_RESULT; let actual: Vec<&str> = table.lines().collect(); @@ -362,11 +331,9 @@ fn write_struct() -> Result<()> { let array = StructArray::from_data(DataType::Struct(fields), values, validity); - let schema = Schema::new(vec![Field::new("a", array.data_type().clone(), true)]); + let columns = Chunk::new(vec![&array as &dyn Array]); - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)])?; - - let table = write(&[batch]); + let table = write(&[columns], &["a"]); let expected = vec![ "+--------------+", @@ -400,11 +367,9 @@ fn write_union() -> Result<()> { let array = UnionArray::from_data(data_type, types, fields, None); - let schema = Schema::new(vec![Field::new("a", array.data_type().clone(), true)]); - - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)])?; + let batch = Chunk::new(vec![&array as &dyn Array]); - let table = write(&[batch]); + let table = write(&[batch], &["a"]); let expected = vec![ "+---+", "| a |", "+---+", "| 1 |", "| |", "| c |", "+---+", diff --git a/tests/it/record_batch.rs b/tests/it/record_batch.rs deleted file mode 100644 index 8bda0484b88..00000000000 --- a/tests/it/record_batch.rs +++ /dev/null @@ -1,108 +0,0 @@ -use arrow2::array::*; -use arrow2::datatypes::*; -use arrow2::record_batch::RecordBatch; - -#[test] -fn basic() { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Utf8, false), - ]); - - let a = Int32Array::from_slice(&[1, 2, 3, 4, 5]); - let b = Utf8Array::::from_slice(&["a", "b", "c", "d", "e"]); - - let record_batch = - RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)]).unwrap(); - check_batch(record_batch) -} - -fn check_batch(record_batch: RecordBatch) { - assert_eq!(5, record_batch.num_rows()); - assert_eq!(2, record_batch.num_columns()); - assert_eq!(&DataType::Int32, record_batch.schema().field(0).data_type()); - assert_eq!(&DataType::Utf8, record_batch.schema().field(1).data_type()); - assert_eq!(5, record_batch.column(0).len()); - assert_eq!(5, record_batch.column(1).len()); -} - -#[test] -fn try_from_iter() { - let a: ArrayRef = Arc::new(Int32Array::from(vec![ - Some(1), - Some(2), - None, - Some(4), - Some(5), - ])); - let b: ArrayRef = Arc::new(Utf8Array::::from_slice(&["a", "b", "c", "d", "e"])); - - let record_batch = - RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).expect("valid conversion"); - - let expected_schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, false), - ]); - assert_eq!(record_batch.schema().as_ref(), &expected_schema); - check_batch(record_batch); -} - -#[test] -fn try_from_iter_with_nullable() { - let a: ArrayRef = Arc::new(Int32Array::from_slice(&[1, 2, 3, 4, 5])); - let b: ArrayRef = Arc::new(Utf8Array::::from_slice(&["a", "b", "c", "d", "e"])); - - // Note there are no nulls in a or b, but we specify that b is nullable - let record_batch = - RecordBatch::try_from_iter_with_nullable(vec![("a", a, false), ("b", b, true)]) - .expect("valid conversion"); - - let expected_schema = Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Utf8, true), - ]); - assert_eq!(record_batch.schema().as_ref(), &expected_schema); - check_batch(record_batch); -} - -#[test] -fn type_mismatch() { - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - - let a = Int64Array::from_slice(&[1, 2, 3, 4, 5]); - - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]); - assert!(batch.is_err()); -} - -#[test] -fn number_of_fields_mismatch() { - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - - let a = Int32Array::from_slice(&[1, 2, 3, 4, 5]); - let b = Int32Array::from_slice(&[1, 2, 3, 4, 5]); - - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)]); - assert!(batch.is_err()); -} - -#[test] -fn from_struct_array() { - let boolean = Arc::new(BooleanArray::from_slice(&[false, false, true, true])) as ArrayRef; - let int = Arc::new(Int32Array::from_slice(&[42, 28, 19, 31])) as ArrayRef; - - let fields = vec![ - Field::new("b", DataType::Boolean, false), - Field::new("c", DataType::Int32, false), - ]; - - let array = StructArray::from_data(fields.clone(), vec![boolean.clone(), int.clone()], None); - - let batch = RecordBatch::from(array); - assert_eq!(2, batch.num_columns()); - assert_eq!(4, batch.num_rows()); - assert_eq!(&fields, batch.schema().fields()); - assert_eq!(boolean.as_ref(), batch.column(0).as_ref()); - assert_eq!(int.as_ref(), batch.column(1).as_ref()); -}