diff --git a/.gitignore b/.gitignore index 39564fe2b50..c10f9e51df0 100644 --- a/.gitignore +++ b/.gitignore @@ -3,6 +3,7 @@ target-tarpaulin venv lcov.info Cargo.lock +example.arrow fixtures settings.json dev/ diff --git a/Cargo.toml b/Cargo.toml index db0e2e89bc6..d88a6c340f9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -30,6 +30,7 @@ simdutf8 = "0.1.3" # for csv io csv = { version = "^1.1", optional = true } +csv-core = { version = "0.1", optional = true } # for csv async io csv-async = { version = "^1.1", optional = true } @@ -76,7 +77,7 @@ avro-schema = { version = "0.2", optional = true } # compression of avro libflate = { version = "1.1.1", optional = true } snap = { version = "1", optional = true } -crc = { version = "1", optional = true } +crc = { version = "2", optional = true } # async avro async-stream = { version = "0.3.2", optional = true } @@ -129,7 +130,7 @@ io_csv = ["io_csv_read", "io_csv_write"] io_csv_async = ["io_csv_read_async"] io_csv_read = ["csv", "lexical-core"] io_csv_read_async = ["csv-async", "lexical-core", "futures"] -io_csv_write = ["csv", "streaming-iterator", "lexical-core"] +io_csv_write = ["csv", "csv-core", "streaming-iterator", "lexical-core"] io_json = ["serde", "serde_json", "streaming-iterator", "fallible-streaming-iterator", "indexmap", "lexical-core"] io_ipc = ["arrow-format"] io_ipc_write_async = ["io_ipc", "futures"] @@ -181,8 +182,7 @@ compute_substring = [] compute_take = [] compute_temporal = [] compute_window = ["compute_concatenate"] -compute_lower = [] -compute_upper = [] +compute_utf8 = [] compute = [ "compute_aggregate", "compute_arithmetics", @@ -207,12 +207,12 @@ compute = [ "compute_substring", "compute_take", "compute_temporal", - "compute_window", - "compute_lower", - "compute_upper" + "compute_utf8", + "compute_window" ] benchmarks = ["rand"] simd = ["packed_simd"] +serde_types = ["serde", "serde_derive"] [package.metadata.cargo-all-features] allowlist = ["compute", "compute_sort", "compute_hash", "compute_nullif"] @@ -249,6 +249,7 @@ harness = false name = "comparison_kernels" harness = false + [[bench]] name = "read_parquet" harness = false diff --git a/README.md b/README.md index 8da9509ce39..e54883f6eed 100644 --- a/README.md +++ b/README.md @@ -20,7 +20,8 @@ documentation of each of its APIs. * Most feature-complete implementation of Apache Arrow after the reference implementation (C++) * Float 16 unsupported (not a Rust native type) * Decimal 256 unsupported (not a Rust native type) -* FFI supported for all Arrow types +* C data interface supported for all Arrow types (read and write) +* C stream interface supported for all Arrow types (read and write) * Full interoperability with Rust's `Vec` * MutableArray API to work with bitmaps and arrays in-place * Full support for timestamps with timezones, including arithmetics that take diff --git a/arrow-pyarrow-integration-testing/README.md b/arrow-pyarrow-integration-testing/README.md index 2d1bdca4b94..beb18618e5d 100644 --- a/arrow-pyarrow-integration-testing/README.md +++ b/arrow-pyarrow-integration-testing/README.md @@ -31,7 +31,7 @@ Note that this crate uses two languages and an external ABI: Pyarrow exposes a C ABI to convert arrow arrays from and to its C implementation, see [here](https://arrow.apache.org/docs/format/CDataInterface.html). -This package uses the equivalent struct in Rust (`arrow::array::ArrowArray`), and verifies that +This package uses the equivalent struct in Rust (`arrow::array::InternalArrowArray`), and verifies that we can use pyarrow's interface to move pointers from and to Rust. ## Relevant literature diff --git a/arrow-pyarrow-integration-testing/pyproject.toml b/arrow-pyarrow-integration-testing/pyproject.toml index 27480690e06..5c143694e18 100644 --- a/arrow-pyarrow-integration-testing/pyproject.toml +++ b/arrow-pyarrow-integration-testing/pyproject.toml @@ -16,5 +16,5 @@ # under the License. [build-system] -requires = ["maturin"] +requires = ["maturin>=0.12,<0.13"] build-backend = "maturin" diff --git a/arrow-pyarrow-integration-testing/src/c_stream.rs b/arrow-pyarrow-integration-testing/src/c_stream.rs new file mode 100644 index 00000000000..efdf224c1c6 --- /dev/null +++ b/arrow-pyarrow-integration-testing/src/c_stream.rs @@ -0,0 +1,63 @@ +//! This library demonstrates a minimal usage of Rust's C data interface to pass +//! arrays from and to Python. +use pyo3::ffi::Py_uintptr_t; +use pyo3::prelude::*; + +use arrow2::array::{Int32Array, StructArray}; +use arrow2::datatypes::DataType; +use arrow2::ffi; + +use super::*; + +pub fn to_rust_iterator(ob: PyObject, py: Python) -> PyResult> { + let stream = Box::new(ffi::ArrowArrayStream::empty()); + + let stream_ptr = &*stream as *const ffi::ArrowArrayStream; + + // make the conversion through PyArrow's private API + // this changes the pointer's memory and is thus unsafe. In particular, `_export_to_c` can go out of bounds + ob.call_method1(py, "_export_to_c", (stream_ptr as Py_uintptr_t,))?; + + let mut iter = + unsafe { ffi::ArrowArrayStreamReader::try_new(stream).map_err(PyO3ArrowError::from) }?; + + let mut arrays = vec![]; + while let Some(array) = unsafe { iter.next() } { + let py_array = to_py_array(array.unwrap().into(), py)?; + arrays.push(py_array) + } + Ok(arrays) +} + +pub fn from_rust_iterator(py: Python) -> PyResult { + // initialize an array + let array = Int32Array::from(&[Some(2), None, Some(1), None]); + let array = StructArray::from_data( + DataType::Struct(vec![Field::new("a", array.data_type().clone(), true)]), + vec![Arc::new(array)], + None, + ); + // and a field with its datatype + let field = Field::new("a", array.data_type().clone(), true); + + // Arc it, since it will be shared with an external program + let array: Arc = Arc::new(array.clone()); + + // create an iterator of arrays + let arrays = vec![array.clone(), array.clone(), array]; + let iter = Box::new(arrays.clone().into_iter().map(Ok)) as _; + + // create an [`ArrowArrayStream`] based on this iterator and field + let mut stream = Box::new(ffi::ArrowArrayStream::empty()); + unsafe { ffi::export_iterator(iter, field, &mut *stream) }; + + // call pyarrow's interface to read this stream + let pa = py.import("pyarrow.ipc")?; + let py_stream = pa.getattr("RecordBatchReader")?.call_method1( + "_import_from_c", + ((&*stream as *const ffi::ArrowArrayStream) as Py_uintptr_t,), + )?; + Box::leak(stream); // this is not ideal => the struct should be allocated by pyarrow + + Ok(py_stream.to_object(py)) +} diff --git a/arrow-pyarrow-integration-testing/src/lib.rs b/arrow-pyarrow-integration-testing/src/lib.rs index 48d39adcd61..9b18dab716c 100644 --- a/arrow-pyarrow-integration-testing/src/lib.rs +++ b/arrow-pyarrow-integration-testing/src/lib.rs @@ -1,5 +1,6 @@ //! This library demonstrates a minimal usage of Rust's C data interface to pass //! arrays from and to Python. +mod c_stream; use std::error; use std::fmt; @@ -51,11 +52,11 @@ impl From for PyErr { fn to_rust_array(ob: PyObject, py: Python) -> PyResult> { // prepare a pointer to receive the Array struct - let array = Box::new(ffi::Ffi_ArrowArray::empty()); - let schema = Box::new(ffi::Ffi_ArrowSchema::empty()); + let array = Box::new(ffi::ArrowArray::empty()); + let schema = Box::new(ffi::ArrowSchema::empty()); - let array_ptr = &*array as *const ffi::Ffi_ArrowArray; - let schema_ptr = &*schema as *const ffi::Ffi_ArrowSchema; + let array_ptr = &*array as *const ffi::ArrowArray; + let schema_ptr = &*schema as *const ffi::ArrowSchema; // make the conversion through PyArrow's private API // this changes the pointer's memory and is thus unsafe. In particular, `_export_to_c` can go out of bounds @@ -66,14 +67,15 @@ fn to_rust_array(ob: PyObject, py: Python) -> PyResult> { )?; let field = unsafe { ffi::import_field_from_c(schema.as_ref()).map_err(PyO3ArrowError::from)? }; - let array = unsafe { ffi::import_array_from_c(array, &field).map_err(PyO3ArrowError::from)? }; + let array = + unsafe { ffi::import_array_from_c(array, field.data_type).map_err(PyO3ArrowError::from)? }; Ok(array.into()) } fn to_py_array(array: Arc, py: Python) -> PyResult { - let array_ptr = Box::new(ffi::Ffi_ArrowArray::empty()); - let schema_ptr = Box::new(ffi::Ffi_ArrowSchema::empty()); + let array_ptr = Box::new(ffi::ArrowArray::empty()); + let schema_ptr = Box::new(ffi::ArrowSchema::empty()); let array_ptr = Box::into_raw(array_ptr); let schema_ptr = Box::into_raw(schema_ptr); @@ -100,9 +102,9 @@ fn to_py_array(array: Arc, py: Python) -> PyResult { fn to_rust_field(ob: PyObject, py: Python) -> PyResult { // prepare a pointer to receive the Array struct - let schema = Box::new(ffi::Ffi_ArrowSchema::empty()); + let schema = Box::new(ffi::ArrowSchema::empty()); - let schema_ptr = &*schema as *const ffi::Ffi_ArrowSchema; + let schema_ptr = &*schema as *const ffi::ArrowSchema; // make the conversion through PyArrow's private API // this changes the pointer's memory and is thus unsafe. In particular, `_export_to_c` can go out of bounds @@ -114,7 +116,7 @@ fn to_rust_field(ob: PyObject, py: Python) -> PyResult { } fn to_py_field(field: &Field, py: Python) -> PyResult { - let schema_ptr = Box::new(ffi::Ffi_ArrowSchema::empty()); + let schema_ptr = Box::new(ffi::ArrowSchema::empty()); let schema_ptr = Box::into_raw(schema_ptr); unsafe { @@ -152,9 +154,21 @@ fn round_trip_field(array: PyObject, py: Python) -> PyResult { to_py_field(&field, py) } +#[pyfunction] +pub fn to_rust_iterator(ob: PyObject, py: Python) -> PyResult> { + c_stream::to_rust_iterator(ob, py) +} + +#[pyfunction] +pub fn from_rust_iterator(py: Python) -> PyResult { + c_stream::from_rust_iterator(py) +} + #[pymodule] fn arrow_pyarrow_integration_testing(_py: Python, m: &PyModule) -> PyResult<()> { m.add_function(wrap_pyfunction!(round_trip_array, m)?)?; m.add_function(wrap_pyfunction!(round_trip_field, m)?)?; + m.add_function(wrap_pyfunction!(to_rust_iterator, m)?)?; + m.add_function(wrap_pyfunction!(from_rust_iterator, m)?)?; Ok(()) } diff --git a/arrow-pyarrow-integration-testing/tests/test_c_stream.py b/arrow-pyarrow-integration-testing/tests/test_c_stream.py new file mode 100644 index 00000000000..74f3c6f7209 --- /dev/null +++ b/arrow-pyarrow-integration-testing/tests/test_c_stream.py @@ -0,0 +1,29 @@ +import unittest + +import pyarrow.ipc + +import arrow_pyarrow_integration_testing + + +class TestCase(unittest.TestCase): + def test_rust_reads(self): + schema = pyarrow.schema([pyarrow.field("aa", pyarrow.int32())]) + a = pyarrow.array([1, None, 2], type=pyarrow.int32()) + + batch = pyarrow.record_batch([a], schema) + reader = pyarrow.ipc.RecordBatchStreamReader.from_batches(schema, [batch]) + + arrays = arrow_pyarrow_integration_testing.to_rust_iterator(reader) + + array = arrays[0].field(0) + assert array == a + + def test_pyarrow_reads(self): + stream = arrow_pyarrow_integration_testing.from_rust_iterator() + + arrays = [a for a in stream] + + expected = pyarrow.RecordBatch.from_arrays([pyarrow.array([2, None, 1, None], pyarrow.int32())], names=["a"]) + expected = [expected, expected, expected] + + self.assertEqual(arrays, expected) diff --git a/benches/avro_read.rs b/benches/avro_read.rs index 029d3ab7d6b..9143db9e7e0 100644 --- a/benches/avro_read.rs +++ b/benches/avro_read.rs @@ -24,12 +24,11 @@ fn schema() -> AvroSchema { fn write(size: usize, has_codec: bool) -> Result> { let avro = schema(); // a writer needs a schema and something to write to - let mut writer: Writer>; - if has_codec { - writer = Writer::with_codec(&avro, Vec::new(), Codec::Deflate); + let mut writer = if has_codec { + Writer::with_codec(&avro, Vec::new(), Codec::Deflate) } else { - writer = Writer::new(&avro, Vec::new()); - } + Writer::new(&avro, Vec::new()) + }; (0..size).for_each(|_| { let mut record = Record::new(writer.schema()).unwrap(); diff --git a/benches/filter_kernels.rs b/benches/filter_kernels.rs index ef9c725d60e..921950595f6 100644 --- a/benches/filter_kernels.rs +++ b/benches/filter_kernels.rs @@ -94,12 +94,21 @@ fn add_benchmark(c: &mut Criterion) { }); let data_array = create_primitive_array::(size, 0.5); + let data_array_nonull = create_primitive_array::(size, 0.0); c.bench_function("filter f32", |b| { b.iter(|| bench_filter(&data_array, &filter_array)) }); c.bench_function("filter f32 high selectivity", |b| { b.iter(|| bench_filter(&data_array, &dense_filter_array)) }); + + c.bench_function("filter f32 nonull", |b| { + b.iter(|| bench_filter(&data_array_nonull, &filter_array)) + }); + c.bench_function("filter f32 nonull high selectivity", |b| { + b.iter(|| bench_filter(&data_array_nonull, &dense_filter_array)) + }); + c.bench_function("filter context f32", |b| { b.iter(|| bench_built_filter(&filter, &data_array)) }); diff --git a/benches/write_csv.rs b/benches/write_csv.rs index 0d341cea9d7..faba0701c65 100644 --- a/benches/write_csv.rs +++ b/benches/write_csv.rs @@ -11,13 +11,13 @@ use arrow2::util::bench_util::*; type ChunkArc = Chunk>; fn write_batch(columns: &ChunkArc) -> Result<()> { - let writer = &mut write::WriterBuilder::new().from_writer(vec![]); + let mut writer = vec![]; assert_eq!(columns.arrays().len(), 1); - write::write_header(writer, &["a"])?; - let options = write::SerializeOptions::default(); - write::write_chunk(writer, columns, &options) + write::write_header(&mut writer, &["a"], &options)?; + + write::write_chunk(&mut writer, columns, &options) } fn make_chunk(array: impl Array + 'static) -> Chunk> { diff --git a/benches/write_json.rs b/benches/write_json.rs index 11f550c90f5..9f52140303c 100644 --- a/benches/write_json.rs +++ b/benches/write_json.rs @@ -1,55 +1,44 @@ -use std::sync::Arc; - use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::*; -use arrow2::chunk::Chunk; -use arrow2::error::Result; +use arrow2::error::ArrowError; use arrow2::io::json::write; use arrow2::util::bench_util::*; -fn write_batch(columns: &Chunk>) -> Result<()> { +fn write_array(array: Box) -> Result<(), ArrowError> { let mut writer = vec![]; - let format = write::JsonArray::default(); - let batches = vec![Ok(columns.clone())].into_iter(); + let arrays = vec![Ok(array)].into_iter(); - // Advancing this iterator serializes the next batch to its internal buffer (i.e. CPU-bounded) - let blocks = write::Serializer::new(batches, vec!["c1".to_string()], vec![], format); + // Advancing this iterator serializes the next array to its internal buffer (i.e. CPU-bounded) + let blocks = write::Serializer::new(arrays, vec![]); // the operation of writing is IO-bounded. - write::write(&mut writer, format, blocks)?; + write::write(&mut writer, blocks)?; Ok(()) } -fn make_chunk(array: impl Array + 'static) -> Chunk> { - Chunk::new(vec![Arc::new(array) as Arc]) -} - fn add_benchmark(c: &mut Criterion) { (10..=18).step_by(2).for_each(|log2_size| { let size = 2usize.pow(log2_size); let array = create_primitive_array::(size, 0.1); - let columns = make_chunk(array); c.bench_function(&format!("json write i32 2^{}", log2_size), |b| { - b.iter(|| write_batch(&columns)) + b.iter(|| write_array(Box::new(array.clone()))) }); let array = create_string_array::(size, 100, 0.1, 42); - let columns = make_chunk(array); c.bench_function(&format!("json write utf8 2^{}", log2_size), |b| { - b.iter(|| write_batch(&columns)) + b.iter(|| write_array(Box::new(array.clone()))) }); let array = create_primitive_array::(size, 0.1); - let columns = make_chunk(array); c.bench_function(&format!("json write f64 2^{}", log2_size), |b| { - b.iter(|| write_batch(&columns)) + b.iter(|| write_array(Box::new(array.clone()))) }); }); } diff --git a/benches/write_parquet.rs b/benches/write_parquet.rs index 3709aedba65..32b264bfe53 100644 --- a/benches/write_parquet.rs +++ b/benches/write_parquet.rs @@ -1,4 +1,3 @@ -use std::io::Cursor; use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; @@ -29,7 +28,7 @@ fn write(array: &dyn Array, encoding: Encoding) -> Result<()> { vec![encoding], )?; - let mut writer = vec![]; + let writer = vec![]; let mut writer = FileWriter::try_new(writer, schema, options)?; diff --git a/examples/csv_write.rs b/examples/csv_write.rs index f9f73cfef53..6a40fb7b515 100644 --- a/examples/csv_write.rs +++ b/examples/csv_write.rs @@ -6,14 +6,14 @@ use arrow2::{ }; fn write_batch>(path: &str, columns: &[Chunk]) -> Result<()> { - let writer = &mut write::WriterBuilder::new().from_path(path)?; - - write::write_header(writer, &["c1"])?; + let mut writer = std::fs::File::create(path)?; let options = write::SerializeOptions::default(); + write::write_header(&mut writer, &["c1"], &options)?; + columns .iter() - .try_for_each(|batch| write::write_chunk(writer, batch, &options)) + .try_for_each(|batch| write::write_chunk(&mut writer, batch, &options)) } fn main() -> Result<()> { diff --git a/examples/csv_write_parallel.rs b/examples/csv_write_parallel.rs index f616a1b8292..5611d39b1b3 100644 --- a/examples/csv_write_parallel.rs +++ b/examples/csv_write_parallel.rs @@ -1,3 +1,4 @@ +use std::io::Write; use std::sync::mpsc; use std::sync::mpsc::{Receiver, Sender}; use std::sync::Arc; @@ -14,8 +15,8 @@ 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, &["c1"])?; + let mut writer = std::fs::File::create(path)?; + write::write_header(&mut writer, &["c1"], &options)?; // prepare a channel to send serialized records from threads let (tx, rx): (Sender<_>, Receiver<_>) = mpsc::channel(); @@ -28,8 +29,8 @@ fn parallel_write(path: &str, batches: [Chunk>; 2]) -> Result<()> let options = options.clone(); let batch = batches[id].clone(); // note: this is cheap let child = thread::spawn(move || { - let records = write::serialize(&batch, &options).unwrap(); - thread_tx.send(records).unwrap(); + let rows = write::serialize(&batch, &options).unwrap(); + thread_tx.send(rows).unwrap(); }); children.push(child); @@ -38,9 +39,7 @@ fn parallel_write(path: &str, batches: [Chunk>; 2]) -> Result<()> for _ in 0..2 { // block: assumes that the order of batches matter. let records = rx.recv().unwrap(); - records - .iter() - .try_for_each(|record| writer.write_byte_record(record))? + records.iter().try_for_each(|row| writer.write_all(&row))? } for child in children { diff --git a/examples/ffi.rs b/examples/ffi.rs index 87933385659..d2f3c68fc28 100644 --- a/examples/ffi.rs +++ b/examples/ffi.rs @@ -6,20 +6,18 @@ use std::sync::Arc; unsafe fn export( array: Arc, - array_ptr: *mut ffi::Ffi_ArrowArray, - schema_ptr: *mut ffi::Ffi_ArrowSchema, + array_ptr: *mut ffi::ArrowArray, + schema_ptr: *mut ffi::ArrowSchema, ) { + // exporting an array requires an associated field so that the consumer knows its datatype let field = Field::new("a", array.data_type().clone(), true); ffi::export_array_to_c(array, array_ptr); ffi::export_field_to_c(&field, schema_ptr); } -unsafe fn import( - array: Box, - schema: &ffi::Ffi_ArrowSchema, -) -> Result> { +unsafe fn import(array: Box, schema: &ffi::ArrowSchema) -> Result> { let field = ffi::import_field_from_c(schema)?; - ffi::import_array_from_c(array, &field) + ffi::import_array_from_c(array, field.data_type) } fn main() -> Result<()> { @@ -28,23 +26,13 @@ fn main() -> Result<()> { // the goal is to export this array and import it back via FFI. // to import, we initialize the structs that will receive the data - let array_ptr = Box::new(ffi::Ffi_ArrowArray::empty()); - let schema_ptr = Box::new(ffi::Ffi_ArrowSchema::empty()); - - // since FFIs work in raw pointers, let's temporarily relinquish ownership so that producers - // can write into it in a thread-safe manner - let array_ptr = Box::into_raw(array_ptr); - let schema_ptr = Box::into_raw(schema_ptr); + let mut array_ptr = Box::new(ffi::ArrowArray::empty()); + let mut schema_ptr = Box::new(ffi::ArrowSchema::empty()); // this is where a producer (in this case also us ^_^) writes to the pointers' location. // `array` here could be anything or not even be available, if this was e.g. from Python. - // Safety: we just allocated the pointers correctly. - unsafe { export(array.clone(), array_ptr, schema_ptr) }; - - // we can now take ownership back, since we are responsible for deallocating this memory. - // Safety: we just into_raw them. - let array_ptr = unsafe { Box::from_raw(array_ptr) }; - let schema_ptr = unsafe { Box::from_raw(schema_ptr) }; + // Safety: we just allocated the pointers + unsafe { export(array.clone(), &mut *array_ptr, &mut *schema_ptr) }; // and finally interpret the written memory into a new array. // Safety: we used `export`, which is a valid exporter to the C data interface diff --git a/examples/json_read.rs b/examples/json_read.rs index b6a25a74ef6..1630aa65047 100644 --- a/examples/json_read.rs +++ b/examples/json_read.rs @@ -3,23 +3,17 @@ use std::io::BufReader; use std::sync::Arc; use arrow2::array::Array; -use arrow2::error::{ArrowError, Result}; +use arrow2::error::Result; use arrow2::io::json::read; fn read_path(path: &str) -> Result> { // Example of reading a JSON file. let reader = BufReader::new(File::open(path)?); - let data = serde_json::from_reader(reader)?; + let json = serde_json::from_reader(reader)?; - let values = if let serde_json::Value::Array(values) = data { - Ok(values) - } else { - Err(ArrowError::InvalidArgumentError("".to_string())) - }?; + let data_type = read::infer(&json)?; - let data_type = read::infer_rows(&values)?; - - Ok(read::deserialize_json(&values, data_type)) + read::deserialize(&json, data_type) } fn main() -> Result<()> { diff --git a/examples/json_write.rs b/examples/json_write.rs index 07213fc3ddc..fc5191de51a 100644 --- a/examples/json_write.rs +++ b/examples/json_write.rs @@ -1,42 +1,32 @@ use std::fs::File; -use std::sync::Arc; use arrow2::{ array::{Array, Int32Array}, - chunk::Chunk, - error::Result, + error::ArrowError, io::json::write, }; -fn write_batches(path: &str, names: Vec, batches: &[Chunk>]) -> Result<()> { +fn write_array(path: &str, array: Box) -> Result<(), ArrowError> { let mut writer = File::create(path)?; - let format = write::JsonArray::default(); - let batches = batches.iter().cloned().map(Ok); + let arrays = vec![Ok(array)].into_iter(); - // Advancing this iterator serializes the next batch to its internal buffer (i.e. CPU-bounded) - let blocks = write::Serializer::new(batches, names, vec![], format); + // Advancing this iterator serializes the next array to its internal buffer (i.e. CPU-bounded) + let blocks = write::Serializer::new(arrays, vec![]); // the operation of writing is IO-bounded. - write::write(&mut writer, format, blocks)?; + write::write(&mut writer, blocks)?; Ok(()) } -fn main() -> Result<()> { - 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])], - ) +fn main() -> Result<(), ArrowError> { + use std::env; + let args: Vec = env::args().collect(); + + let file_path = &args[1]; + + let array = Int32Array::from(&[Some(0), None, Some(2), Some(3), Some(4), Some(5), Some(6)]); + + write_array(file_path, Box::new(array)) } diff --git a/examples/ndjson_read.rs b/examples/ndjson_read.rs index 1df6e2b6e59..0490b23e2f1 100644 --- a/examples/ndjson_read.rs +++ b/examples/ndjson_read.rs @@ -1,48 +1,40 @@ use std::fs::File; -use std::io::BufReader; +use std::io::{BufReader, Seek}; use std::sync::Arc; use arrow2::array::Array; -use arrow2::chunk::Chunk; use arrow2::error::Result; -use arrow2::io::json::read; +use arrow2::io::ndjson::read; +use arrow2::io::ndjson::read::FallibleStreamingIterator; -fn read_path(path: &str, projection: Option>) -> Result>> { - // Example of reading a NDJSON file. +fn read_path(path: &str) -> Result>> { + let batch_size = 1024; // number of rows per array let mut reader = BufReader::new(File::open(path)?); - let fields = read::infer_and_reset(&mut reader, None)?; - - let fields = if let Some(projection) = projection { - fields - .into_iter() - .filter(|field| projection.contains(&field.name.as_ref())) - .collect() - } else { - fields - }; - - // at most 1024 rows. This container can be re-used across batches. - let mut rows = vec![String::default(); 1024]; - - // Reads up to 1024 rows. - // this is IO-intensive and performs minimal CPU work. In particular, - // no deserialization is performed. - let read = read::read_rows(&mut reader, &mut rows)?; - let rows = &rows[..read]; - - // 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) + let data_type = read::infer(&mut reader, None)?; + reader.rewind()?; + + let mut reader = read::FileReader::new(reader, vec!["".to_string(); batch_size], None); + + let mut arrays = vec![]; + // `next` is IO-bounded + while let Some(rows) = reader.next()? { + // `deserialize` is CPU-bounded + let array = read::deserialize(rows, data_type.clone())?; + arrays.push(array); + } + + Ok(arrays) } fn main() -> Result<()> { + // Example of reading a NDJSON file from a path use std::env; let args: Vec = env::args().collect(); let file_path = &args[1]; - let batch = read_path(file_path, None)?; - println!("{:#?}", batch); + let arrays = read_path(file_path)?; + println!("{:#?}", arrays); Ok(()) } diff --git a/examples/ndjson_write.rs b/examples/ndjson_write.rs new file mode 100644 index 00000000000..91a0e1a9ed7 --- /dev/null +++ b/examples/ndjson_write.rs @@ -0,0 +1,35 @@ +use std::fs::File; + +use arrow2::array::{Array, Int32Array}; +use arrow2::error::Result; +use arrow2::io::ndjson::write; + +fn write_path(path: &str, array: Box) -> Result<()> { + let writer = File::create(path)?; + + let serializer = write::Serializer::new(vec![Ok(array)].into_iter(), vec![]); + + let mut writer = write::FileWriter::new(writer, serializer); + writer.by_ref().collect::>() +} + +fn main() -> Result<()> { + // Example of reading a NDJSON file from a path + use std::env; + let args: Vec = env::args().collect(); + + let file_path = &args[1]; + + let array = Box::new(Int32Array::from(&[ + Some(0), + None, + Some(2), + Some(3), + Some(4), + Some(5), + Some(6), + ])); + + write_path(file_path, array)?; + Ok(()) +} diff --git a/examples/parquet_read_async.rs b/examples/parquet_read_async.rs index e9ac530bb89..65a802b0008 100644 --- a/examples/parquet_read_async.rs +++ b/examples/parquet_read_async.rs @@ -2,8 +2,6 @@ use std::sync::Arc; use std::time::SystemTime; use futures::future::BoxFuture; -use futures::FutureExt; -use tokio; use tokio::fs::File; use tokio::io::BufReader; use tokio_util::compat::*; diff --git a/guide/src/README.md b/guide/src/README.md index 3042526a2e8..de2cd418bea 100644 --- a/guide/src/README.md +++ b/guide/src/README.md @@ -5,8 +5,10 @@ interoperability with the arrow format. The typical use-case for this library is to perform CPU and memory-intensive analytics in a format that supports heterogeneous data structures, null values, and IPC and FFI interfaces across languages. -Arrow2 is divided into three main parts: +Arrow2 is divided into 5 main parts: * a [low-level API](./low_level.md) to efficiently operate with contiguous memory regions; * a [high-level API](./high_level.md) to operate with arrow arrays; -* a [metadata API](./metadata.md) to declare and operate with logical types and metadata. +* a [metadata API](./metadata.md) to declare and operate with logical types and metadata; +* a [compute API](./compute.md) with operators to operate over arrays; +* an [IO API](./io/README.md) with interfaces to read from, and write to, other formats. diff --git a/guide/src/io/json_read.md b/guide/src/io/json_read.md index cd3a19f4b22..41c7a71d77d 100644 --- a/guide/src/io/json_read.md +++ b/guide/src/io/json_read.md @@ -14,3 +14,16 @@ This crate also supports reading JSON, at the expense of being unable to read th ```rust {{#include ../../../examples/json_read.rs}} ``` + +## Metadata and inference + +This crate uses the following mapping between Arrow's data type and JSON: + +| `JSON` | `DataType` | +| ------ | ---------- | +| Bool | Boolean | +| Int | Int64 | +| Float | Float64 | +| String | Utf8 | +| List | List | +| Object | Struct | diff --git a/guide/src/io/json_write.md b/guide/src/io/json_write.md index b237e014eca..2483a056d00 100644 --- a/guide/src/io/json_write.md +++ b/guide/src/io/json_write.md @@ -1,8 +1,14 @@ # Write JSON -When compiled with feature `io_json`, you can use this crate to write JSON files. -The following example writes a batch as a JSON file: +When compiled with feature `io_json`, you can use this crate to write JSON. +The following example writes an array to JSON: ```rust {{#include ../../../examples/json_write.rs}} ``` + +Likewise, you can also use it to write to NDJSON: + +```rust +{{#include ../../../examples/ndjson_write.rs}} +``` diff --git a/parquet_integration/write_parquet.py b/parquet_integration/write_parquet.py index bf2e39cb678..6c10f8b5f00 100644 --- a/parquet_integration/write_parquet.py +++ b/parquet_integration/write_parquet.py @@ -32,6 +32,7 @@ def case_basic_nullable(size=1): pa.field("timestamp_us", pa.timestamp("us")), pa.field("timestamp_s", pa.timestamp("s")), pa.field("emoji", pa.utf8()), + pa.field("timestamp_s_utc", pa.timestamp("s", "UTC")), ] schema = pa.schema(fields) @@ -50,6 +51,7 @@ def case_basic_nullable(size=1): "timestamp_us": int64 * size, "timestamp_s": int64 * size, "emoji": emoji * size, + "timestamp_s_utc": int64 * size, }, schema, f"basic_nullable_{size*10}.parquet", diff --git a/src/array/README.md b/src/array/README.md index a814839527c..ca5376c214c 100644 --- a/src/array/README.md +++ b/src/array/README.md @@ -16,17 +16,16 @@ This document describes the overall design of this module. * An array with a null bitmap MUST implement it as `Option` -* An array MUST be `#[derive(Debug, Clone)]` +* An array MUST be `#[derive(Clone)]` * The trait `Array` MUST only be implemented by structs in this module. * Every child array on the struct MUST be `Arc`. This enables the struct to be clonable. -* An array MUST implement `from_data(...) -> Self`. This method MUST panic iff: - * the data does not follow the arrow specification - * the arguments lead to unsound code (e.g. a Utf8 array MUST verify that its each item is valid `utf8`) +* An array MUST implement `try_new(...) -> Self`. This method MUST error iff + the data does not follow the arrow specification, including any sentinel types such as utf8. -* An array MAY implement `unsafe from_data_unchecked` that skips the soundness validation. `from_data_unchecked` MUST panic if the specification is incorrect. +* An array MAY implement `unsafe try_new_unchecked` that skips validation steps that are `O(N)`. * An array MUST implement either `new_empty()` or `new_empty(DataType)` that returns a zero-len of `Self`. @@ -36,7 +35,7 @@ This document describes the overall design of this module. * functions to create new arrays from native Rust SHOULD be named as follows: * `from`: from a slice of optional values (e.g. `AsRef<[Option]` for `BooleanArray`) - * `from_slice`: from a slice of values (e.g. `AsRef<[bool]` for `BooleanArray`) + * `from_slice`: from a slice of values (e.g. `AsRef<[bool]>` for `BooleanArray`) * `from_trusted_len_iter` from an iterator of trusted len of optional values * `from_trusted_len_values_iter` from an iterator of trusted len of values * `try_from_trusted_len_iter` from an fallible iterator of trusted len of optional values diff --git a/src/array/binary/ffi.rs b/src/array/binary/ffi.rs index 2cd1fb2e628..c8a72615bce 100644 --- a/src/array/binary/ffi.rs +++ b/src/array/binary/ffi.rs @@ -52,7 +52,7 @@ unsafe impl ToFfi for BinaryArray { impl FromFfi for BinaryArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let offsets = unsafe { array.buffer::(1) }?; diff --git a/src/array/binary/mod.rs b/src/array/binary/mod.rs index 437854eba4c..934ff811f67 100644 --- a/src/array/binary/mod.rs +++ b/src/array/binary/mod.rs @@ -6,7 +6,7 @@ use crate::{ }; use super::{ - specification::{check_offsets_minimal, try_check_offsets}, + specification::{try_check_offsets, try_check_offsets_bounds}, Array, GenericBinaryArray, Offset, }; @@ -33,9 +33,77 @@ pub struct BinaryArray { // constructors impl BinaryArray { + /// Creates a new [`BinaryArray`]. + /// + /// # Errors + /// This function returns an error iff: + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Binary` or `LargeBinary`. + /// # Implementation + /// This function is `O(N)` - checking monotinicity is `O(N)` + pub fn try_new( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Result { + try_check_offsets(&offsets, values.len())?; + + if validity + .as_ref() + .map_or(false, |validity| validity.len() != offsets.len() - 1) + { + return Err(ArrowError::oos( + "validity mask length must match the number of values", + )); + } + + if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { + return Err(ArrowError::oos( + "BinaryArray can only be initialized with DataType::Binary or DataType::LargeBinary", + )); + } + + Ok(Self { + data_type, + offsets, + values, + validity, + }) + } + + /// Creates a new [`BinaryArray`]. + /// # Panics + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Binary` or `LargeBinary`. + /// # Implementation + /// This function is `O(N)` - checking monotinicity is `O(N)` + pub fn new( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Self { + Self::try_new(data_type, offsets, values, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Self { + Self::new(data_type, offsets, values, validity) + } + /// Creates an empty [`BinaryArray`], i.e. whose `.len` is zero. pub fn new_empty(data_type: DataType) -> Self { - Self::from_data( + Self::new( data_type, Buffer::from(vec![O::zero()]), Buffer::new(), @@ -46,7 +114,7 @@ impl BinaryArray { /// Creates an null [`BinaryArray`], i.e. whose `.null_count() == .len()`. #[inline] pub fn new_null(data_type: DataType, length: usize) -> Self { - Self::from_data( + Self::new( data_type, Buffer::new_zeroed(length + 1), Buffer::new(), @@ -54,35 +122,37 @@ impl BinaryArray { ) } - /// Creates a new [`BinaryArray`] from lower-level parts - /// # Panics - /// * the offsets are not monotonically increasing - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len() - 1`. - /// * The `data_type`'s physical type is not equal to `Binary` or `LargeBinary`. - pub fn from_data( - data_type: DataType, - offsets: Buffer, - values: Buffer, - validity: Option, - ) -> Self { - Self::try_new(data_type, offsets, values, validity).unwrap() + /// Returns the default [`DataType`], `DataType::Binary` or `DataType::LargeBinary` + pub fn default_data_type() -> DataType { + if O::is_large() { + DataType::LargeBinary + } else { + DataType::Binary + } } +} - /// Creates a new [`BinaryArray`] from lower-level parts. +// unsafe constructors +impl BinaryArray { + /// Creates a new [`BinaryArray`] without checking for offsets monotinicity. /// + /// # Errors /// This function returns an error iff: - /// * the offsets are not monotonically increasing /// * The last offset is not equal to the values' length. /// * the validity's length is not equal to `offsets.len() - 1`. - /// * The `data_type`'s physical type is not equal to `Binary` or `LargeBinary`. - pub fn try_new( + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Binary` or `LargeBinary`. + /// # Safety + /// This function is unsafe iff: + /// * the offsets are not monotonically increasing + /// # Implementation + /// This function is `O(1)` + pub unsafe fn try_new_unchecked( data_type: DataType, offsets: Buffer, values: Buffer, validity: Option, ) -> Result { - try_check_offsets(&offsets, values.len())?; + try_check_offsets_bounds(&offsets, values.len())?; if validity .as_ref() @@ -107,52 +177,49 @@ impl BinaryArray { }) } - /// Returns the default [`DataType`], `DataType::Binary` or `DataType::LargeBinary` - pub fn default_data_type() -> DataType { - if O::is_large() { - DataType::LargeBinary - } else { - DataType::Binary - } + /// Creates a new [`BinaryArray`] without checking for offsets monotinicity. + /// + /// # Panics + /// This function returns an error iff: + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Binary` or `LargeBinary`. + /// # Safety + /// This function is unsafe iff: + /// * the offsets are not monotonically increasing + /// # Implementation + /// This function is `O(1)` + pub unsafe fn new_unchecked( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Self { + Self::try_new_unchecked(data_type, offsets, values, validity).unwrap() } - /// The same as [`BinaryArray::from_data`] but does not check for offsets. + /// Alias for [`new_unchecked`] /// # Safety - /// * `offsets` MUST be monotonically increasing - /// # Panics - /// This function panics iff: - /// * The `data_type`'s physical type is not consistent with the offset `O`. - /// * The last element of `offsets` is different from `values.len()`. - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. + /// This function is unsafe iff: + /// * the offsets are not monotonically increasing pub unsafe fn from_data_unchecked( data_type: DataType, offsets: Buffer, values: Buffer, validity: Option, ) -> Self { - check_offsets_minimal(&offsets, values.len()); - - if let Some(validity) = &validity { - assert_eq!(offsets.len() - 1, validity.len()); - } - - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - panic!("BinaryArray can only be initialized with DataType::Binary or DataType::LargeBinary") - } - - Self { - data_type, - offsets, - values, - validity, - } + Self::new_unchecked(data_type, offsets, values, validity) } +} +// must use +impl BinaryArray { /// Creates a new [`BinaryArray`] by slicing this [`BinaryArray`]. /// # Implementation /// This function is `O(1)`: all data will be shared between both arrays. /// # Panics /// iff `offset + length > self.len()`. + #[must_use] pub fn slice(&self, offset: usize, length: usize) -> Self { assert!( offset + length <= self.len(), @@ -166,6 +233,7 @@ impl BinaryArray { /// This function is `O(1)`: all data will be shared between both arrays. /// # Safety /// The caller must ensure that `offset + length <= self.len()`. + #[must_use] pub unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Self { let validity = self .validity @@ -183,6 +251,7 @@ impl BinaryArray { /// Clones this [`BinaryArray`] with a different validity. /// # Panic /// Panics iff `validity.len() != self.len()`. + #[must_use] pub fn with_validity(&self, validity: Option) -> Self { if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { panic!("validity's length must be equal to the array's length") diff --git a/src/array/boolean/ffi.rs b/src/array/boolean/ffi.rs index f07d2d20263..9998e8f00e1 100644 --- a/src/array/boolean/ffi.rs +++ b/src/array/boolean/ffi.rs @@ -50,7 +50,7 @@ unsafe impl ToFfi for BooleanArray { impl FromFfi for BooleanArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let values = unsafe { array.bitmap(1) }?; Ok(Self::from_data(data_type, values, validity)) diff --git a/src/array/boolean/mod.rs b/src/array/boolean/mod.rs index 0c5126a2707..4ee5ccfa33e 100644 --- a/src/array/boolean/mod.rs +++ b/src/array/boolean/mod.rs @@ -1,6 +1,7 @@ use crate::{ bitmap::Bitmap, datatypes::{DataType, PhysicalType}, + error::ArrowError, }; use either::Either; @@ -25,6 +26,52 @@ pub struct BooleanArray { } impl BooleanArray { + /// The canonical method to create a [`BooleanArray`] out of low-end APIs. + /// # Errors + /// This function errors iff: + /// * The validity is not `None` and its length is different from `values`'s length + /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Boolean`]. + pub fn try_new( + data_type: DataType, + values: Bitmap, + validity: Option, + ) -> Result { + if validity + .as_ref() + .map_or(false, |validity| validity.len() != values.len()) + { + return Err(ArrowError::oos( + "validity mask length must match the number of values", + )); + } + + if data_type.to_physical_type() != PhysicalType::Boolean { + return Err(ArrowError::oos( + "BooleanArray can only be initialized with a DataType whose physical type is Boolean", + )); + } + + Ok(Self { + data_type, + values, + validity, + }) + } + + /// The canonical method to create a [`BooleanArray`] + /// # Panics + /// This function errors iff: + /// * The validity is not `None` and its length is different from `values`'s length + /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Boolean`]. + pub fn new(data_type: DataType, values: Bitmap, validity: Option) -> Self { + Self::try_new(data_type, values, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data(data_type: DataType, values: Bitmap, validity: Option) -> Self { + Self::new(data_type, values, validity) + } + /// Returns a new empty [`BooleanArray`]. pub fn new_empty(data_type: DataType) -> Self { Self::from_data(data_type, Bitmap::new(), None) @@ -35,31 +82,17 @@ impl BooleanArray { let bitmap = Bitmap::new_zeroed(length); Self::from_data(data_type, bitmap.clone(), Some(bitmap)) } +} - /// The canonical method to create a [`BooleanArray`] out of low-end APIs. - /// # Panics - /// This function panics iff: - /// * The validity is not `None` and its length is different from `values`'s length - pub fn from_data(data_type: DataType, values: Bitmap, validity: Option) -> Self { - if let Some(ref validity) = validity { - assert_eq!(values.len(), validity.len()); - } - if data_type.to_physical_type() != PhysicalType::Boolean { - panic!("BooleanArray can only be initialized with DataType::Boolean") - } - Self { - data_type, - values, - validity, - } - } - +// must use +impl BooleanArray { /// Returns a slice of this [`BooleanArray`]. /// # Implementation /// This operation is `O(1)` as it amounts to increase two ref counts. /// # Panic /// This function panics iff `offset + length >= self.len()`. #[inline] + #[must_use] pub fn slice(&self, offset: usize, length: usize) -> Self { assert!( offset + length <= self.len(), @@ -74,6 +107,7 @@ impl BooleanArray { /// # Safety /// The caller must ensure that `offset + length <= self.len()`. #[inline] + #[must_use] pub unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Self { let validity = self .validity @@ -89,6 +123,7 @@ impl BooleanArray { /// Sets the validity bitmap on this [`BooleanArray`]. /// # Panic /// This function panics iff `validity.len() != self.len()`. + #[must_use] pub fn with_validity(&self, validity: Option) -> Self { if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { panic!("validity should be as least as large as the array") @@ -98,7 +133,7 @@ impl BooleanArray { arr } - /// Try to convert this `BooleanArray` to a `MutableBooleanArray` + /// Try to convert this [`BooleanArray`] to a [`MutableBooleanArray`] pub fn into_mut(self) -> Either { use Either::*; diff --git a/src/array/fixed_size_binary/ffi.rs b/src/array/fixed_size_binary/ffi.rs index 4d371c44643..7728045c0f4 100644 --- a/src/array/fixed_size_binary/ffi.rs +++ b/src/array/fixed_size_binary/ffi.rs @@ -50,7 +50,7 @@ unsafe impl ToFfi for FixedSizeBinaryArray { impl FromFfi for FixedSizeBinaryArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let values = unsafe { array.buffer::(1) }?; diff --git a/src/array/fixed_size_binary/mod.rs b/src/array/fixed_size_binary/mod.rs index f796340c93a..935fbd4ac54 100644 --- a/src/array/fixed_size_binary/mod.rs +++ b/src/array/fixed_size_binary/mod.rs @@ -1,4 +1,4 @@ -use crate::{bitmap::Bitmap, buffer::Buffer, datatypes::DataType, error::Result}; +use crate::{bitmap::Bitmap, buffer::Buffer, datatypes::DataType, error::ArrowError}; use super::Array; @@ -19,6 +19,61 @@ pub struct FixedSizeBinaryArray { } impl FixedSizeBinaryArray { + /// Creates a new [`FixedSizeBinaryArray`]. + /// + /// # Errors + /// This function returns an error iff: + /// * The `data_type`'s physical type is not [`crate::datatypes::PhysicalType::FixedSizeBinary`] + /// * The length of `values` is not a multiple of `size` in `data_type` + /// * the validity's length is not equal to `values.len() / size`. + pub fn try_new( + data_type: DataType, + values: Buffer, + validity: Option, + ) -> Result { + let size = Self::maybe_get_size(&data_type)?; + + if values.len() % size != 0 { + return Err(ArrowError::oos(format!( + "values (of len {}) must be a multiple of size ({}) in FixedSizeBinaryArray.", + values.len(), + size + ))); + } + let len = values.len() / size; + + if validity + .as_ref() + .map_or(false, |validity| validity.len() != len) + { + return Err(ArrowError::oos( + "validity mask length must be equal to the number of values divided by size", + )); + } + + Ok(Self { + size, + data_type, + values, + validity, + }) + } + + /// Creates a new [`FixedSizeBinaryArray`]. + /// # Panics + /// This function panics iff: + /// * The `data_type`'s physical type is not [`crate::datatypes::PhysicalType::FixedSizeBinary`] + /// * The length of `values` is not a multiple of `size` in `data_type` + /// * the validity's length is not equal to `values.len() / size`. + pub fn new(data_type: DataType, values: Buffer, validity: Option) -> Self { + Self::try_new(data_type, values, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data(data_type: DataType, values: Buffer, validity: Option) -> Self { + Self::new(data_type, values, validity) + } + /// Returns a new empty [`FixedSizeBinaryArray`]. pub fn new_empty(data_type: DataType) -> Self { Self::from_data(data_type, Buffer::new(), None) @@ -32,30 +87,16 @@ impl FixedSizeBinaryArray { Some(Bitmap::new_zeroed(length)), ) } +} - /// Returns a new [`FixedSizeBinaryArray`]. - pub fn from_data(data_type: DataType, values: Buffer, validity: Option) -> Self { - let size = Self::get_size(&data_type); - - assert_eq!(values.len() % size, 0); - - if let Some(ref validity) = validity { - assert_eq!(values.len() / size, validity.len()); - } - - Self { - size, - data_type, - values, - validity, - } - } - +// must use +impl FixedSizeBinaryArray { /// Returns a slice of this [`FixedSizeBinaryArray`]. /// # Implementation /// This operation is `O(1)` as it amounts to increase 3 ref counts. /// # Panics /// panics iff `offset + length > self.len()` + #[must_use] pub fn slice(&self, offset: usize, length: usize) -> Self { assert!( offset + length <= self.len(), @@ -69,6 +110,7 @@ impl FixedSizeBinaryArray { /// This operation is `O(1)` as it amounts to increase 3 ref counts. /// # Safety /// The caller must ensure that `offset + length <= self.len()`. + #[must_use] pub unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Self { let validity = self .validity @@ -89,6 +131,7 @@ impl FixedSizeBinaryArray { /// Sets the validity bitmap on this [`FixedSizeBinaryArray`]. /// # Panic /// This function panics iff `validity.len() != self.len()`. + #[must_use] pub fn with_validity(&self, validity: Option) -> Self { if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { panic!("validity should be as least as large as the array") @@ -167,12 +210,18 @@ impl FixedSizeBinaryArray { } impl FixedSizeBinaryArray { - pub(crate) fn get_size(data_type: &DataType) -> usize { + pub(crate) fn maybe_get_size(data_type: &DataType) -> Result { match data_type.to_logical_type() { - DataType::FixedSizeBinary(size) => *size, - _ => panic!("Wrong DataType"), + DataType::FixedSizeBinary(size) => Ok(*size), + _ => Err(ArrowError::oos( + "FixedSizeBinaryArray expects DataType::FixedSizeBinary", + )), } } + + pub(crate) fn get_size(data_type: &DataType) -> usize { + Self::maybe_get_size(data_type).unwrap() + } } impl Array for FixedSizeBinaryArray { @@ -211,7 +260,7 @@ impl FixedSizeBinaryArray { pub fn try_from_iter, I: IntoIterator>>( iter: I, size: usize, - ) -> Result { + ) -> Result { MutableFixedSizeBinaryArray::try_from_iter(iter, size).map(|x| x.into()) } diff --git a/src/array/fixed_size_list/ffi.rs b/src/array/fixed_size_list/ffi.rs index 1c2a2063e6d..5e862f56b48 100644 --- a/src/array/fixed_size_list/ffi.rs +++ b/src/array/fixed_size_list/ffi.rs @@ -35,7 +35,7 @@ unsafe impl ToFfi for FixedSizeListArray { impl FromFfi for FixedSizeListArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let child = unsafe { array.child(0)? }; let values = ffi::try_from(child)?.into(); diff --git a/src/array/fixed_size_list/mod.rs b/src/array/fixed_size_list/mod.rs index 197bad87d5e..61ac7970f88 100644 --- a/src/array/fixed_size_list/mod.rs +++ b/src/array/fixed_size_list/mod.rs @@ -3,6 +3,7 @@ use std::sync::Arc; use crate::{ bitmap::Bitmap, datatypes::{DataType, Field}, + error::ArrowError, }; use super::{new_empty_array, new_null_array, Array}; @@ -25,6 +26,75 @@ pub struct FixedSizeListArray { } impl FixedSizeListArray { + /// Creates a new [`FixedSizeListArray`]. + /// + /// # Errors + /// This function returns an error iff: + /// * The `data_type`'s physical type is not [`crate::datatypes::PhysicalType::FixedSizeList`] + /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. + /// * The length of `values` is not a multiple of `size` in `data_type` + /// * the validity's length is not equal to `values.len() / size`. + pub fn try_new( + data_type: DataType, + values: Arc, + validity: Option, + ) -> Result { + let (child, size) = Self::try_child_and_size(&data_type)?; + + let child_data_type = &child.data_type; + let values_data_type = values.data_type(); + if child_data_type != values_data_type { + return Err(ArrowError::oos( + format!("FixedSizeListArray's child's DataType must match. However, the expected DataType is {child_data_type:?} while it got {values_data_type:?}."), + )); + } + + if values.len() % size != 0 { + return Err(ArrowError::oos(format!( + "values (of len {}) must be a multiple of size ({}) in FixedSizeListArray.", + values.len(), + size + ))); + } + let len = values.len() / size; + + if validity + .as_ref() + .map_or(false, |validity| validity.len() != len) + { + return Err(ArrowError::oos( + "validity mask length must be equal to the number of values divided by size", + )); + } + + Ok(Self { + size, + data_type, + values, + validity, + }) + } + + /// Creates a new [`FixedSizeListArray`]. + /// # Panics + /// This function panics iff: + /// * The `data_type`'s physical type is not [`crate::datatypes::PhysicalType::FixedSizeList`] + /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. + /// * The length of `values` is not a multiple of `size` in `data_type` + /// * the validity's length is not equal to `values.len() / size`. + pub fn new(data_type: DataType, values: Arc, validity: Option) -> Self { + Self::try_new(data_type, values, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data( + data_type: DataType, + values: Arc, + validity: Option, + ) -> Self { + Self::new(data_type, values, validity) + } + /// Returns a new empty [`FixedSizeListArray`]. pub fn new_empty(data_type: DataType) -> Self { let values = @@ -41,34 +111,16 @@ impl FixedSizeListArray { .into(); Self::from_data(data_type, values, Some(Bitmap::new_zeroed(length))) } +} - /// Returns a [`FixedSizeListArray`]. - pub fn from_data( - data_type: DataType, - values: Arc, - validity: Option, - ) -> Self { - let (_, size) = Self::get_child_and_size(&data_type); - - assert_eq!(values.len() % size, 0); - - if let Some(ref validity) = validity { - assert_eq!(values.len() / size, validity.len()); - } - - Self { - size, - data_type, - values, - validity, - } - } - +// must use +impl FixedSizeListArray { /// Returns a slice of this [`FixedSizeListArray`]. /// # Implementation /// This operation is `O(1)`. /// # Panics /// panics iff `offset + length > self.len()` + #[must_use] pub fn slice(&self, offset: usize, length: usize) -> Self { assert!( offset + length <= self.len(), @@ -82,6 +134,7 @@ impl FixedSizeListArray { /// This operation is `O(1)`. /// # Safety /// The caller must ensure that `offset + length <= self.len()`. + #[must_use] pub unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Self { let validity = self .validity @@ -103,6 +156,7 @@ impl FixedSizeListArray { /// Sets the validity bitmap on this [`FixedSizeListArray`]. /// # Panic /// This function panics iff `validity.len() != self.len()`. + #[must_use] pub fn with_validity(&self, validity: Option) -> Self { if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { panic!("validity should be as least as large as the array") @@ -152,13 +206,19 @@ impl FixedSizeListArray { } impl FixedSizeListArray { - pub(crate) fn get_child_and_size(data_type: &DataType) -> (&Field, usize) { + pub(crate) fn try_child_and_size(data_type: &DataType) -> Result<(&Field, usize), ArrowError> { match data_type.to_logical_type() { - DataType::FixedSizeList(child, size) => (child.as_ref(), *size as usize), - _ => panic!("FixedSizeListArray expects DataType::FixedSizeList"), + DataType::FixedSizeList(child, size) => Ok((child.as_ref(), *size as usize)), + _ => Err(ArrowError::oos( + "FixedSizeListArray expects DataType::FixedSizeList", + )), } } + pub(crate) fn get_child_and_size(data_type: &DataType) -> (&Field, usize) { + Self::try_child_and_size(data_type).unwrap() + } + /// Returns a [`DataType`] consistent with [`FixedSizeListArray`]. pub fn default_datatype(data_type: DataType, size: usize) -> DataType { let field = Box::new(Field::new("item", data_type, true)); diff --git a/src/array/list/ffi.rs b/src/array/list/ffi.rs index 11af11bfd6a..67349792305 100644 --- a/src/array/list/ffi.rs +++ b/src/array/list/ffi.rs @@ -52,7 +52,7 @@ unsafe impl ToFfi for ListArray { impl FromFfi for ListArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let offsets = unsafe { array.buffer::(1) }?; let child = unsafe { array.child(0)? }; diff --git a/src/array/list/mod.rs b/src/array/list/mod.rs index 4da1c2835af..b34ce2b0c32 100644 --- a/src/array/list/mod.rs +++ b/src/array/list/mod.rs @@ -4,9 +4,14 @@ use crate::{ bitmap::Bitmap, buffer::Buffer, datatypes::{DataType, Field}, + error::ArrowError, }; -use super::{new_empty_array, specification::check_offsets, Array, Offset}; +use super::{ + new_empty_array, + specification::{try_check_offsets, try_check_offsets_bounds}, + Array, Offset, +}; mod ffi; pub(super) mod fmt; @@ -25,58 +30,171 @@ pub struct ListArray { } impl ListArray { + /// Creates a new [`ListArray`]. + /// + /// # Errors + /// This function returns an error iff: + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either [`crate::datatypes::PhysicalType::List`] or [`crate::datatypes::PhysicalType::LargeList`]. + /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. + /// # Implementation + /// This function is `O(N)` - checking monotinicity is `O(N)` + pub fn try_new( + data_type: DataType, + offsets: Buffer, + values: Arc, + validity: Option, + ) -> Result { + try_check_offsets(&offsets, values.len())?; + + if validity + .as_ref() + .map_or(false, |validity| validity.len() != offsets.len() - 1) + { + return Err(ArrowError::oos( + "validity mask length must match the number of values", + )); + } + + let child_data_type = Self::try_get_child(&data_type)?.data_type(); + let values_data_type = values.data_type(); + if child_data_type != values_data_type { + return Err(ArrowError::oos( + format!("ListArray's child's DataType must match. However, the expected DataType is {child_data_type:?} while it got {values_data_type:?}."), + )); + } + + Ok(Self { + data_type, + offsets, + values, + validity, + }) + } + + /// Creates a new [`ListArray`]. + /// + /// # Panics + /// This function panics iff: + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either [`crate::datatypes::PhysicalType::List`] or [`crate::datatypes::PhysicalType::LargeList`]. + /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. + /// # Implementation + /// This function is `O(N)` - checking monotinicity is `O(N)` + pub fn new( + data_type: DataType, + offsets: Buffer, + values: Arc, + validity: Option, + ) -> Self { + Self::try_new(data_type, offsets, values, validity).unwrap() + } + + /// Alias of `new` + pub fn from_data( + data_type: DataType, + offsets: Buffer, + values: Arc, + validity: Option, + ) -> Self { + Self::new(data_type, offsets, values, validity) + } + /// Returns a new empty [`ListArray`]. pub fn new_empty(data_type: DataType) -> Self { let values = new_empty_array(Self::get_child_type(&data_type).clone()).into(); - Self::from_data(data_type, Buffer::from(vec![O::zero()]), values, None) + Self::new(data_type, Buffer::from(vec![O::zero()]), values, None) } /// Returns a new null [`ListArray`]. #[inline] pub fn new_null(data_type: DataType, length: usize) -> Self { let child = Self::get_child_type(&data_type).clone(); - Self::from_data( + Self::new( data_type, Buffer::new_zeroed(length + 1), new_empty_array(child).into(), Some(Bitmap::new_zeroed(length)), ) } +} - /// Returns a new [`ListArray`]. - /// # Panic - /// This function panics iff: - /// * The `data_type`'s physical type is not consistent with the offset `O`. - /// * The `offsets` and `values` are inconsistent - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. - pub fn from_data( +// unsafe construtors +impl ListArray { + /// Creates a new [`ListArray`]. + /// + /// # Errors + /// This function returns an error iff: + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either [`crate::datatypes::PhysicalType::List`] or [`crate::datatypes::PhysicalType::LargeList`]. + /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. + /// # Safety + /// This function is unsafe iff: + /// * the offsets are not monotonically increasing + /// # Implementation + /// This function is `O(1)` + pub unsafe fn try_new_unchecked( data_type: DataType, offsets: Buffer, values: Arc, validity: Option, - ) -> Self { - check_offsets(&offsets, values.len()); + ) -> Result { + try_check_offsets_bounds(&offsets, values.len())?; - if let Some(ref validity) = validity { - assert_eq!(offsets.len() - 1, validity.len()); + if validity + .as_ref() + .map_or(false, |validity| validity.len() != offsets.len() - 1) + { + return Err(ArrowError::oos( + "validity mask length must match the number of values", + )); } - // validate data_type - let child_data_type = Self::get_child_type(&data_type); - assert_eq!( - child_data_type, - values.data_type(), - "The child's datatype must match the inner type of the \'data_type\'" - ); + let child_data_type = Self::try_get_child(&data_type)?.data_type(); + let values_data_type = values.data_type(); + if child_data_type != values_data_type { + return Err(ArrowError::oos( + format!("ListArray's child's DataType must match. However, the expected DataType is {child_data_type:?} while it got {values_data_type:?}."), + )); + } - Self { + Ok(Self { data_type, offsets, values, validity, - } + }) + } + + /// Creates a new [`ListArray`]. + /// + /// # Panics + /// This function panics iff: + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either [`crate::datatypes::PhysicalType::List`] or [`crate::datatypes::PhysicalType::LargeList`]. + /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. + /// # Safety + /// This function is unsafe iff: + /// * the offsets are not monotonically increasing + /// # Implementation + /// This function is `O(1)` + pub unsafe fn new_unchecked( + data_type: DataType, + offsets: Buffer, + values: Arc, + validity: Option, + ) -> Self { + Self::try_new_unchecked(data_type, offsets, values, validity).unwrap() } +} +impl ListArray { /// Returns a slice of this [`ListArray`]. /// # Panics /// panics iff `offset + length >= self.len()` @@ -185,15 +303,24 @@ impl ListArray { /// # Panics /// Panics iff the logical type is not consistent with this struct. pub fn get_child_field(data_type: &DataType) -> &Field { + Self::try_get_child(data_type).unwrap() + } + + /// Returns a the inner [`Field`] + /// # Errors + /// Panics iff the logical type is not consistent with this struct. + fn try_get_child(data_type: &DataType) -> Result<&Field, ArrowError> { if O::is_large() { match data_type.to_logical_type() { - DataType::LargeList(child) => child.as_ref(), - _ => panic!("ListArray expects DataType::List or DataType::LargeList"), + DataType::LargeList(child) => Ok(child.as_ref()), + _ => Err(ArrowError::oos( + "ListArray expects DataType::LargeList", + )), } } else { match data_type.to_logical_type() { - DataType::List(child) => child.as_ref(), - _ => panic!("ListArray expects DataType::List or DataType::List"), + DataType::List(child) => Ok(child.as_ref()), + _ => Err(ArrowError::oos("ListArray expects DataType::List")), } } } diff --git a/src/array/map/ffi.rs b/src/array/map/ffi.rs index 0ab6881a78b..5607110ae48 100644 --- a/src/array/map/ffi.rs +++ b/src/array/map/ffi.rs @@ -52,7 +52,7 @@ unsafe impl ToFfi for MapArray { impl FromFfi for MapArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let offsets = unsafe { array.buffer::(1) }?; let child = array.child(0)?; diff --git a/src/array/map/mod.rs b/src/array/map/mod.rs index dfac80b6f0e..a8224051b41 100644 --- a/src/array/map/mod.rs +++ b/src/array/map/mod.rs @@ -4,9 +4,10 @@ use crate::{ bitmap::Bitmap, buffer::Buffer, datatypes::{DataType, Field}, + error::ArrowError, }; -use super::{new_empty_array, specification::check_offsets, Array}; +use super::{new_empty_array, specification::try_check_offsets, Array}; mod ffi; mod iterator; @@ -24,12 +25,80 @@ pub struct MapArray { } impl MapArray { - pub(crate) fn get_field(datatype: &DataType) -> &Field { - if let DataType::Map(field, _) = datatype.to_logical_type() { - field.as_ref() + /// Returns a new [`MapArray`]. + /// # Errors + /// This function errors iff: + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the field' length + /// * The `data_type`'s physical type is not [`crate::datatypes::PhysicalType::Map`] + /// * The fields' `data_type` is not equal to the inner field of `data_type` + /// * The validity is not `None` and its length is different from `offsets.len() - 1`. + pub fn try_new( + data_type: DataType, + offsets: Buffer, + field: Arc, + validity: Option, + ) -> Result { + try_check_offsets(&offsets, field.len())?; + + let inner_field = Self::try_get_field(&data_type)?; + if let DataType::Struct(inner) = inner_field.data_type() { + if inner.len() != 2 { + return Err(ArrowError::InvalidArgumentError( + "MapArray's inner `Struct` must have 2 fields (keys and maps)".to_string(), + )); + } } else { - panic!("MapArray expects `DataType::Map` logical type") + return Err(ArrowError::InvalidArgumentError( + "MapArray expects `DataType::Struct` as its inner logical type".to_string(), + )); } + if field.data_type() != inner_field.data_type() { + return Err(ArrowError::InvalidArgumentError( + "MapArray expects `field.data_type` to match its inner DataType".to_string(), + )); + } + + if validity + .as_ref() + .map_or(false, |validity| validity.len() != offsets.len() - 1) + { + return Err(ArrowError::oos( + "validity mask length must match the number of values", + )); + } + + Ok(Self { + data_type, + field, + offsets, + validity, + }) + } + + /// Creates a new [`MapArray`]. + /// # Panics + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the field' length. + /// * The `data_type`'s physical type is not [`crate::datatypes::PhysicalType::Map`], + /// * The validity is not `None` and its length is different from `offsets.len() - 1`. + pub fn new( + data_type: DataType, + offsets: Buffer, + field: Arc, + validity: Option, + ) -> Self { + Self::try_new(data_type, offsets, field, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data( + data_type: DataType, + offsets: Buffer, + field: Arc, + validity: Option, + ) -> Self { + Self::new(data_type, offsets, field, validity) } /// Returns a new null [`MapArray`] of `length`. @@ -48,41 +117,9 @@ impl MapArray { let field = new_empty_array(Self::get_field(&data_type).data_type().clone()).into(); Self::from_data(data_type, Buffer::from(vec![0i32]), field, None) } +} - /// Returns a new [`MapArray`]. - /// # Panic - /// This function panics iff: - /// * The `data_type`'s physical type is not consistent with [`MapArray`], - /// * The `offsets` and `field` are inconsistent - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. - pub fn from_data( - data_type: DataType, - offsets: Buffer, - field: Arc, - validity: Option, - ) -> Self { - check_offsets(&offsets, field.len()); - - if let Some(ref validity) = validity { - assert_eq!(offsets.len() - 1, validity.len()); - } - - if let DataType::Struct(inner) = Self::get_field(&data_type).data_type() { - if inner.len() != 2 { - panic!("MapArray expects its inner `Struct` to have 2 fields (keys and maps)") - } - } else { - panic!("MapArray expects `DataType::Struct` as its inner logical type") - } - - Self { - data_type, - field, - offsets, - validity, - } - } - +impl MapArray { /// Returns a slice of this [`MapArray`]. /// # Panics /// panics iff `offset + length >= self.len()` @@ -110,6 +147,20 @@ impl MapArray { validity, } } + + pub(crate) fn try_get_field(data_type: &DataType) -> Result<&Field, ArrowError> { + if let DataType::Map(field, _) = data_type.to_logical_type() { + Ok(field.as_ref()) + } else { + Err(ArrowError::oos( + "The data_type's logical type must be DataType::Map", + )) + } + } + + pub(crate) fn get_field(data_type: &DataType) -> &Field { + Self::try_get_field(data_type).unwrap() + } } // Accessors diff --git a/src/array/null.rs b/src/array/null.rs index 9c09b8ce705..a93cec49d42 100644 --- a/src/array/null.rs +++ b/src/array/null.rs @@ -2,7 +2,8 @@ use crate::{bitmap::Bitmap, datatypes::DataType}; use crate::{ array::{Array, FromFfi, ToFfi}, - error::Result, + datatypes::PhysicalType, + error::ArrowError, ffi, }; @@ -14,6 +15,33 @@ pub struct NullArray { } impl NullArray { + /// Returns a new [`NullArray`]. + /// # Errors + /// This function errors iff: + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to [`crate::datatypes::PhysicalType::Null`]. + pub fn try_new(data_type: DataType, length: usize) -> Result { + if data_type.to_physical_type() != PhysicalType::Null { + return Err(ArrowError::oos( + "BooleanArray can only be initialized with a DataType whose physical type is Boolean", + )); + } + + Ok(Self { data_type, length }) + } + + /// Returns a new [`NullArray`]. + /// # Panics + /// This function errors iff: + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to [`crate::datatypes::PhysicalType::Null`]. + pub fn new(data_type: DataType, length: usize) -> Self { + Self::try_new(data_type, length).unwrap() + } + + /// Alias for `new` + pub fn from_data(data_type: DataType, length: usize) -> Self { + Self::new(data_type, length) + } + /// Returns a new empty [`NullArray`]. pub fn new_empty(data_type: DataType) -> Self { Self::from_data(data_type, 0) @@ -23,12 +51,9 @@ impl NullArray { pub fn new_null(data_type: DataType, length: usize) -> Self { Self::from_data(data_type, length) } +} - /// Returns a new [`NullArray`]. - pub fn from_data(data_type: DataType, length: usize) -> Self { - Self { data_type, length } - } - +impl NullArray { /// Returns a slice of the [`NullArray`]. pub fn slice(&self, _offset: usize, length: usize) -> Self { Self { @@ -66,9 +91,11 @@ impl Array for NullArray { fn slice(&self, offset: usize, length: usize) -> Box { Box::new(self.slice(offset, length)) } + unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Box { Box::new(self.slice(offset, length)) } + fn with_validity(&self, _: Option) -> Box { panic!("cannot set validity of a null array") } @@ -95,8 +122,8 @@ unsafe impl ToFfi for NullArray { } impl FromFfi for NullArray { - unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + unsafe fn try_from_ffi(array: A) -> Result { + let data_type = array.data_type().clone(); Ok(Self::from_data(data_type, array.array().len())) } } diff --git a/src/array/primitive/ffi.rs b/src/array/primitive/ffi.rs index 1c35009c739..f5b4e5438ec 100644 --- a/src/array/primitive/ffi.rs +++ b/src/array/primitive/ffi.rs @@ -51,7 +51,7 @@ unsafe impl ToFfi for PrimitiveArray { impl FromFfi for PrimitiveArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let values = unsafe { array.buffer::(1) }?; diff --git a/src/array/primitive/mod.rs b/src/array/primitive/mod.rs index 2f9df8d66fe..a71de921d27 100644 --- a/src/array/primitive/mod.rs +++ b/src/array/primitive/mod.rs @@ -39,51 +39,76 @@ pub struct PrimitiveArray { } impl PrimitiveArray { + /// The canonical method to create a [`PrimitiveArray`]. + /// # Errors + /// This function errors iff: + /// * The validity is not `None` and its length is different from `values`'s length + /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Primitive`]. + pub fn try_new( + data_type: DataType, + values: Buffer, + validity: Option, + ) -> Result { + if validity + .as_ref() + .map_or(false, |validity| validity.len() != values.len()) + { + return Err(ArrowError::oos( + "validity mask length must match the number of values", + )); + } + + if data_type.to_physical_type() != PhysicalType::Primitive(T::PRIMITIVE) { + return Err(ArrowError::oos( + "BooleanArray can only be initialized with a DataType whose physical type is Primitive", + )); + } + + Ok(Self { + data_type, + values, + validity, + }) + } + + /// The canonical method to create a [`PrimitiveArray`] + /// # Panics + /// This function errors iff: + /// * The validity is not `None` and its length is different from `values`'s length + /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Primitive`]. + pub fn new(data_type: DataType, values: Buffer, validity: Option) -> Self { + Self::try_new(data_type, values, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data(data_type: DataType, values: Buffer, validity: Option) -> Self { + Self::new(data_type, values, validity) + } + /// Returns a new empty [`PrimitiveArray`]. pub fn new_empty(data_type: DataType) -> Self { - Self::from_data(data_type, Buffer::new(), None) + Self::new(data_type, Buffer::new(), None) } /// Returns a new [`PrimitiveArray`] whose all slots are null / `None`. #[inline] pub fn new_null(data_type: DataType, length: usize) -> Self { - Self::from_data( + Self::new( data_type, Buffer::new_zeroed(length), Some(Bitmap::new_zeroed(length)), ) } +} - /// The canonical method to create a [`PrimitiveArray`] out of low-end APIs. - /// # Panics - /// This function panics iff: - /// * `data_type` is not supported by the physical type - /// * The validity is not `None` and its length is different from the `values`'s length - pub fn from_data(data_type: DataType, values: Buffer, validity: Option) -> Self { - if !data_type.to_physical_type().eq_primitive(T::PRIMITIVE) { - Err(ArrowError::InvalidArgumentError(format!( - "Type {} does not support logical type {:?}", - std::any::type_name::(), - data_type - ))) - .unwrap() - } - if let Some(ref validity) = validity { - assert_eq!(values.len(), validity.len()); - } - Self { - data_type, - values, - validity, - } - } - +impl PrimitiveArray { /// Returns a slice of this [`PrimitiveArray`]. /// # Implementation /// This operation is `O(1)` as it amounts to increase two ref counts. /// # Panic /// This function panics iff `offset + length >= self.len()`. #[inline] + #[must_use] pub fn slice(&self, offset: usize, length: usize) -> Self { assert!( offset + length <= self.len(), @@ -98,6 +123,7 @@ impl PrimitiveArray { /// # Safety /// The caller must ensure that `offset + length <= self.len()`. #[inline] + #[must_use] pub unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Self { let validity = self .validity @@ -113,6 +139,7 @@ impl PrimitiveArray { /// Sets the validity bitmap on this [`PrimitiveArray`]. /// # Panics /// This function panics iff `validity.len() != self.len()`. + #[must_use] pub fn with_validity(&self, validity: Option) -> Self { if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { panic!("validity should be as least as large as the array") @@ -191,13 +218,13 @@ impl PrimitiveArray { if let Some(bitmap) = self.validity { match bitmap.into_mut() { - Left(bitmap) => Left(PrimitiveArray::from_data( + Left(bitmap) => Left(PrimitiveArray::new( self.data_type, self.values, Some(bitmap), )), Right(mutable_bitmap) => match self.values.into_mut() { - Left(buffer) => Left(PrimitiveArray::from_data( + Left(buffer) => Left(PrimitiveArray::new( self.data_type, buffer, Some(mutable_bitmap.into()), @@ -211,7 +238,7 @@ impl PrimitiveArray { } } else { match self.values.into_mut() { - Left(buffer) => Left(PrimitiveArray::from_data(self.data_type, buffer, None)), + Left(values) => Left(PrimitiveArray::new(self.data_type, values, None)), Right(values) => Right(MutablePrimitiveArray::from_data( self.data_type, values, diff --git a/src/array/specification.rs b/src/array/specification.rs index 530daed5300..93171762f56 100644 --- a/src/array/specification.rs +++ b/src/array/specification.rs @@ -1,6 +1,18 @@ use crate::error::{ArrowError, Result}; use crate::types::Offset; +pub fn try_check_offsets_bounds(offsets: &[O], values_len: usize) -> Result { + if let Some(last_offset) = offsets.last() { + if last_offset.to_usize() > values_len { + Err(ArrowError::oos("offsets must not exceed the values length")) + } else { + Ok(last_offset.to_usize()) + } + } else { + Err(ArrowError::oos("offsets must have at least one element")) + } +} + pub fn check_offsets_minimal(offsets: &[O], values_len: usize) -> usize { assert!( !offsets.is_empty(), @@ -58,9 +70,11 @@ pub fn try_check_offsets_and_utf8(offsets: &[O], values: &[u8]) -> Re // check bounds if offsets .last() - .map_or(false, |last| last.to_usize() > values.len()) + .map_or(true, |last| last.to_usize() > values.len()) { - return Err(ArrowError::oos("offsets must not exceed values length")); + return Err(ArrowError::oos( + "offsets must have at least one element and must not exceed values length", + )); }; Ok(()) @@ -81,9 +95,11 @@ pub fn try_check_offsets(offsets: &[O], values_len: usize) -> Result< Err(ArrowError::oos("offsets must be monotonically increasing")) } else if offsets .last() - .map_or(false, |last| last.to_usize() > values_len) + .map_or(true, |last| last.to_usize() > values_len) { - Err(ArrowError::oos("offsets must not exceed values length")) + Err(ArrowError::oos( + "offsets must have at least one element and must not exceed values length", + )) } else { Ok(()) } diff --git a/src/array/struct_/ffi.rs b/src/array/struct_/ffi.rs index 0168520cdcb..b25b233e2e8 100644 --- a/src/array/struct_/ffi.rs +++ b/src/array/struct_/ffi.rs @@ -29,7 +29,7 @@ unsafe impl ToFfi for StructArray { impl FromFfi for StructArray { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let fields = Self::get_fields(&data_type); let validity = unsafe { array.validity() }?; diff --git a/src/array/struct_/mod.rs b/src/array/struct_/mod.rs index d4ae90e3fb4..dc80cd906a4 100644 --- a/src/array/struct_/mod.rs +++ b/src/array/struct_/mod.rs @@ -3,6 +3,7 @@ use std::sync::Arc; use crate::{ bitmap::Bitmap, datatypes::{DataType, Field}, + error::ArrowError, }; use super::{new_empty_array, new_null_array, Array}; @@ -26,7 +27,7 @@ mod iterator; /// Field::new("c", DataType::Int32, false), /// ]; /// -/// let array = StructArray::from_data(DataType::Struct(fields), vec![boolean, int], None); +/// let array = StructArray::new(DataType::Struct(fields), vec![boolean, int], None); /// ``` #[derive(Clone)] pub struct StructArray { @@ -36,6 +37,101 @@ pub struct StructArray { } impl StructArray { + /// Returns a new [`StructArray`]. + /// # Errors + /// This function errors iff: + /// * `data_type`'s physical type is not [`crate::datatypes::PhysicalType::Struct`]. + /// * the children of `data_type` are empty + /// * the values's len is different from children's length + /// * any of the values's data type is different from its corresponding children' data type + /// * any element of values has a different length than the first element + /// * the validity's length is not equal to the length of the first element + pub fn try_new( + data_type: DataType, + values: Vec>, + validity: Option, + ) -> Result { + let fields = Self::try_get_fields(&data_type)?; + if fields.is_empty() { + return Err(ArrowError::oos( + "A StructArray must contain at least one field", + )); + } + if fields.len() != values.len() { + return Err(ArrowError::oos( + "A StructArray must a number of fields in its DataType equal to the number of child values", + )); + } + + fields + .iter().map(|a| &a.data_type) + .zip(values.iter().map(|a| a.data_type())) + .enumerate() + .try_for_each(|(index, (data_type, child))| { + if data_type != child { + Err(ArrowError::oos(format!( + "The children DataTypes of a StructArray must equal the children data types. + However, the field {index} has data type {data_type:?} but the value has data type {child:?}" + ))) + } else { + Ok(()) + } + })?; + + let len = values[0].len(); + values + .iter() + .map(|a| a.len()) + .enumerate() + .try_for_each(|(index, a_len)| { + if a_len != len { + Err(ArrowError::oos(format!( + "The children DataTypes of a StructArray must equal the children data types. + However, the values {index} has a length of {a_len}, which is different from values 0, {len}." + ))) + } else { + Ok(()) + } + })?; + + if validity + .as_ref() + .map_or(false, |validity| validity.len() != len) + { + return Err(ArrowError::oos( + "The validity length of a StructArray must match its number of elements", + )); + } + + Ok(Self { + data_type, + values, + validity, + }) + } + + /// Returns a new [`StructArray`] + /// # Panics + /// This function panics iff: + /// * `data_type`'s physical type is not [`crate::datatypes::PhysicalType::Struct`]. + /// * the children of `data_type` are empty + /// * the values's len is different from children's length + /// * any of the values's data type is different from its corresponding children' data type + /// * any element of values has a different length than the first element + /// * the validity's length is not equal to the length of the first element + pub fn new(data_type: DataType, values: Vec>, validity: Option) -> Self { + Self::try_new(data_type, values, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data( + data_type: DataType, + values: Vec>, + validity: Option, + ) -> Self { + Self::new(data_type, values, validity) + } + /// Creates an empty [`StructArray`]. pub fn new_empty(data_type: DataType) -> Self { if let DataType::Struct(fields) = &data_type { @@ -43,7 +139,7 @@ impl StructArray { .iter() .map(|field| new_empty_array(field.data_type().clone()).into()) .collect(); - Self::from_data(data_type, values, None) + Self::new(data_type, values, None) } else { panic!("StructArray must be initialized with DataType::Struct"); } @@ -56,44 +152,17 @@ impl StructArray { .iter() .map(|field| new_null_array(field.data_type().clone(), length).into()) .collect(); - Self::from_data(data_type, values, Some(Bitmap::new_zeroed(length))) + Self::new(data_type, values, Some(Bitmap::new_zeroed(length))) } else { panic!("StructArray must be initialized with DataType::Struct"); } } +} - /// Canonical method to create a [`StructArray`]. - /// # Panics - /// * fields are empty - /// * values's len is different from Fields' length. - /// * any element of values has a different length than the first element. - pub fn from_data( - data_type: DataType, - values: Vec>, - validity: Option, - ) -> Self { - let fields = Self::get_fields(&data_type); - assert!(!fields.is_empty()); - assert_eq!(fields.len(), values.len()); - assert!( - fields - .iter() - .map(|f| f.data_type()) - .eq(values.iter().map(|a| a.data_type())), - "The fields' datatypes must equal the values datatypes" - ); - assert!(values.iter().all(|x| x.len() == values[0].len())); - if let Some(ref validity) = validity { - assert_eq!(values[0].len(), validity.len()); - } - Self { - data_type, - values, - validity, - } - } - +// must use +impl StructArray { /// Deconstructs the [`StructArray`] into its individual components. + #[must_use] pub fn into_data(self) -> (Vec, Vec>, Option) { let Self { data_type, @@ -113,6 +182,7 @@ impl StructArray { /// * `offset + length` must be smaller than `self.len()`. /// # Implementation /// This operation is `O(F)` where `F` is the number of fields. + #[must_use] pub fn slice(&self, offset: usize, length: usize) -> Self { assert!( offset + length <= self.len(), @@ -126,6 +196,7 @@ impl StructArray { /// This operation is `O(F)` where `F` is the number of fields. /// # Safety /// The caller must ensure that `offset + length <= self.len()`. + #[must_use] pub unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Self { let validity = self .validity @@ -145,6 +216,7 @@ impl StructArray { /// Sets the validity bitmap on this [`StructArray`]. /// # Panic /// This function panics iff `validity.len() != self.len()`. + #[must_use] pub fn with_validity(&self, validity: Option) -> Self { if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { panic!("validity should be as least as large as the array") @@ -181,13 +253,19 @@ impl StructArray { impl StructArray { /// Returns the fields the `DataType::Struct`. - pub fn get_fields(data_type: &DataType) -> &[Field] { - match data_type { - DataType::Struct(fields) => fields, - DataType::Extension(_, inner, _) => Self::get_fields(inner), - _ => panic!("Wrong datatype passed to Struct."), + pub(crate) fn try_get_fields(data_type: &DataType) -> Result<&[Field], ArrowError> { + match data_type.to_logical_type() { + DataType::Struct(fields) => Ok(fields), + _ => Err(ArrowError::oos( + "Struct array must be created with a DataType whose physical type is Struct", + )), } } + + /// Returns the fields the `DataType::Struct`. + pub fn get_fields(data_type: &DataType) -> &[Field] { + Self::try_get_fields(data_type).unwrap() + } } impl Array for StructArray { diff --git a/src/array/union/ffi.rs b/src/array/union/ffi.rs index 5702e4f9d6c..8dd15321ff1 100644 --- a/src/array/union/ffi.rs +++ b/src/array/union/ffi.rs @@ -32,9 +32,8 @@ unsafe impl ToFfi for UnionArray { impl FromFfi for UnionArray { unsafe fn try_from_ffi(array: A) -> Result { - let field = array.field(); - let data_type = field.data_type().clone(); - let fields = Self::get_fields(field.data_type()); + let data_type = array.data_type().clone(); + let fields = Self::get_fields(&data_type); let mut types = unsafe { array.buffer::(0) }?; let offsets = if Self::is_sparse(&data_type) { diff --git a/src/array/union/mod.rs b/src/array/union/mod.rs index 4cbc51d6edf..351f88f56cb 100644 --- a/src/array/union/mod.rs +++ b/src/array/union/mod.rs @@ -4,6 +4,7 @@ use crate::{ bitmap::Bitmap, buffer::Buffer, datatypes::{DataType, Field, UnionMode}, + error::ArrowError, scalar::{new_scalar, Scalar}, }; @@ -14,6 +15,7 @@ pub(super) mod fmt; mod iterator; type FieldEntry = (usize, Arc); +type UnionComponents<'a> = (&'a [Field], Option<&'a [i32]>, UnionMode); /// [`UnionArray`] represents an array whose each slot can contain different values. /// @@ -37,6 +39,94 @@ pub struct UnionArray { } impl UnionArray { + /// Returns a new [`UnionArray`]. + /// # Errors + /// This function errors iff: + /// * `data_type`'s physical type is not [`crate::datatypes::PhysicalType::Union`]. + /// * the fields's len is different from the `data_type`'s children's length + /// * any of the values's data type is different from its corresponding children' data type + pub fn try_new( + data_type: DataType, + types: Buffer, + fields: Vec>, + offsets: Option>, + ) -> Result { + let (f, ids, mode) = Self::try_get_all(&data_type)?; + + if f.len() != fields.len() { + return Err(ArrowError::oos( + "The number of `fields` must equal the number of children fields in DataType::Union", + )); + }; + + f + .iter().map(|a| a.data_type()) + .zip(fields.iter().map(|a| a.data_type())) + .enumerate() + .try_for_each(|(index, (data_type, child))| { + if data_type != child { + Err(ArrowError::oos(format!( + "The children DataTypes of a UnionArray must equal the children data types. + However, the field {index} has data type {data_type:?} but the value has data type {child:?}" + ))) + } else { + Ok(()) + } + })?; + + if offsets.is_none() != mode.is_sparse() { + return Err(ArrowError::oos( + "The offsets must be set when the Union is dense and vice-versa", + )); + } + + let fields_hash = ids.as_ref().map(|ids| { + ids.iter() + .map(|x| *x as i8) + .enumerate() + .zip(fields.iter().cloned()) + .map(|((i, type_), field)| (type_, (i, field))) + .collect() + }); + + // not validated: + // * `offsets` is valid + // * max id < fields.len() + Ok(Self { + data_type, + fields_hash, + fields, + offsets, + types, + offset: 0, + }) + } + + /// Returns a new [`UnionArray`]. + /// # Panics + /// This function panics iff: + /// * `data_type`'s physical type is not [`crate::datatypes::PhysicalType::Union`]. + /// * the fields's len is different from the `data_type`'s children's length + /// * any of the values's data type is different from its corresponding children' data type + pub fn new( + data_type: DataType, + types: Buffer, + fields: Vec>, + offsets: Option>, + ) -> Self { + Self::try_new(data_type, types, fields, offsets).unwrap() + } + + /// Alias for `new` + pub fn from_data( + data_type: DataType, + types: Buffer, + fields: Vec>, + offsets: Option>, + ) -> Self { + Self::new(data_type, types, fields, offsets) + } + /// Creates a new null [`UnionArray`]. pub fn new_null(data_type: DataType, length: usize) -> Self { if let DataType::Union(f, _, mode) = &data_type { @@ -86,51 +176,9 @@ impl UnionArray { panic!("Union struct must be created with the corresponding Union DataType") } } +} - /// Creates a new [`UnionArray`]. - pub fn from_data( - data_type: DataType, - types: Buffer, - fields: Vec>, - offsets: Option>, - ) -> Self { - let (f, ids, mode) = Self::get_all(&data_type); - - if f.len() != fields.len() { - panic!("The number of `fields` must equal the number of fields in the Union DataType") - }; - let same_data_types = f - .iter() - .zip(fields.iter()) - .all(|(f, array)| f.data_type() == array.data_type()); - if !same_data_types { - panic!("All fields' datatype in the union must equal the datatypes on the fields.") - } - if offsets.is_none() != mode.is_sparse() { - panic!("Sparsness flag must equal to noness of offsets in UnionArray") - } - let fields_hash = ids.as_ref().map(|ids| { - ids.iter() - .map(|x| *x as i8) - .enumerate() - .zip(fields.iter().cloned()) - .map(|((i, type_), field)| (type_, (i, field))) - .collect() - }); - - // not validated: - // * `offsets` is valid - // * max id < fields.len() - Self { - data_type, - fields_hash, - fields, - offsets, - types, - offset: 0, - } - } - +impl UnionArray { /// Returns a slice of this [`UnionArray`]. /// # Implementation /// This operation is `O(F)` where `F` is the number of fields. @@ -254,13 +302,21 @@ impl Array for UnionArray { } impl UnionArray { - fn get_all(data_type: &DataType) -> (&[Field], Option<&[i32]>, UnionMode) { + fn try_get_all(data_type: &DataType) -> Result { match data_type.to_logical_type() { - DataType::Union(fields, ids, mode) => (fields, ids.as_ref().map(|x| x.as_ref()), *mode), - _ => panic!("Wrong datatype passed to UnionArray."), + DataType::Union(fields, ids, mode) => { + Ok((fields, ids.as_ref().map(|x| x.as_ref()), *mode)) + } + _ => Err(ArrowError::oos( + "The UnionArray requires a logical type of DataType::Union", + )), } } + fn get_all(data_type: &DataType) -> (&[Field], Option<&[i32]>, UnionMode) { + Self::try_get_all(data_type).unwrap() + } + /// Returns all fields from [`DataType::Union`]. /// # Panic /// Panics iff `data_type`'s logical type is not [`DataType::Union`]. diff --git a/src/array/utf8/ffi.rs b/src/array/utf8/ffi.rs index 8f5962fa5ea..e8caa53b65e 100644 --- a/src/array/utf8/ffi.rs +++ b/src/array/utf8/ffi.rs @@ -51,7 +51,7 @@ unsafe impl ToFfi for Utf8Array { impl FromFfi for Utf8Array { unsafe fn try_from_ffi(array: A) -> Result { - let data_type = array.field().data_type().clone(); + let data_type = array.data_type().clone(); let validity = unsafe { array.validity() }?; let offsets = unsafe { array.buffer::(1) }?; let values = unsafe { array.buffer::(2)? }; diff --git a/src/array/utf8/mod.rs b/src/array/utf8/mod.rs index 5d9220e25dc..a2538dd58a0 100644 --- a/src/array/utf8/mod.rs +++ b/src/array/utf8/mod.rs @@ -7,7 +7,7 @@ use crate::{ use either::Either; use super::{ - specification::{check_offsets_minimal, try_check_offsets_and_utf8}, + specification::{try_check_offsets_and_utf8, try_check_offsets_bounds}, Array, GenericBinaryArray, Offset, }; @@ -44,7 +44,78 @@ pub struct Utf8Array { validity: Option, } +// constructors impl Utf8Array { + /// Returns a new [`Utf8Array`]. + /// + /// # Errors + /// This function returns an error iff: + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. + /// * The `values` between two consecutive `offsets` are not valid utf8 + /// # Implementation + /// This function is `O(N)` - checking monotinicity and utf8 is `O(N)` + pub fn try_new( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Result { + try_check_offsets_and_utf8(&offsets, &values)?; + if validity + .as_ref() + .map_or(false, |validity| validity.len() != offsets.len() - 1) + { + return Err(ArrowError::oos( + "validity mask length must match the number of values", + )); + } + + if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { + return Err(ArrowError::oos( + "Utf8Array can only be initialized with DataType::Utf8 or DataType::LargeUtf8", + )); + } + + Ok(Self { + data_type, + offsets, + values, + validity, + }) + } + + /// Creates a new [`Utf8Array`]. + /// # Panics + /// This function panics iff: + /// * the offsets are not monotonically increasing + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. + /// * The `values` between two consecutive `offsets` are not valid utf8 + /// # Implementation + /// This function is `O(N)` - checking monotinicity and utf8 is `O(N)` + pub fn new( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Self { + Self::try_new(data_type, offsets, values, validity).unwrap() + } + + /// Alias for `new` + pub fn from_data( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Self { + Self::new(data_type, offsets, values, validity) + } + /// Returns a new empty [`Utf8Array`]. #[inline] pub fn new_empty(data_type: DataType) -> Self { @@ -69,36 +140,39 @@ impl Utf8Array { ) } - /// The canonical method to create a [`Utf8Array`] out of low-end APIs. - /// # Panics - /// This function panics iff: - /// * The `data_type`'s physical type is not consistent with the offset `O`. - /// * The `offsets` and `values` are inconsistent - /// * The `values` between `offsets` are utf8 encoded - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. - pub fn from_data( - data_type: DataType, - offsets: Buffer, - values: Buffer, - validity: Option, - ) -> Self { - Utf8Array::try_new(data_type, offsets, values, validity).unwrap() + /// Returns the default [`DataType`], `DataType::Utf8` or `DataType::LargeUtf8` + pub fn default_data_type() -> DataType { + if O::is_large() { + DataType::LargeUtf8 + } else { + DataType::Utf8 + } } +} - /// The canonical method to create a [`Utf8Array`] out of low-end APIs. +// unsafe constructors +impl Utf8Array { + /// Creates a new [`Utf8Array`] without checking for offsets monotinicity nor utf8-validity /// + /// # Errors /// This function returns an error iff: - /// * The `data_type`'s physical type is not consistent with the offset `O`. - /// * The `offsets` and `values` are inconsistent - /// * The `values` between `offsets` are utf8 encoded - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. - pub fn try_new( + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. + /// # Safety + /// This function is unsound iff: + /// * the offsets are not monotonically increasing + /// * The `values` between two consecutive `offsets` are not valid utf8 + /// # Implementation + /// This function is `O(1)` + pub unsafe fn try_new_unchecked( data_type: DataType, offsets: Buffer, values: Buffer, validity: Option, ) -> Result { - try_check_offsets_and_utf8(&offsets, &values)?; + try_check_offsets_bounds(&offsets, values.len())?; + if validity .as_ref() .map_or(false, |validity| validity.len() != offsets.len() - 1) @@ -110,7 +184,7 @@ impl Utf8Array { if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { return Err(ArrowError::oos( - "Utf8Array can only be initialized with DataType::Utf8 or DataType::LargeUtf8", + "BinaryArray can only be initialized with DataType::Utf8 or DataType::LargeUtf8", )); } @@ -122,52 +196,51 @@ impl Utf8Array { }) } - /// Returns the default [`DataType`], `DataType::Utf8` or `DataType::LargeUtf8` - pub fn default_data_type() -> DataType { - if O::is_large() { - DataType::LargeUtf8 - } else { - DataType::Utf8 - } + /// Creates a new [`Utf8Array`] without checking for offsets monotinicity. + /// + /// # Errors + /// This function returns an error iff: + /// * The last offset is not equal to the values' length. + /// * the validity's length is not equal to `offsets.len() - 1`. + /// * The `data_type`'s [`crate::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. + /// # Safety + /// This function is unsound iff: + /// * the offsets are not monotonically increasing + /// * The `values` between two consecutive `offsets` are not valid utf8 + /// # Implementation + /// This function is `O(1)` + pub unsafe fn new_unchecked( + data_type: DataType, + offsets: Buffer, + values: Buffer, + validity: Option, + ) -> Self { + Self::try_new_unchecked(data_type, offsets, values, validity).unwrap() } - /// The same as [`Utf8Array::from_data`] but does not check for offsets nor utf8 validity. + /// Alias for [`new_unchecked`] /// # Safety - /// * `offsets` MUST be monotonically increasing; and - /// * every slice of `values` constructed from `offsets` MUST be valid utf8 - /// # Panics - /// This function panics iff: - /// * The `data_type`'s physical type is not consistent with the offset `O`. - /// * The last element of `offsets` is different from `values.len()`. - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. + /// This function is unsafe iff: + /// * the offsets are not monotonically increasing + /// * The `values` between two consecutive `offsets` are not valid utf8 pub unsafe fn from_data_unchecked( data_type: DataType, offsets: Buffer, values: Buffer, validity: Option, ) -> Self { - check_offsets_minimal(&offsets, values.len()); - if let Some(ref validity) = validity { - assert_eq!(offsets.len() - 1, validity.len()); - } - - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - panic!("Utf8Array can only be initialized with DataType::Utf8 or DataType::LargeUtf8") - } - - Self { - data_type, - offsets, - values, - validity, - } + Self::new_unchecked(data_type, offsets, values, validity) } +} +// must use +impl Utf8Array { /// Returns a slice of this [`Utf8Array`]. /// # Implementation /// This operation is `O(1)` as it amounts to essentially increase two ref counts. /// # Panic /// This function panics iff `offset + length >= self.len()`. + #[must_use] pub fn slice(&self, offset: usize, length: usize) -> Self { assert!( offset + length <= self.len(), @@ -180,6 +253,7 @@ impl Utf8Array { /// This operation is `O(1)` as it amounts to essentially increase two ref counts. /// # Safety /// The caller must ensure that `offset + length <= self.len()`. + #[must_use] pub unsafe fn slice_unchecked(&self, offset: usize, length: usize) -> Self { let validity = self .validity diff --git a/src/bitmap/bitmap_ops.rs b/src/bitmap/bitmap_ops.rs index bcc4e433ee8..282a7bca3e4 100644 --- a/src/bitmap/bitmap_ops.rs +++ b/src/bitmap/bitmap_ops.rs @@ -7,6 +7,7 @@ use super::{ Bitmap, }; +/// Creates a [Vec] from an [`Iterator`] of [`BitChunk`]. /// # Safety /// The iterator must be [`TrustedLen`]. pub unsafe fn from_chunk_iter_unchecked>( @@ -35,7 +36,7 @@ pub unsafe fn from_chunk_iter_unchecked>( buffer } -/// Creates a Vec from a [`TrustedLen`] of [`BitChunk`], +/// Creates a [`Vec`] from a [`TrustedLen`] of [`BitChunk`]. pub fn chunk_iter_to_vec>(iter: I) -> Vec { unsafe { from_chunk_iter_unchecked(iter) } } diff --git a/src/bitmap/mutable.rs b/src/bitmap/mutable.rs index 1868d9b1854..5d0bbbb3849 100644 --- a/src/bitmap/mutable.rs +++ b/src/bitmap/mutable.rs @@ -58,7 +58,7 @@ impl MutableBitmap { } } - /// Initializes an a pre-allocated [`MutableBitmap`] with capacity for `capacity` bits. + /// Initializes a pre-allocated [`MutableBitmap`] with capacity for `capacity` bits. #[inline] pub fn with_capacity(capacity: usize) -> Self { Self { @@ -67,7 +67,7 @@ impl MutableBitmap { } } - /// Initializes an a pre-allocated [`MutableBitmap`] with capacity for `capacity` bits. + /// Reserves `additional` bits in the [`MutableBitmap`], potentially re-allocating its buffer. #[inline(always)] pub fn reserve(&mut self, additional: usize) { self.buffer diff --git a/src/buffer/bytes.rs b/src/buffer/bytes.rs index 109302a4816..3e33fc7ce45 100644 --- a/src/buffer/bytes.rs +++ b/src/buffer/bytes.rs @@ -13,7 +13,7 @@ pub enum Deallocation { /// Native deallocation, using Rust deallocator with Arrow-specific memory aligment Native, // Foreign interface, via a callback - Foreign(Arc), + Foreign(Arc), } impl Debug for Deallocation { diff --git a/src/compute/filter.rs b/src/compute/filter.rs index bdda0adbd6d..27ddf4fdee9 100644 --- a/src/compute/filter.rs +++ b/src/compute/filter.rs @@ -1,13 +1,15 @@ //! Contains operators to filter arrays such as [`filter`]. use crate::array::growable::{make_growable, Growable}; -use crate::bitmap::utils::{BitChunkIterExact, BitChunksExact}; +use crate::bitmap::utils::{BitChunk, BitChunkIterExact, BitChunksExact}; use crate::bitmap::{utils::SlicesIterator, Bitmap, MutableBitmap}; use crate::chunk::Chunk; use crate::datatypes::DataType; use crate::error::Result; use crate::types::simd::{NativeSimd, Simd}; -use crate::types::BitChunkIter; +use crate::types::BitChunkOnes; use crate::{array::*, types::NativeType}; +use num_traits::One; +use num_traits::Zero; /// Function that can filter arbitrary arrays pub type Filter<'a> = Box Box + 'a + Send + Sync>; @@ -21,20 +23,25 @@ where I: BitChunkIterExact<<::Simd as NativeSimd>::Chunk>, { let mut chunks = values.chunks_exact(T::Simd::LANES); - let mut new = Vec::::with_capacity(filter_count); let mut dst = new.as_mut_ptr(); chunks .by_ref() .zip(mask_chunks.by_ref()) .for_each(|(chunk, validity_chunk)| { - let iter = BitChunkIter::new(validity_chunk, T::Simd::LANES); - for (value, b) in chunk.iter().zip(iter) { - if b { - unsafe { - dst.write(*value); - dst = dst.add(1); - }; + let ones_iter = BitChunkOnes::new(validity_chunk); + + let (size, _) = ones_iter.size_hint(); + if size == T::Simd::LANES { + // Fast path: all lanes are set + unsafe { + std::ptr::copy(chunk.as_ptr(), dst, size); + dst = dst.add(size); + } + } else { + for pos in ones_iter { + dst.write(chunk[pos]); + dst = dst.add(1); } } }); @@ -74,22 +81,32 @@ where let mut validity_chunks = validity.chunks::<::Chunk>(); let mut new = Vec::::with_capacity(filter_count); - let mut new_validity = MutableBitmap::with_capacity(filter_count); let mut dst = new.as_mut_ptr(); + let mut new_validity = MutableBitmap::with_capacity(filter_count); + chunks .by_ref() .zip(validity_chunks.by_ref()) .zip(mask_chunks.by_ref()) .for_each(|((chunk, validity_chunk), mask_chunk)| { - let mask_iter = BitChunkIter::new(mask_chunk, T::Simd::LANES); - let validity_iter = BitChunkIter::new(validity_chunk, T::Simd::LANES); - for ((value, is_valid), is_selected) in chunk.iter().zip(validity_iter).zip(mask_iter) { - if is_selected { - unsafe { - dst.write(*value); - dst = dst.add(1); - new_validity.push_unchecked(is_valid); - }; + let ones_iter = BitChunkOnes::new(mask_chunk); + let (size, _) = ones_iter.size_hint(); + + if size == T::Simd::LANES { + // Fast path: all lanes are set + unsafe { + std::ptr::copy(chunk.as_ptr(), dst, size); + dst = dst.add(size); + new_validity.extend_from_slice(validity_chunk.to_ne_bytes().as_ref(), 0, size); + } + } else { + for pos in ones_iter { + dst.write(chunk[pos]); + dst = dst.add(1); + new_validity.push( + validity_chunk & (<<::Simd as NativeSimd>::Chunk>::one() << pos) + > <<::Simd as NativeSimd>::Chunk>::zero(), + ); } } }); diff --git a/src/compute/like.rs b/src/compute/like.rs index 334c1fe74ca..3c262585672 100644 --- a/src/compute/like.rs +++ b/src/compute/like.rs @@ -17,6 +17,10 @@ fn is_like_pattern(c: char) -> bool { c == '%' || c == '_' } +fn replace_pattern(pattern: &str) -> String { + pattern.replace('%', ".*").replace('_', ".") +} + #[inline] fn a_like_utf8 bool>( lhs: &Utf8Array, @@ -40,7 +44,7 @@ fn a_like_utf8 bool>( let pattern = if let Some(pattern) = map.get(pattern) { pattern } else { - let re_pattern = pattern.replace("%", ".*").replace("_", "."); + let re_pattern = replace_pattern(pattern); let re = Regex::new(&format!("^{}$", re_pattern)).map_err(|e| { ArrowError::InvalidArgumentError(format!( "Unable to build regex from LIKE pattern: {}", @@ -113,7 +117,7 @@ fn a_like_utf8_scalar bool>( let ends_with = &rhs[1..]; Bitmap::from_trusted_len_iter(lhs.values_iter().map(|x| op(x.ends_with(ends_with)))) } else { - let re_pattern = rhs.replace("%", ".*").replace("_", "."); + let re_pattern = replace_pattern(rhs); let re = Regex::new(&format!("^{}$", re_pattern)).map_err(|e| { ArrowError::InvalidArgumentError(format!( "Unable to build regex from LIKE pattern: {}", @@ -187,10 +191,8 @@ fn a_like_binary bool>( let pattern = if let Some(pattern) = map.get(pattern) { pattern } else { - let re_pattern = simdutf8::basic::from_utf8(pattern) - .unwrap() - .replace("%", ".*") - .replace("_", "."); + let re_pattern = simdutf8::basic::from_utf8(pattern).unwrap(); + let re_pattern = replace_pattern(re_pattern); let re = BytesRegex::new(&format!("^{}$", re_pattern)).map_err(|e| { ArrowError::InvalidArgumentError(format!( "Unable to build regex from LIKE pattern: {}", @@ -270,7 +272,7 @@ fn a_like_binary_scalar bool>( let ends_with = &rhs[1..]; Bitmap::from_trusted_len_iter(lhs.values_iter().map(|x| op(x.ends_with(ends_with)))) } else { - let re_pattern = pattern.replace("%", ".*").replace("_", "."); + let re_pattern = replace_pattern(pattern); let re = BytesRegex::new(&format!("^{}$", re_pattern)).map_err(|e| { ArrowError::InvalidArgumentError(format!( "Unable to build regex from LIKE pattern: {}", diff --git a/src/compute/lower.rs b/src/compute/lower.rs deleted file mode 100644 index 5a9978179b4..00000000000 --- a/src/compute/lower.rs +++ /dev/null @@ -1,67 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Defines kernel to extract a lower case of a \[Large\]StringArray - -use super::utils::utf8_apply; -use crate::array::*; -use crate::{ - datatypes::DataType, - error::{ArrowError, Result}, -}; - -/// Returns a new `Array` where each of each of the elements is lower-cased. -/// this function errors when the passed array is not a \[Large\]String array. -pub fn lower(array: &dyn Array) -> Result> { - match array.data_type() { - DataType::LargeUtf8 => Ok(Box::new(utf8_apply( - str::to_lowercase, - array - .as_any() - .downcast_ref::>() - .expect("A large string is expected"), - ))), - DataType::Utf8 => Ok(Box::new(utf8_apply( - str::to_lowercase, - array - .as_any() - .downcast_ref::>() - .expect("A string is expected"), - ))), - _ => Err(ArrowError::InvalidArgumentError(format!( - "lower does not support type {:?}", - array.data_type() - ))), - } -} - -/// Checks if an array of type `datatype` can perform lower operation -/// -/// # Examples -/// ``` -/// use arrow2::compute::lower::can_lower; -/// use arrow2::datatypes::{DataType}; -/// -/// let data_type = DataType::Utf8; -/// assert_eq!(can_lower(&data_type), true); -/// -/// let data_type = DataType::Null; -/// assert_eq!(can_lower(&data_type), false); -/// ``` -pub fn can_lower(data_type: &DataType) -> bool { - matches!(data_type, DataType::LargeUtf8 | DataType::Utf8) -} diff --git a/src/compute/mod.rs b/src/compute/mod.rs index 7667b320e53..801abd2bff1 100644 --- a/src/compute/mod.rs +++ b/src/compute/mod.rs @@ -57,9 +57,6 @@ pub mod like; #[cfg(feature = "compute_limit")] #[cfg_attr(docsrs, doc(cfg(feature = "compute_limit")))] pub mod limit; -#[cfg(feature = "compute_lower")] -#[cfg_attr(docsrs, doc(cfg(feature = "compute_lower")))] -pub mod lower; #[cfg(feature = "compute_merge_sort")] #[cfg_attr(docsrs, doc(cfg(feature = "compute_merge_sort")))] pub mod merge_sort; @@ -84,9 +81,9 @@ pub mod take; #[cfg(feature = "compute_temporal")] #[cfg_attr(docsrs, doc(cfg(feature = "compute_temporal")))] pub mod temporal; -#[cfg(feature = "compute_upper")] -#[cfg_attr(docsrs, doc(cfg(feature = "compute_upper")))] -pub mod upper; +#[cfg(feature = "compute_utf8")] +#[cfg_attr(docsrs, doc(cfg(feature = "compute_utf8")))] +pub mod utf8; mod utils; #[cfg(feature = "compute_window")] #[cfg_attr(docsrs, doc(cfg(feature = "compute_window")))] diff --git a/src/compute/regex_match.rs b/src/compute/regex_match.rs index 3cdc94cfad1..5866a4310d9 100644 --- a/src/compute/regex_match.rs +++ b/src/compute/regex_match.rs @@ -4,7 +4,7 @@ use std::collections::HashMap; use regex::Regex; -use super::utils::{combine_validities, unary_utf8_boolean}; +use super::utils::combine_validities; use crate::array::{BooleanArray, Offset, Utf8Array}; use crate::bitmap::Bitmap; use crate::datatypes::DataType; @@ -69,3 +69,19 @@ pub fn regex_match_scalar(values: &Utf8Array, regex: &str) -> Resu .map_err(|e| ArrowError::InvalidArgumentError(format!("Unable to compile regex: {}", e)))?; Ok(unary_utf8_boolean(values, |x| regex.is_match(x))) } + +fn unary_utf8_boolean bool>( + values: &Utf8Array, + op: F, +) -> BooleanArray { + let validity = values.validity().cloned(); + + let iterator = values.iter().map(|value| { + if value.is_none() { + return false; + }; + op(value.unwrap()) + }); + let values = Bitmap::from_trusted_len_iter(iterator); + BooleanArray::from_data(DataType::Boolean, values, validity) +} diff --git a/src/compute/sort/mod.rs b/src/compute/sort/mod.rs index 8463e2f61a9..767cde4cf32 100644 --- a/src/compute/sort/mod.rs +++ b/src/compute/sort/mod.rs @@ -355,7 +355,7 @@ where values.chain(null_indices.into_iter()).collect::>() }; - values.truncate(limit.unwrap_or_else(|| values.len())); + values.truncate(limit.unwrap_or(values.len())); let data_type = I::PRIMITIVE.into(); PrimitiveArray::::from_data(data_type, values.into(), None) diff --git a/src/compute/upper.rs b/src/compute/upper.rs deleted file mode 100644 index 6afbdd09286..00000000000 --- a/src/compute/upper.rs +++ /dev/null @@ -1,67 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Defines kernel to extract a upper case of a \[Large\]StringArray - -use super::utils::utf8_apply; -use crate::array::*; -use crate::{ - datatypes::DataType, - error::{ArrowError, Result}, -}; - -/// Returns a new `Array` where each of each of the elements is upper-cased. -/// this function errors when the passed array is not a \[Large\]String array. -pub fn upper(array: &dyn Array) -> Result> { - match array.data_type() { - DataType::LargeUtf8 => Ok(Box::new(utf8_apply( - str::to_uppercase, - array - .as_any() - .downcast_ref::>() - .expect("A large string is expected"), - ))), - DataType::Utf8 => Ok(Box::new(utf8_apply( - str::to_uppercase, - array - .as_any() - .downcast_ref::>() - .expect("A string is expected"), - ))), - _ => Err(ArrowError::InvalidArgumentError(format!( - "upper does not support type {:?}", - array.data_type() - ))), - } -} - -/// Checks if an array of type `datatype` can perform upper operation -/// -/// # Examples -/// ``` -/// use arrow2::compute::upper::can_upper; -/// use arrow2::datatypes::{DataType}; -/// -/// let data_type = DataType::Utf8; -/// assert_eq!(can_upper(&data_type), true); -/// -/// let data_type = DataType::Null; -/// assert_eq!(can_upper(&data_type), false); -/// ``` -pub fn can_upper(data_type: &DataType) -> bool { - matches!(data_type, DataType::LargeUtf8 | DataType::Utf8) -} diff --git a/src/compute/utf8.rs b/src/compute/utf8.rs new file mode 100644 index 00000000000..1b252e47505 --- /dev/null +++ b/src/compute/utf8.rs @@ -0,0 +1,99 @@ +//! Defines common maps to a [`Utf8Array`] + +use crate::{ + array::{Array, Offset, Utf8Array}, + datatypes::DataType, + error::{ArrowError, Result}, +}; + +/// utf8_apply will apply `Fn(&str) -> String` to every value in Utf8Array. +pub fn utf8_apply String>(f: F, array: &Utf8Array) -> Utf8Array { + let iter = array.values_iter().map(f); + + let new = Utf8Array::::from_trusted_len_values_iter(iter); + new.with_validity(array.validity().cloned()) +} + +/// Returns a new `Array` where each of each of the elements is upper-cased. +/// this function errors when the passed array is not a \[Large\]String array. +pub fn upper(array: &dyn Array) -> Result> { + match array.data_type() { + DataType::LargeUtf8 => Ok(Box::new(utf8_apply( + str::to_uppercase, + array + .as_any() + .downcast_ref::>() + .expect("A large string is expected"), + ))), + DataType::Utf8 => Ok(Box::new(utf8_apply( + str::to_uppercase, + array + .as_any() + .downcast_ref::>() + .expect("A string is expected"), + ))), + _ => Err(ArrowError::InvalidArgumentError(format!( + "upper does not support type {:?}", + array.data_type() + ))), + } +} + +/// Checks if an array of type `datatype` can perform upper operation +/// +/// # Examples +/// ``` +/// use arrow2::compute::utf8::can_upper; +/// use arrow2::datatypes::{DataType}; +/// +/// let data_type = DataType::Utf8; +/// assert_eq!(can_upper(&data_type), true); +/// +/// let data_type = DataType::Null; +/// assert_eq!(can_upper(&data_type), false); +/// ``` +pub fn can_upper(data_type: &DataType) -> bool { + matches!(data_type, DataType::LargeUtf8 | DataType::Utf8) +} + +/// Returns a new `Array` where each of each of the elements is lower-cased. +/// this function errors when the passed array is not a \[Large\]String array. +pub fn lower(array: &dyn Array) -> Result> { + match array.data_type() { + DataType::LargeUtf8 => Ok(Box::new(utf8_apply( + str::to_lowercase, + array + .as_any() + .downcast_ref::>() + .expect("A large string is expected"), + ))), + DataType::Utf8 => Ok(Box::new(utf8_apply( + str::to_lowercase, + array + .as_any() + .downcast_ref::>() + .expect("A string is expected"), + ))), + _ => Err(ArrowError::InvalidArgumentError(format!( + "lower does not support type {:?}", + array.data_type() + ))), + } +} + +/// Checks if an array of type `datatype` can perform lower operation +/// +/// # Examples +/// ``` +/// use arrow2::compute::utf8::can_lower; +/// use arrow2::datatypes::{DataType}; +/// +/// let data_type = DataType::Utf8; +/// assert_eq!(can_lower(&data_type), true); +/// +/// let data_type = DataType::Null; +/// assert_eq!(can_lower(&data_type), false); +/// ``` +pub fn can_lower(data_type: &DataType) -> bool { + matches!(data_type, DataType::LargeUtf8 | DataType::Utf8) +} diff --git a/src/compute/utils.rs b/src/compute/utils.rs index 269a9c9ce37..ea9e44d2896 100644 --- a/src/compute/utils.rs +++ b/src/compute/utils.rs @@ -1,7 +1,6 @@ use crate::{ - array::{Array, BooleanArray, Offset, Utf8Array}, + array::Array, bitmap::Bitmap, - datatypes::DataType, error::{ArrowError, Result}, }; @@ -14,30 +13,6 @@ pub fn combine_validities(lhs: Option<&Bitmap>, rhs: Option<&Bitmap>) -> Option< } } -pub fn unary_utf8_boolean bool>( - values: &Utf8Array, - op: F, -) -> BooleanArray { - let validity = values.validity().cloned(); - - let iterator = values.iter().map(|value| { - if value.is_none() { - return false; - }; - op(value.unwrap()) - }); - let values = Bitmap::from_trusted_len_iter(iterator); - BooleanArray::from_data(DataType::Boolean, values, validity) -} - -/// utf8_apply will apply `Fn(&str) -> String` to every value in Utf8Array. -pub fn utf8_apply String>(f: F, array: &Utf8Array) -> Utf8Array { - let iter = array.values_iter().map(f); - - let new = Utf8Array::::from_trusted_len_values_iter(iter); - new.with_validity(array.validity().cloned()) -} - // Errors iff the two arrays have a different length. #[inline] pub fn check_same_len(lhs: &dyn Array, rhs: &dyn Array) -> Result<()> { diff --git a/src/datatypes/field.rs b/src/datatypes/field.rs index f9ef6b1f0cc..07d1b760211 100644 --- a/src/datatypes/field.rs +++ b/src/datatypes/field.rs @@ -1,5 +1,8 @@ use super::{DataType, Metadata}; +#[cfg(feature = "serde_types")] +use serde_derive::{Deserialize, Serialize}; + /// Represents Arrow's metadata of a "column". /// /// A [`Field`] is the closest representation of the traditional "column": a logical type @@ -9,6 +12,7 @@ use super::{DataType, Metadata}; /// Almost all IO in this crate uses [`Field`] to represent logical information about the data /// to be serialized. #[derive(Debug, Clone, Eq, PartialEq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub struct Field { /// Its name pub name: String, diff --git a/src/datatypes/mod.rs b/src/datatypes/mod.rs index 8af8245d6c4..e7e2c9b98e8 100644 --- a/src/datatypes/mod.rs +++ b/src/datatypes/mod.rs @@ -12,6 +12,9 @@ pub use schema::Schema; use std::collections::BTreeMap; use std::sync::Arc; +#[cfg(feature = "serde_types")] +use serde_derive::{Deserialize, Serialize}; + /// typedef for [BTreeMap] denoting [`Field`]'s and [`Schema`]'s metadata. pub type Metadata = BTreeMap; /// typedef fpr [Option<(String, Option)>] descr @@ -26,6 +29,7 @@ pub(crate) type Extension = Option<(String, Option)>; /// The [`DataType::Extension`] is special in that it augments a [`DataType`] with metadata to support custom types. /// Use `to_logical_type` to desugar such type and return its correspoding logical type. #[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub enum DataType { /// Null type Null, @@ -156,6 +160,7 @@ pub enum DataType { /// Mode of [`DataType::Union`] #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub enum UnionMode { /// Dense union Dense, @@ -187,6 +192,7 @@ impl UnionMode { /// The time units defined in Arrow. #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub enum TimeUnit { /// Time in seconds. Second, @@ -200,6 +206,7 @@ pub enum TimeUnit { /// Interval units defined in Arrow #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub enum IntervalUnit { /// The number of elapsed whole months. YearMonth, diff --git a/src/datatypes/physical_type.rs b/src/datatypes/physical_type.rs index 7e15cb19629..828df9541f0 100644 --- a/src/datatypes/physical_type.rs +++ b/src/datatypes/physical_type.rs @@ -1,9 +1,13 @@ pub use crate::types::PrimitiveType; +#[cfg(feature = "serde_types")] +use serde_derive::{Deserialize, Serialize}; + /// The set of physical types: unique in-memory representations of an Arrow array. /// A physical type has a one-to-many relationship with a [`crate::datatypes::DataType`] and /// a one-to-one mapping to each struct in this crate that implements [`crate::array::Array`]. #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub enum PhysicalType { /// A Null with no allocation. Null, @@ -51,6 +55,7 @@ impl PhysicalType { /// the set of valid indices types of a dictionary-encoded Array. /// Each type corresponds to a variant of [`crate::array::DictionaryArray`]. #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub enum IntegerType { /// A signed 8-bit integer. Int8, diff --git a/src/datatypes/schema.rs b/src/datatypes/schema.rs index 38dcb5ccd41..baa04476360 100644 --- a/src/datatypes/schema.rs +++ b/src/datatypes/schema.rs @@ -1,11 +1,15 @@ use super::{Field, Metadata}; +#[cfg(feature = "serde_types")] +use serde_derive::{Deserialize, Serialize}; + /// An ordered sequence of [`Field`]s with associated [`Metadata`]. /// /// [`Schema`] is an abstration used to read from, and write to, Arrow IPC format, /// Apache Parquet, and Apache Avro. All these formats have a concept of a schema /// with fields and metadata. #[derive(Debug, Clone, PartialEq, Eq, Default)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub struct Schema { /// The fields composing this schema. pub fields: Vec, diff --git a/src/error.rs b/src/error.rs index aca64d3d659..22faa164c35 100644 --- a/src/error.rs +++ b/src/error.rs @@ -34,6 +34,7 @@ impl ArrowError { Self::OutOfSpec(msg.into()) } + #[allow(dead_code)] pub(crate) fn nyi>(msg: A) -> Self { Self::NotYetImplemented(msg.into()) } diff --git a/src/ffi/array.rs b/src/ffi/array.rs index 947a3c4e3df..abaa0363597 100644 --- a/src/ffi/array.rs +++ b/src/ffi/array.rs @@ -1,9 +1,20 @@ //! Contains functionality to load an ArrayData from the C Data Interface +use std::{ptr::NonNull, sync::Arc}; -use super::ffi::ArrowArrayRef; -use crate::array::{BooleanArray, FromFfi}; -use crate::error::Result; -use crate::{array::*, datatypes::PhysicalType}; +use crate::{ + array::*, + bitmap::{utils::bytes_for, Bitmap}, + buffer::{ + bytes::{Bytes, Deallocation}, + Buffer, + }, + datatypes::{DataType, PhysicalType}, + error::{ArrowError, Result}, + ffi::schema::get_child, + types::NativeType, +}; + +use super::ArrowArray; /// Reads a valid `ffi` interface into a `Box` /// # Errors @@ -11,7 +22,7 @@ use crate::{array::*, datatypes::PhysicalType}; /// * the interface is not valid (e.g. a null pointer) pub unsafe fn try_from(array: A) -> Result> { use PhysicalType::*; - Ok(match array.field().data_type().to_physical_type() { + Ok(match array.data_type().to_physical_type() { Null => Box::new(NullArray::try_from_ffi(array)?), Boolean => Box::new(BooleanArray::try_from_ffi(array)?), Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { @@ -35,3 +46,439 @@ pub unsafe fn try_from(array: A) -> Result> { Map => Box::new(MapArray::try_from_ffi(array)?), }) } + +// Sound because the arrow specification does not allow multiple implementations +// to change this struct +// This is intrinsically impossible to prove because the implementations agree +// on this as part of the Arrow specification +unsafe impl Send for ArrowArray {} +unsafe impl Sync for ArrowArray {} + +impl Drop for ArrowArray { + fn drop(&mut self) { + match self.release { + None => (), + Some(release) => unsafe { release(self) }, + }; + } +} + +// callback used to drop [ArrowArray] when it is exported +unsafe extern "C" fn c_release_array(array: *mut ArrowArray) { + if array.is_null() { + return; + } + let array = &mut *array; + + // take ownership of `private_data`, therefore dropping it + let private = Box::from_raw(array.private_data as *mut PrivateData); + for child in private.children_ptr.iter() { + let _ = Box::from_raw(*child); + } + + if let Some(ptr) = private.dictionary_ptr { + let _ = Box::from_raw(ptr); + } + + array.release = None; +} + +#[allow(dead_code)] +struct PrivateData { + array: Arc, + buffers_ptr: Box<[*const std::os::raw::c_void]>, + children_ptr: Box<[*mut ArrowArray]>, + dictionary_ptr: Option<*mut ArrowArray>, +} + +impl ArrowArray { + /// creates a new `ArrowArray` from existing data. + /// # Safety + /// This method releases `buffers`. Consumers of this struct *must* call `release` before + /// releasing this struct, or contents in `buffers` leak. + pub(crate) fn new(array: Arc) -> Self { + let (offset, buffers, children, dictionary) = + offset_buffers_children_dictionary(array.as_ref()); + + let buffers_ptr = buffers + .iter() + .map(|maybe_buffer| match maybe_buffer { + Some(b) => b.as_ptr() as *const std::os::raw::c_void, + None => std::ptr::null(), + }) + .collect::>(); + let n_buffers = buffers.len() as i64; + + let children_ptr = children + .into_iter() + .map(|child| Box::into_raw(Box::new(ArrowArray::new(child)))) + .collect::>(); + let n_children = children_ptr.len() as i64; + + let dictionary_ptr = + dictionary.map(|array| Box::into_raw(Box::new(ArrowArray::new(array)))); + + let length = array.len() as i64; + let null_count = array.null_count() as i64; + + let mut private_data = Box::new(PrivateData { + array, + buffers_ptr, + children_ptr, + dictionary_ptr, + }); + + Self { + length, + null_count, + offset: offset as i64, + n_buffers, + n_children, + buffers: private_data.buffers_ptr.as_mut_ptr(), + children: private_data.children_ptr.as_mut_ptr(), + dictionary: private_data.dictionary_ptr.unwrap_or(std::ptr::null_mut()), + release: Some(c_release_array), + private_data: Box::into_raw(private_data) as *mut ::std::os::raw::c_void, + } + } + + /// creates an empty [`ArrowArray`], which can be used to import data into + pub fn empty() -> Self { + Self { + length: 0, + null_count: 0, + offset: 0, + n_buffers: 0, + n_children: 0, + buffers: std::ptr::null_mut(), + children: std::ptr::null_mut(), + dictionary: std::ptr::null_mut(), + release: None, + private_data: std::ptr::null_mut(), + } + } + + /// the length of the array + pub(crate) fn len(&self) -> usize { + self.length as usize + } + + /// the offset of the array + pub(crate) fn offset(&self) -> usize { + self.offset as usize + } + + /// the null count of the array + pub(crate) fn null_count(&self) -> usize { + self.null_count as usize + } +} + +/// interprets the buffer `index` as a [`Buffer`]. +/// # Safety +/// The caller must guarantee that the buffer `index` corresponds to a buffer of type `T`. +/// This function assumes that the buffer created from FFI is valid; this is impossible to prove. +unsafe fn create_buffer( + array: &ArrowArray, + data_type: &DataType, + deallocation: Deallocation, + index: usize, +) -> Result> { + if array.buffers.is_null() { + return Err(ArrowError::OutOfSpec( + "The array buffers are null".to_string(), + )); + } + + let buffers = array.buffers as *mut *const u8; + + assert!(index < array.n_buffers as usize); + let ptr = *buffers.add(index); + let ptr = NonNull::new(ptr as *mut T); + + let len = buffer_len(array, data_type, index)?; + let offset = buffer_offset(array, data_type, index); + let bytes = ptr + .map(|ptr| Bytes::from_ffi(ptr, len, deallocation)) + .ok_or_else(|| { + ArrowError::OutOfSpec(format!("The buffer at position {} is null", index)) + })?; + + Ok(Buffer::from_bytes(bytes).slice(offset, len - offset)) +} + +/// returns a new buffer corresponding to the index `i` of the FFI array. It may not exist (null pointer). +/// `bits` is the number of bits that the native type of this buffer has. +/// The size of the buffer will be `ceil(self.length * bits, 8)`. +/// # Panic +/// This function panics if `i` is larger or equal to `n_buffers`. +/// # Safety +/// This function assumes that `ceil(self.length * bits, 8)` is the size of the buffer +unsafe fn create_bitmap( + array: &ArrowArray, + deallocation: Deallocation, + index: usize, +) -> Result { + if array.buffers.is_null() { + return Err(ArrowError::OutOfSpec( + "The array buffers are null".to_string(), + )); + } + let len = array.length as usize; + let offset = array.offset as usize; + let buffers = array.buffers as *mut *const u8; + + assert!(index < array.n_buffers as usize); + let ptr = *buffers.add(index); + + let bytes_len = bytes_for(offset + len); + let ptr = NonNull::new(ptr as *mut u8); + let bytes = ptr + .map(|ptr| Bytes::from_ffi(ptr, bytes_len, deallocation)) + .ok_or_else(|| { + ArrowError::OutOfSpec(format!( + "The buffer {} is a null pointer and cannot be interpreted as a bitmap", + index + )) + })?; + + Ok(Bitmap::from_bytes(bytes, offset + len).slice(offset, len)) +} + +fn buffer_offset(array: &ArrowArray, data_type: &DataType, i: usize) -> usize { + use PhysicalType::*; + match (data_type.to_physical_type(), i) { + (LargeUtf8, 2) | (LargeBinary, 2) | (Utf8, 2) | (Binary, 2) => 0, + _ => array.offset as usize, + } +} + +/// Returns the length, in slots, of the buffer `i` (indexed according to the C data interface) +// Rust implementation uses fixed-sized buffers, which require knowledge of their `len`. +// for variable-sized buffers, such as the second buffer of a stringArray, we need +// to fetch offset buffer's len to build the second buffer. +fn buffer_len(array: &ArrowArray, data_type: &DataType, i: usize) -> Result { + Ok(match (data_type.to_physical_type(), i) { + (PhysicalType::FixedSizeBinary, 1) => { + if let DataType::FixedSizeBinary(size) = data_type.to_logical_type() { + *size * (array.offset as usize + array.length as usize) + } else { + unreachable!() + } + } + (PhysicalType::FixedSizeList, 1) => { + if let DataType::FixedSizeList(_, size) = data_type.to_logical_type() { + *size * (array.offset as usize + array.length as usize) + } else { + unreachable!() + } + } + (PhysicalType::Utf8, 1) + | (PhysicalType::LargeUtf8, 1) + | (PhysicalType::Binary, 1) + | (PhysicalType::LargeBinary, 1) + | (PhysicalType::List, 1) + | (PhysicalType::LargeList, 1) + | (PhysicalType::Map, 1) => { + // the len of the offset buffer (buffer 1) equals length + 1 + array.offset as usize + array.length as usize + 1 + } + (PhysicalType::Utf8, 2) | (PhysicalType::Binary, 2) => { + // the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1) + let len = buffer_len(array, data_type, 1)?; + // first buffer is the null buffer => add(1) + let offset_buffer = unsafe { *(array.buffers as *mut *const u8).add(1) }; + // interpret as i32 + let offset_buffer = offset_buffer as *const i32; + // get last offset + + (unsafe { *offset_buffer.add(len - 1) }) as usize + } + (PhysicalType::LargeUtf8, 2) | (PhysicalType::LargeBinary, 2) => { + // the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1) + let len = buffer_len(array, data_type, 1)?; + // first buffer is the null buffer => add(1) + let offset_buffer = unsafe { *(array.buffers as *mut *const u8).add(1) }; + // interpret as i64 + let offset_buffer = offset_buffer as *const i64; + // get last offset + (unsafe { *offset_buffer.add(len - 1) }) as usize + } + // buffer len of primitive types + _ => array.offset as usize + array.length as usize, + }) +} + +fn create_child( + array: &ArrowArray, + field: &DataType, + parent: Arc, + index: usize, +) -> Result> { + let data_type = get_child(field, index)?; + assert!(index < array.n_children as usize); + assert!(!array.children.is_null()); + unsafe { + let arr_ptr = *array.children.add(index); + assert!(!arr_ptr.is_null()); + let arr_ptr = &*arr_ptr; + + Ok(ArrowArrayChild::from_raw(arr_ptr, data_type, parent)) + } +} + +fn create_dictionary( + array: &ArrowArray, + data_type: &DataType, + parent: Arc, +) -> Result>> { + if let DataType::Dictionary(_, values, _) = data_type { + let data_type = values.as_ref().clone(); + assert!(!array.dictionary.is_null()); + let array = unsafe { &*array.dictionary }; + Ok(Some(ArrowArrayChild::from_raw(array, data_type, parent))) + } else { + Ok(None) + } +} + +pub trait ArrowArrayRef: std::fmt::Debug { + fn deallocation(&self) -> Deallocation { + Deallocation::Foreign(self.parent().clone()) + } + + /// returns the null bit buffer. + /// Rust implementation uses a buffer that is not part of the array of buffers. + /// The C Data interface's null buffer is part of the array of buffers. + /// # Safety + /// The caller must guarantee that the buffer `index` corresponds to a bitmap. + /// This function assumes that the bitmap created from FFI is valid; this is impossible to prove. + unsafe fn validity(&self) -> Result> { + if self.array().null_count() == 0 { + Ok(None) + } else { + create_bitmap(self.array(), self.deallocation(), 0).map(Some) + } + } + + /// # Safety + /// The caller must guarantee that the buffer `index` corresponds to a bitmap. + /// This function assumes that the bitmap created from FFI is valid; this is impossible to prove. + unsafe fn buffer(&self, index: usize) -> Result> { + create_buffer::(self.array(), self.data_type(), self.deallocation(), index) + } + + /// # Safety + /// The caller must guarantee that the buffer `index` corresponds to a bitmap. + /// This function assumes that the bitmap created from FFI is valid; this is impossible to prove. + unsafe fn bitmap(&self, index: usize) -> Result { + // +1 to ignore null bitmap + create_bitmap(self.array(), self.deallocation(), index) + } + + /// # Safety + /// The caller must guarantee that the child `index` is valid per c data interface. + unsafe fn child(&self, index: usize) -> Result { + create_child(self.array(), self.data_type(), self.parent().clone(), index) + } + + fn dictionary(&self) -> Result> { + create_dictionary(self.array(), self.data_type(), self.parent().clone()) + } + + fn n_buffers(&self) -> usize; + + fn parent(&self) -> &Arc; + fn array(&self) -> &ArrowArray; + fn data_type(&self) -> &DataType; +} + +/// Struct used to move an Array from and to the C Data Interface. +/// Its main responsibility is to expose functionality that requires +/// both [ArrowArray] and [ArrowSchema]. +/// +/// This struct has two main paths: +/// +/// ## Import from the C Data Interface +/// * [InternalArrowArray::empty] to allocate memory to be filled by an external call +/// * [InternalArrowArray::try_from_raw] to consume two non-null allocated pointers +/// ## Export to the C Data Interface +/// * [InternalArrowArray::try_new] to create a new [InternalArrowArray] from Rust-specific information +/// * [InternalArrowArray::into_raw] to expose two pointers for [ArrowArray] and [ArrowSchema]. +/// +/// # Safety +/// Whoever creates this struct is responsible for releasing their resources. Specifically, +/// consumers *must* call [InternalArrowArray::into_raw] and take ownership of the individual pointers, +/// calling [ArrowArray::release] and [ArrowSchema::release] accordingly. +/// +/// Furthermore, this struct assumes that the incoming data agrees with the C data interface. +#[derive(Debug)] +pub struct InternalArrowArray { + array: Box, + data_type: DataType, +} + +impl InternalArrowArray { + pub fn new(array: Box, data_type: DataType) -> Self { + Self { array, data_type } + } +} + +impl ArrowArrayRef for Arc { + /// the data_type as declared in the schema + fn data_type(&self) -> &DataType { + &self.data_type + } + + fn parent(&self) -> &Arc { + self + } + + fn array(&self) -> &ArrowArray { + self.array.as_ref() + } + + fn n_buffers(&self) -> usize { + self.array.n_buffers as usize + } +} + +#[derive(Debug)] +pub struct ArrowArrayChild<'a> { + array: &'a ArrowArray, + data_type: DataType, + parent: Arc, +} + +impl<'a> ArrowArrayRef for ArrowArrayChild<'a> { + /// the data_type as declared in the schema + fn data_type(&self) -> &DataType { + &self.data_type + } + + fn parent(&self) -> &Arc { + &self.parent + } + + fn array(&self) -> &ArrowArray { + self.array + } + + fn n_buffers(&self) -> usize { + self.array.n_buffers as usize + } +} + +impl<'a> ArrowArrayChild<'a> { + fn from_raw( + array: &'a ArrowArray, + data_type: DataType, + parent: Arc, + ) -> Self { + Self { + array, + data_type, + parent, + } + } +} diff --git a/src/ffi/ffi.rs b/src/ffi/ffi.rs deleted file mode 100644 index 528a679dd5b..00000000000 --- a/src/ffi/ffi.rs +++ /dev/null @@ -1,491 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::{ptr::NonNull, sync::Arc}; - -use crate::{ - array::{offset_buffers_children_dictionary, Array}, - bitmap::{utils::bytes_for, Bitmap}, - buffer::{ - bytes::{Bytes, Deallocation}, - Buffer, - }, - datatypes::{DataType, Field, PhysicalType}, - error::{ArrowError, Result}, - ffi::schema::get_field_child, - types::NativeType, -}; - -/// ABI-compatible struct for ArrowArray from C Data Interface -/// See -/// This was created by bindgen -#[repr(C)] -#[derive(Debug, Clone)] -pub struct Ffi_ArrowArray { - pub(crate) length: i64, - pub(crate) null_count: i64, - pub(crate) offset: i64, - pub(crate) n_buffers: i64, - pub(crate) n_children: i64, - pub(crate) buffers: *mut *const ::std::os::raw::c_void, - children: *mut *mut Ffi_ArrowArray, - dictionary: *mut Ffi_ArrowArray, - release: ::std::option::Option, - // When exported, this MUST contain everything that is owned by this array. - // for example, any buffer pointed to in `buffers` must be here, as well as the `buffers` pointer - // itself. - // In other words, everything in [Ffi_ArrowArray] must be owned by `private_data` and can assume - // that they do not outlive `private_data`. - private_data: *mut ::std::os::raw::c_void, -} - -// Sound because the arrow specification does not allow multiple implementations -// to change this struct -// This is intrinsically impossible to prove because the implementations agree -// on this as part of the Arrow specification -unsafe impl Send for Ffi_ArrowArray {} -unsafe impl Sync for Ffi_ArrowArray {} - -impl Drop for Ffi_ArrowArray { - fn drop(&mut self) { - match self.release { - None => (), - Some(release) => unsafe { release(self) }, - }; - } -} - -// callback used to drop [Ffi_ArrowArray] when it is exported -unsafe extern "C" fn c_release_array(array: *mut Ffi_ArrowArray) { - if array.is_null() { - return; - } - let array = &mut *array; - - // take ownership of `private_data`, therefore dropping it - let private = Box::from_raw(array.private_data as *mut PrivateData); - for child in private.children_ptr.iter() { - let _ = Box::from_raw(*child); - } - - if let Some(ptr) = private.dictionary_ptr { - let _ = Box::from_raw(ptr); - } - - array.release = None; -} - -#[allow(dead_code)] -struct PrivateData { - array: Arc, - buffers_ptr: Box<[*const std::os::raw::c_void]>, - children_ptr: Box<[*mut Ffi_ArrowArray]>, - dictionary_ptr: Option<*mut Ffi_ArrowArray>, -} - -impl Ffi_ArrowArray { - /// creates a new `Ffi_ArrowArray` from existing data. - /// # Safety - /// This method releases `buffers`. Consumers of this struct *must* call `release` before - /// releasing this struct, or contents in `buffers` leak. - pub(crate) fn new(array: Arc) -> Self { - let (offset, buffers, children, dictionary) = - offset_buffers_children_dictionary(array.as_ref()); - - let buffers_ptr = buffers - .iter() - .map(|maybe_buffer| match maybe_buffer { - Some(b) => b.as_ptr() as *const std::os::raw::c_void, - None => std::ptr::null(), - }) - .collect::>(); - let n_buffers = buffers.len() as i64; - - let children_ptr = children - .into_iter() - .map(|child| Box::into_raw(Box::new(Ffi_ArrowArray::new(child)))) - .collect::>(); - let n_children = children_ptr.len() as i64; - - let dictionary_ptr = - dictionary.map(|array| Box::into_raw(Box::new(Ffi_ArrowArray::new(array)))); - - let length = array.len() as i64; - let null_count = array.null_count() as i64; - - let mut private_data = Box::new(PrivateData { - array, - buffers_ptr, - children_ptr, - dictionary_ptr, - }); - - Self { - length, - null_count, - offset: offset as i64, - n_buffers, - n_children, - buffers: private_data.buffers_ptr.as_mut_ptr(), - children: private_data.children_ptr.as_mut_ptr(), - dictionary: private_data.dictionary_ptr.unwrap_or(std::ptr::null_mut()), - release: Some(c_release_array), - private_data: Box::into_raw(private_data) as *mut ::std::os::raw::c_void, - } - } - - /// creates an empty [`Ffi_ArrowArray`], which can be used to import data into - pub fn empty() -> Self { - Self { - length: 0, - null_count: 0, - offset: 0, - n_buffers: 0, - n_children: 0, - buffers: std::ptr::null_mut(), - children: std::ptr::null_mut(), - dictionary: std::ptr::null_mut(), - release: None, - private_data: std::ptr::null_mut(), - } - } - - /// the length of the array - pub(crate) fn len(&self) -> usize { - self.length as usize - } - - /// the offset of the array - pub(crate) fn offset(&self) -> usize { - self.offset as usize - } - - /// the null count of the array - pub(crate) fn null_count(&self) -> usize { - self.null_count as usize - } -} - -/// interprets the buffer `index` as a [`Buffer`]. -/// # Safety -/// The caller must guarantee that the buffer `index` corresponds to a buffer of type `T`. -/// This function assumes that the buffer created from FFI is valid; this is impossible to prove. -unsafe fn create_buffer( - array: &Ffi_ArrowArray, - data_type: &DataType, - deallocation: Deallocation, - index: usize, -) -> Result> { - if array.buffers.is_null() { - return Err(ArrowError::OutOfSpec( - "The array buffers are null".to_string(), - )); - } - - let buffers = array.buffers as *mut *const u8; - - assert!(index < array.n_buffers as usize); - let ptr = *buffers.add(index); - let ptr = NonNull::new(ptr as *mut T); - - let len = buffer_len(array, data_type, index)?; - let offset = buffer_offset(array, data_type, index); - let bytes = ptr - .map(|ptr| Bytes::from_ffi(ptr, len, deallocation)) - .ok_or_else(|| { - ArrowError::OutOfSpec(format!("The buffer at position {} is null", index)) - })?; - - Ok(Buffer::from_bytes(bytes).slice(offset, len - offset)) -} - -/// returns a new buffer corresponding to the index `i` of the FFI array. It may not exist (null pointer). -/// `bits` is the number of bits that the native type of this buffer has. -/// The size of the buffer will be `ceil(self.length * bits, 8)`. -/// # Panic -/// This function panics if `i` is larger or equal to `n_buffers`. -/// # Safety -/// This function assumes that `ceil(self.length * bits, 8)` is the size of the buffer -unsafe fn create_bitmap( - array: &Ffi_ArrowArray, - deallocation: Deallocation, - index: usize, -) -> Result { - if array.buffers.is_null() { - return Err(ArrowError::OutOfSpec( - "The array buffers are null".to_string(), - )); - } - let len = array.length as usize; - let offset = array.offset as usize; - let buffers = array.buffers as *mut *const u8; - - assert!(index < array.n_buffers as usize); - let ptr = *buffers.add(index); - - let bytes_len = bytes_for(offset + len); - let ptr = NonNull::new(ptr as *mut u8); - let bytes = ptr - .map(|ptr| Bytes::from_ffi(ptr, bytes_len, deallocation)) - .ok_or_else(|| { - ArrowError::OutOfSpec(format!( - "The buffer {} is a null pointer and cannot be interpreted as a bitmap", - index - )) - })?; - - Ok(Bitmap::from_bytes(bytes, offset + len).slice(offset, len)) -} - -fn buffer_offset(array: &Ffi_ArrowArray, data_type: &DataType, i: usize) -> usize { - use PhysicalType::*; - match (data_type.to_physical_type(), i) { - (LargeUtf8, 2) | (LargeBinary, 2) | (Utf8, 2) | (Binary, 2) => 0, - _ => array.offset as usize, - } -} - -/// Returns the length, in slots, of the buffer `i` (indexed according to the C data interface) -// Rust implementation uses fixed-sized buffers, which require knowledge of their `len`. -// for variable-sized buffers, such as the second buffer of a stringArray, we need -// to fetch offset buffer's len to build the second buffer. -fn buffer_len(array: &Ffi_ArrowArray, data_type: &DataType, i: usize) -> Result { - Ok(match (data_type.to_physical_type(), i) { - (PhysicalType::FixedSizeBinary, 1) => { - if let DataType::FixedSizeBinary(size) = data_type.to_logical_type() { - *size * (array.offset as usize + array.length as usize) - } else { - unreachable!() - } - } - (PhysicalType::FixedSizeList, 1) => { - if let DataType::FixedSizeList(_, size) = data_type.to_logical_type() { - *size * (array.offset as usize + array.length as usize) - } else { - unreachable!() - } - } - (PhysicalType::Utf8, 1) - | (PhysicalType::LargeUtf8, 1) - | (PhysicalType::Binary, 1) - | (PhysicalType::LargeBinary, 1) - | (PhysicalType::List, 1) - | (PhysicalType::LargeList, 1) - | (PhysicalType::Map, 1) => { - // the len of the offset buffer (buffer 1) equals length + 1 - array.offset as usize + array.length as usize + 1 - } - (PhysicalType::Utf8, 2) | (PhysicalType::Binary, 2) => { - // the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1) - let len = buffer_len(array, data_type, 1)?; - // first buffer is the null buffer => add(1) - let offset_buffer = unsafe { *(array.buffers as *mut *const u8).add(1) }; - // interpret as i32 - let offset_buffer = offset_buffer as *const i32; - // get last offset - - (unsafe { *offset_buffer.add(len - 1) }) as usize - } - (PhysicalType::LargeUtf8, 2) | (PhysicalType::LargeBinary, 2) => { - // the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1) - let len = buffer_len(array, data_type, 1)?; - // first buffer is the null buffer => add(1) - let offset_buffer = unsafe { *(array.buffers as *mut *const u8).add(1) }; - // interpret as i64 - let offset_buffer = offset_buffer as *const i64; - // get last offset - (unsafe { *offset_buffer.add(len - 1) }) as usize - } - // buffer len of primitive types - _ => array.offset as usize + array.length as usize, - }) -} - -fn create_child( - array: &Ffi_ArrowArray, - field: &Field, - parent: Arc, - index: usize, -) -> Result> { - let field = get_field_child(field, index)?; - assert!(index < array.n_children as usize); - assert!(!array.children.is_null()); - unsafe { - let arr_ptr = *array.children.add(index); - assert!(!arr_ptr.is_null()); - let arr_ptr = &*arr_ptr; - - Ok(ArrowArrayChild::from_raw(arr_ptr, field, parent)) - } -} - -fn create_dictionary( - array: &Ffi_ArrowArray, - field: &Field, - parent: Arc, -) -> Result>> { - if let DataType::Dictionary(_, values, _) = field.data_type() { - let field = Field::new("", values.as_ref().clone(), true); - assert!(!array.dictionary.is_null()); - let array = unsafe { &*array.dictionary }; - Ok(Some(ArrowArrayChild::from_raw(array, field, parent))) - } else { - Ok(None) - } -} - -pub trait ArrowArrayRef: std::fmt::Debug { - fn deallocation(&self) -> Deallocation { - Deallocation::Foreign(self.parent().clone()) - } - - /// returns the null bit buffer. - /// Rust implementation uses a buffer that is not part of the array of buffers. - /// The C Data interface's null buffer is part of the array of buffers. - /// # Safety - /// The caller must guarantee that the buffer `index` corresponds to a bitmap. - /// This function assumes that the bitmap created from FFI is valid; this is impossible to prove. - unsafe fn validity(&self) -> Result> { - if self.array().null_count() == 0 { - Ok(None) - } else { - create_bitmap(self.array(), self.deallocation(), 0).map(Some) - } - } - - /// # Safety - /// The caller must guarantee that the buffer `index` corresponds to a bitmap. - /// This function assumes that the bitmap created from FFI is valid; this is impossible to prove. - unsafe fn buffer(&self, index: usize) -> Result> { - create_buffer::( - self.array(), - self.field().data_type(), - self.deallocation(), - index, - ) - } - - /// # Safety - /// The caller must guarantee that the buffer `index` corresponds to a bitmap. - /// This function assumes that the bitmap created from FFI is valid; this is impossible to prove. - unsafe fn bitmap(&self, index: usize) -> Result { - // +1 to ignore null bitmap - create_bitmap(self.array(), self.deallocation(), index) - } - - /// # Safety - /// The caller must guarantee that the child `index` is valid per c data interface. - unsafe fn child(&self, index: usize) -> Result { - create_child(self.array(), self.field(), self.parent().clone(), index) - } - - fn dictionary(&self) -> Result> { - create_dictionary(self.array(), self.field(), self.parent().clone()) - } - - fn n_buffers(&self) -> usize; - - fn parent(&self) -> &Arc; - fn array(&self) -> &Ffi_ArrowArray; - fn field(&self) -> &Field; -} - -/// Struct used to move an Array from and to the C Data Interface. -/// Its main responsibility is to expose functionality that requires -/// both [Ffi_ArrowArray] and [Ffi_ArrowSchema]. -/// -/// This struct has two main paths: -/// -/// ## Import from the C Data Interface -/// * [ArrowArray::empty] to allocate memory to be filled by an external call -/// * [ArrowArray::try_from_raw] to consume two non-null allocated pointers -/// ## Export to the C Data Interface -/// * [ArrowArray::try_new] to create a new [ArrowArray] from Rust-specific information -/// * [ArrowArray::into_raw] to expose two pointers for [Ffi_ArrowArray] and [Ffi_ArrowSchema]. -/// -/// # Safety -/// Whoever creates this struct is responsible for releasing their resources. Specifically, -/// consumers *must* call [ArrowArray::into_raw] and take ownership of the individual pointers, -/// calling [Ffi_ArrowArray::release] and [Ffi_ArrowSchema::release] accordingly. -/// -/// Furthermore, this struct assumes that the incoming data agrees with the C data interface. -#[derive(Debug)] -pub struct ArrowArray { - array: Box, - field: Field, -} - -impl ArrowArray { - pub fn new(array: Box, field: Field) -> Self { - Self { array, field } - } -} - -impl ArrowArrayRef for Arc { - /// the data_type as declared in the schema - fn field(&self) -> &Field { - &self.field - } - - fn parent(&self) -> &Arc { - self - } - - fn array(&self) -> &Ffi_ArrowArray { - self.array.as_ref() - } - - fn n_buffers(&self) -> usize { - self.array.n_buffers as usize - } -} - -#[derive(Debug)] -pub struct ArrowArrayChild<'a> { - array: &'a Ffi_ArrowArray, - field: Field, - parent: Arc, -} - -impl<'a> ArrowArrayRef for ArrowArrayChild<'a> { - /// the data_type as declared in the schema - fn field(&self) -> &Field { - &self.field - } - - fn parent(&self) -> &Arc { - &self.parent - } - - fn array(&self) -> &Ffi_ArrowArray { - self.array - } - - fn n_buffers(&self) -> usize { - self.array.n_buffers as usize - } -} - -impl<'a> ArrowArrayChild<'a> { - fn from_raw(array: &'a Ffi_ArrowArray, field: Field, parent: Arc) -> Self { - Self { - array, - field, - parent, - } - } -} diff --git a/src/ffi/generated.rs b/src/ffi/generated.rs new file mode 100644 index 00000000000..0fc00e5300e --- /dev/null +++ b/src/ffi/generated.rs @@ -0,0 +1,55 @@ +/* automatically generated by rust-bindgen 0.59.2 */ + +/// ABI-compatible struct for [`ArrowSchema`](https://arrow.apache.org/docs/format/CDataInterface.html#structure-definitions) +#[repr(C)] +#[derive(Debug, Clone)] +pub struct ArrowSchema { + pub(super) format: *const ::std::os::raw::c_char, + pub(super) name: *const ::std::os::raw::c_char, + pub(super) metadata: *const ::std::os::raw::c_char, + pub(super) flags: i64, + pub(super) n_children: i64, + pub(super) children: *mut *mut ArrowSchema, + pub(super) dictionary: *mut ArrowSchema, + pub(super) release: ::std::option::Option, + pub(super) private_data: *mut ::std::os::raw::c_void, +} + +/// ABI-compatible struct for [`ArrowArray`](https://arrow.apache.org/docs/format/CDataInterface.html#structure-definitions) +#[repr(C)] +#[derive(Debug, Clone)] +pub struct ArrowArray { + pub(super) length: i64, + pub(super) null_count: i64, + pub(super) offset: i64, + pub(super) n_buffers: i64, + pub(super) n_children: i64, + pub(super) buffers: *mut *const ::std::os::raw::c_void, + pub(super) children: *mut *mut ArrowArray, + pub(super) dictionary: *mut ArrowArray, + pub(super) release: ::std::option::Option, + pub(super) private_data: *mut ::std::os::raw::c_void, +} + +/// ABI-compatible struct for [`ArrowArrayStream`](https://arrow.apache.org/docs/format/CStreamInterface.html). +#[repr(C)] +#[derive(Debug, Clone)] +pub struct ArrowArrayStream { + pub(super) get_schema: ::std::option::Option< + unsafe extern "C" fn( + arg1: *mut ArrowArrayStream, + out: *mut ArrowSchema, + ) -> ::std::os::raw::c_int, + >, + pub(super) get_next: ::std::option::Option< + unsafe extern "C" fn( + arg1: *mut ArrowArrayStream, + out: *mut ArrowArray, + ) -> ::std::os::raw::c_int, + >, + pub(super) get_last_error: ::std::option::Option< + unsafe extern "C" fn(arg1: *mut ArrowArrayStream) -> *const ::std::os::raw::c_char, + >, + pub(super) release: ::std::option::Option, + pub(super) private_data: *mut ::std::os::raw::c_void, +} diff --git a/src/ffi/mod.rs b/src/ffi/mod.rs index bcb20c06331..f3d689b7f8c 100644 --- a/src/ffi/mod.rs +++ b/src/ffi/mod.rs @@ -2,55 +2,55 @@ //! Arrow's [C Data Interface](https://arrow.apache.org/docs/format/CDataInterface.html) mod array; mod bridge; -#[allow(clippy::module_inception)] -mod ffi; +mod generated; mod schema; +mod stream; pub(crate) use array::try_from; -pub(crate) use ffi::{ArrowArray, ArrowArrayRef}; +pub(crate) use array::{ArrowArrayRef, InternalArrowArray}; use std::sync::Arc; use crate::array::Array; -use crate::datatypes::Field; +use crate::datatypes::{DataType, Field}; use crate::error::Result; -pub use ffi::Ffi_ArrowArray; -pub use schema::Ffi_ArrowSchema; - use self::schema::to_field; +pub use generated::{ArrowArray, ArrowArrayStream, ArrowSchema}; +pub use stream::{export_iterator, ArrowArrayStreamReader}; + /// Exports an [`Arc`] to the C data interface. /// # Safety /// The pointer `ptr` must be allocated and valid -pub unsafe fn export_array_to_c(array: Arc, ptr: *mut Ffi_ArrowArray) { +pub unsafe fn export_array_to_c(array: Arc, ptr: *mut ArrowArray) { let array = bridge::align_to_c_data_interface(array); - *ptr = Ffi_ArrowArray::new(array); + std::ptr::write_unaligned(ptr, ArrowArray::new(array)); } /// Exports a [`Field`] to the C data interface. /// # Safety /// The pointer `ptr` must be allocated and valid -pub unsafe fn export_field_to_c(field: &Field, ptr: *mut Ffi_ArrowSchema) { - *ptr = Ffi_ArrowSchema::new(field) +pub unsafe fn export_field_to_c(field: &Field, ptr: *mut ArrowSchema) { + std::ptr::write_unaligned(ptr, ArrowSchema::new(field)); } /// Imports a [`Field`] from the C data interface. /// # Safety -/// This function is intrinsically `unsafe` and relies on a [`Ffi_ArrowSchema`] +/// This function is intrinsically `unsafe` and relies on a [`ArrowSchema`] /// valid according to the [C data interface](https://arrow.apache.org/docs/format/CDataInterface.html) (FFI). -pub unsafe fn import_field_from_c(field: &Ffi_ArrowSchema) -> Result { +pub unsafe fn import_field_from_c(field: &ArrowSchema) -> Result { to_field(field) } /// Imports an [`Array`] from the C data interface. /// # Safety -/// This function is intrinsically `unsafe` and relies on a [`Ffi_ArrowArray`] +/// This function is intrinsically `unsafe` and relies on a [`ArrowArray`] /// valid according to the [C data interface](https://arrow.apache.org/docs/format/CDataInterface.html) (FFI). pub unsafe fn import_array_from_c( - array: Box, - field: &Field, + array: Box, + data_type: DataType, ) -> Result> { - try_from(Arc::new(ArrowArray::new(array, field.clone()))) + try_from(Arc::new(InternalArrowArray::new(array, data_type))) } diff --git a/src/ffi/schema.rs b/src/ffi/schema.rs index 38214101879..a65d5568b7a 100644 --- a/src/ffi/schema.rs +++ b/src/ffi/schema.rs @@ -7,34 +7,19 @@ use crate::{ error::{ArrowError, Result}, }; +use super::ArrowSchema; + #[allow(dead_code)] struct SchemaPrivateData { name: CString, format: CString, metadata: Option>, - children_ptr: Box<[*mut Ffi_ArrowSchema]>, - dictionary: Option<*mut Ffi_ArrowSchema>, -} - -/// ABI-compatible struct for `ArrowSchema` from C Data Interface -/// See -// This was created by bindgen -#[repr(C)] -#[derive(Debug)] -pub struct Ffi_ArrowSchema { - format: *const ::std::os::raw::c_char, - name: *const ::std::os::raw::c_char, - metadata: *const ::std::os::raw::c_char, - flags: i64, - n_children: i64, - children: *mut *mut Ffi_ArrowSchema, - dictionary: *mut Ffi_ArrowSchema, - release: ::std::option::Option, - private_data: *mut ::std::os::raw::c_void, + children_ptr: Box<[*mut ArrowSchema]>, + dictionary: Option<*mut ArrowSchema>, } -// callback used to drop [Ffi_ArrowSchema] when it is exported. -unsafe extern "C" fn c_release_schema(schema: *mut Ffi_ArrowSchema) { +// callback used to drop [ArrowSchema] when it is exported. +unsafe extern "C" fn c_release_schema(schema: *mut ArrowSchema) { if schema.is_null() { return; } @@ -52,8 +37,8 @@ unsafe extern "C" fn c_release_schema(schema: *mut Ffi_ArrowSchema) { schema.release = None; } -impl Ffi_ArrowSchema { - /// creates a new [Ffi_ArrowSchema] +impl ArrowSchema { + /// creates a new [ArrowSchema] pub(crate) fn new(field: &Field) -> Self { let format = to_format(field.data_type()); let name = field.name.clone(); @@ -63,25 +48,25 @@ impl Ffi_ArrowSchema { // allocate (and hold) the children let children_vec = match field.data_type() { DataType::List(field) => { - vec![Box::new(Ffi_ArrowSchema::new(field.as_ref()))] + vec![Box::new(ArrowSchema::new(field.as_ref()))] } DataType::FixedSizeList(field, _) => { - vec![Box::new(Ffi_ArrowSchema::new(field.as_ref()))] + vec![Box::new(ArrowSchema::new(field.as_ref()))] } DataType::LargeList(field) => { - vec![Box::new(Ffi_ArrowSchema::new(field.as_ref()))] + vec![Box::new(ArrowSchema::new(field.as_ref()))] } DataType::Map(field, is_sorted) => { flags += (*is_sorted as i64) * 4; - vec![Box::new(Ffi_ArrowSchema::new(field.as_ref()))] + vec![Box::new(ArrowSchema::new(field.as_ref()))] } DataType::Struct(fields) => fields .iter() - .map(|field| Box::new(Ffi_ArrowSchema::new(field))) + .map(|field| Box::new(ArrowSchema::new(field))) .collect::>(), DataType::Union(fields, _, _) => fields .iter() - .map(|field| Box::new(Ffi_ArrowSchema::new(field))) + .map(|field| Box::new(ArrowSchema::new(field))) .collect::>(), _ => vec![], }; @@ -96,7 +81,7 @@ impl Ffi_ArrowSchema { flags += *is_ordered as i64; // we do not store field info in the dict values, so can't recover it all :( let field = Field::new("", values.as_ref().clone(), true); - Some(Box::new(Ffi_ArrowSchema::new(&field))) + Some(Box::new(ArrowSchema::new(&field))) } else { None }; @@ -153,7 +138,7 @@ impl Ffi_ArrowSchema { } } - /// create an empty [Ffi_ArrowSchema] + /// create an empty [ArrowSchema] pub fn empty() -> Self { Self { format: std::ptr::null_mut(), @@ -202,7 +187,7 @@ impl Ffi_ArrowSchema { } } -impl Drop for Ffi_ArrowSchema { +impl Drop for ArrowSchema { fn drop(&mut self) { match self.release { None => (), @@ -211,7 +196,7 @@ impl Drop for Ffi_ArrowSchema { } } -pub(crate) unsafe fn to_field(schema: &Ffi_ArrowSchema) -> Result { +pub(crate) unsafe fn to_field(schema: &ArrowSchema) -> Result { let dictionary = schema.dictionary(); let data_type = if let Some(dictionary) = dictionary { let indices = to_integer_type(schema.format())?; @@ -251,7 +236,7 @@ fn to_integer_type(format: &str) -> Result { }) } -unsafe fn to_data_type(schema: &Ffi_ArrowSchema) -> Result { +unsafe fn to_data_type(schema: &ArrowSchema) -> Result { Ok(match schema.format() { "n" => DataType::Null, "b" => DataType::Boolean, @@ -455,14 +440,14 @@ fn to_format(data_type: &DataType) -> String { } } -pub(super) fn get_field_child(field: &Field, index: usize) -> Result { - match (index, field.data_type()) { - (0, DataType::List(field)) => Ok(field.as_ref().clone()), - (0, DataType::FixedSizeList(field, _)) => Ok(field.as_ref().clone()), - (0, DataType::LargeList(field)) => Ok(field.as_ref().clone()), - (0, DataType::Map(field, _)) => Ok(field.as_ref().clone()), - (index, DataType::Struct(fields)) => Ok(fields[index].clone()), - (index, DataType::Union(fields, _, _)) => Ok(fields[index].clone()), +pub(super) fn get_child(data_type: &DataType, index: usize) -> Result { + match (index, data_type) { + (0, DataType::List(field)) => Ok(field.data_type().clone()), + (0, DataType::FixedSizeList(field, _)) => Ok(field.data_type().clone()), + (0, DataType::LargeList(field)) => Ok(field.data_type().clone()), + (0, DataType::Map(field, _)) => Ok(field.data_type().clone()), + (index, DataType::Struct(fields)) => Ok(fields[index].data_type().clone()), + (index, DataType::Union(fields, _, _)) => Ok(fields[index].data_type().clone()), (child, data_type) => Err(ArrowError::OutOfSpec(format!( "Requested child {} to type {:?} that has no such child", child, data_type diff --git a/src/ffi/stream.rs b/src/ffi/stream.rs new file mode 100644 index 00000000000..6a316a6847c --- /dev/null +++ b/src/ffi/stream.rs @@ -0,0 +1,223 @@ +use std::ffi::{CStr, CString}; +use std::sync::Arc; + +use crate::{array::Array, datatypes::Field, error::ArrowError}; + +use super::{export_array_to_c, export_field_to_c, import_array_from_c, import_field_from_c}; +use super::{ArrowArray, ArrowArrayStream, ArrowSchema}; + +impl Drop for ArrowArrayStream { + fn drop(&mut self) { + match self.release { + None => (), + Some(release) => unsafe { release(self) }, + }; + } +} + +impl ArrowArrayStream { + /// Creates an empty [`ArrowArrayStream`] used to import from a producer. + pub fn empty() -> Self { + Self { + get_schema: None, + get_next: None, + get_last_error: None, + release: None, + private_data: std::ptr::null_mut(), + } + } +} + +unsafe fn handle_error(iter: &mut ArrowArrayStream) -> ArrowError { + let error = unsafe { (iter.get_last_error.unwrap())(&mut *iter) }; + + if error.is_null() { + return ArrowError::External( + "C stream".to_string(), + Box::new(ArrowError::ExternalFormat( + "an unspecified error".to_string(), + )), + ); + } + + let error = unsafe { CStr::from_ptr(error) }; + ArrowError::External( + "C stream".to_string(), + Box::new(ArrowError::ExternalFormat( + error.to_str().unwrap().to_string(), + )), + ) +} + +/// Implements an iterator of [`Array`] consumed from the [C stream interface](https://arrow.apache.org/docs/format/CStreamInterface.html). +pub struct ArrowArrayStreamReader { + iter: Box, + field: Field, +} + +impl ArrowArrayStreamReader { + /// Returns a new [`ArrowArrayStreamReader`] + /// # Error + /// Errors iff the [`ArrowArrayStream`] is out of specification + /// # Safety + /// This method is intrinsically `unsafe` since it assumes that the `ArrowArrayStream` + /// contains a valid Arrow C stream interface. + /// In particular: + /// * The `ArrowArrayStream` fulfills the invariants of the C stream interface + /// * The schema `get_schema` produces fulfills the C data interface + pub unsafe fn try_new(mut iter: Box) -> Result { + let mut field = Box::new(ArrowSchema::empty()); + + if iter.get_next.is_none() { + return Err(ArrowError::OutOfSpec( + "The C stream MUST contain a non-null get_next".to_string(), + )); + }; + + if iter.get_last_error.is_none() { + return Err(ArrowError::OutOfSpec( + "The C stream MUST contain a non-null get_last_error".to_string(), + )); + }; + + let status = if let Some(f) = iter.get_schema { + unsafe { (f)(&mut *iter, &mut *field) } + } else { + return Err(ArrowError::OutOfSpec( + "The C stream MUST contain a non-null get_schema".to_string(), + )); + }; + + if status != 0 { + return Err(unsafe { handle_error(&mut iter) }); + } + + let field = unsafe { import_field_from_c(&field)? }; + + Ok(Self { iter, field }) + } + + /// Returns the field provided by the stream + pub fn field(&self) -> &Field { + &self.field + } + + /// Advances this iterator by one array + /// # Error + /// Errors iff: + /// * The C stream interface returns an error + /// * The C stream interface returns an invalid array (that we can identify, see Safety below) + /// # Safety + /// Calling this iterator's `next` assumes that the [`ArrowArrayStream`] produces arrow arrays + /// that fulfill the C data interface + pub unsafe fn next(&mut self) -> Option, ArrowError>> { + let mut array = Box::new(ArrowArray::empty()); + let status = unsafe { (self.iter.get_next.unwrap())(&mut *self.iter, &mut *array) }; + + if status != 0 { + return Some(Err(unsafe { handle_error(&mut self.iter) })); + } + + // last paragraph of https://arrow.apache.org/docs/format/CStreamInterface.html#c.ArrowArrayStream.get_next + array.release?; + + // Safety: assumed from the C stream interface + unsafe { import_array_from_c(array, self.field.data_type.clone()) } + .map(Some) + .transpose() + } +} + +struct PrivateData { + iter: Box, ArrowError>>>, + field: Field, + error: Option, +} + +unsafe extern "C" fn get_next(iter: *mut ArrowArrayStream, array: *mut ArrowArray) -> i32 { + if iter.is_null() { + return 2001; + } + let mut private = &mut *((*iter).private_data as *mut PrivateData); + + match private.iter.next() { + Some(Ok(item)) => { + // check that the array has the same data_type as field + let item_dt = item.data_type(); + let expected_dt = private.field.data_type(); + if item_dt != expected_dt { + private.error = Some(CString::new(format!("The iterator produced an item of data type {item_dt:?} but the producer expects data type {expected_dt:?}").as_bytes().to_vec()).unwrap()); + return 2001; // custom application specific error (since this is never a result of this interface) + } + + export_array_to_c(item, array); + private.error = None; + 0 + } + Some(Err(err)) => { + private.error = Some(CString::new(err.to_string().as_bytes().to_vec()).unwrap()); + 2001 // custom application specific error (since this is never a result of this interface) + } + None => { + let a = ArrowArray::empty(); + std::ptr::write_unaligned(array, a); + private.error = None; + 0 + } + } +} + +unsafe extern "C" fn get_schema(iter: *mut ArrowArrayStream, schema: *mut ArrowSchema) -> i32 { + if iter.is_null() { + return 2001; + } + let private = &mut *((*iter).private_data as *mut PrivateData); + + export_field_to_c(&private.field, schema); + 0 +} + +unsafe extern "C" fn get_last_error(iter: *mut ArrowArrayStream) -> *const ::std::os::raw::c_char { + if iter.is_null() { + return std::ptr::null(); + } + let private = &mut *((*iter).private_data as *mut PrivateData); + + private + .error + .as_ref() + .map(|x| x.as_ptr()) + .unwrap_or(std::ptr::null()) +} + +unsafe extern "C" fn release(iter: *mut ArrowArrayStream) { + if iter.is_null() { + return; + } + let _ = Box::from_raw((*iter).private_data as *mut PrivateData); + (*iter).release = None; + // private drops automatically +} + +/// Exports an iterator to the [C stream interface](https://arrow.apache.org/docs/format/CStreamInterface.html) +/// # Safety +/// The pointer `consumer` must be allocated +pub unsafe fn export_iterator( + iter: Box, ArrowError>>>, + field: Field, + consumer: *mut ArrowArrayStream, +) { + let private_data = Box::new(PrivateData { + iter, + field, + error: None, + }); + + *consumer = ArrowArrayStream { + get_schema: Some(get_schema), + get_next: Some(get_next), + get_last_error: Some(get_last_error), + release: Some(release), + private_data: Box::into_raw(private_data) as *mut ::std::os::raw::c_void, + } +} diff --git a/src/io/avro/read/decompress.rs b/src/io/avro/read/decompress.rs index 03fb6261bd6..30f28de9226 100644 --- a/src/io/avro/read/decompress.rs +++ b/src/io/avro/read/decompress.rs @@ -9,6 +9,8 @@ use super::super::{Block, CompressedBlock}; use super::BlockStreamIterator; use super::Compression; +const CRC_TABLE: crc::Crc = crc::Crc::::new(&crc::CRC_32_ISO_HDLC); + /// Decompresses an Avro block. /// Returns whether the buffers where swapped. pub fn decompress_block( @@ -46,7 +48,8 @@ pub fn decompress_block( .map_err(|e| ArrowError::ExternalFormat(e.to_string()))?; let expected_crc = u32::from_be_bytes([crc[0], crc[1], crc[2], crc[3]]); - let actual_crc = crc::crc32::checksum_ieee(decompressed); + + let actual_crc = CRC_TABLE.checksum(decompressed); if expected_crc != actual_crc { return Err(ArrowError::ExternalFormat( "The crc of snap-compressed block does not match".to_string(), diff --git a/src/io/avro/write/compress.rs b/src/io/avro/write/compress.rs index 1e35311ef81..4a98f78860a 100644 --- a/src/io/avro/write/compress.rs +++ b/src/io/avro/write/compress.rs @@ -5,6 +5,8 @@ use crate::error::Result; use super::Compression; use super::{Block, CompressedBlock}; +const CRC_TABLE: crc::Crc = crc::Crc::::new(&crc::CRC_32_ISO_HDLC); + /// Compresses a [`Block`] to a [`CompressedBlock`]. pub fn compress( block: &mut Block, @@ -42,8 +44,7 @@ pub fn compress( .map_err(|e| crate::error::ArrowError::ExternalFormat(e.to_string()))?; compressed.truncate(compressed_bytes); - let crc = crc::crc32::checksum_ieee(block); - compressed.extend(crc.to_be_bytes()); + compressed.extend(CRC_TABLE.checksum(block).to_be_bytes()); Ok(false) } #[cfg(not(feature = "io_avro_compression"))] diff --git a/src/io/csv/write/mod.rs b/src/io/csv/write/mod.rs index 37d926f482e..275705c744e 100644 --- a/src/io/csv/write/mod.rs +++ b/src/io/csv/write/mod.rs @@ -6,7 +6,7 @@ use super::super::iterator::StreamingIterator; use std::io::Write; // re-export necessary public APIs from csv -pub use csv::{ByteRecord, Writer, WriterBuilder}; +pub use csv::{ByteRecord, WriterBuilder}; pub use serialize::*; @@ -26,55 +26,87 @@ fn new_serializers<'a, A: AsRef>( .collect() } -/// Serializes [`Chunk`] to a vector of `ByteRecord`. +/// Serializes [`Chunk`] to a vector of rows. /// The vector is guaranteed to have `columns.len()` entries. -/// Each `ByteRecord` is guaranteed to have `columns.array().len()` fields. +/// Each `row` is guaranteed to have `columns.array().len()` fields. pub fn serialize>( columns: &Chunk, options: &SerializeOptions, -) -> Result> { +) -> Result>> { let mut serializers = new_serializers(columns, options)?; - let rows = columns.len(); - let mut records = vec![ByteRecord::with_capacity(0, columns.arrays().len()); rows]; - records.iter_mut().for_each(|record| { + let mut rows = Vec::with_capacity(columns.len()); + let mut row = vec![]; + + // this is where the (expensive) transposition happens: the outer loop is on rows, the inner on columns + (0..columns.len()).try_for_each(|_| { serializers .iter_mut() - // `unwrap` is infalible because `array.len()` equals `len` in `Chunk::len` - .for_each(|iter| record.push_field(iter.next().unwrap())); - }); - Ok(records) + // `unwrap` is infalible because `array.len()` equals `Chunk::len` + .for_each(|iter| { + let field = iter.next().unwrap(); + row.extend_from_slice(field); + row.push(options.delimiter); + }); + if !row.is_empty() { + // replace last delimiter with new line + let last_byte = row.len() - 1; + row[last_byte] = b'\n'; + rows.push(row.clone()); + row.clear(); + } + Result::Ok(()) + })?; + + Ok(rows) } /// Writes [`Chunk`] to `writer` according to the serialization options `options`. pub fn write_chunk>( - writer: &mut Writer, + writer: &mut W, columns: &Chunk, options: &SerializeOptions, ) -> Result<()> { let mut serializers = new_serializers(columns.arrays(), options)?; let rows = columns.len(); - let mut record = ByteRecord::with_capacity(0, columns.arrays().len()); + let mut row = Vec::with_capacity(columns.arrays().len() * 10); // this is where the (expensive) transposition happens: the outer loop is on rows, the inner on columns (0..rows).try_for_each(|_| { serializers .iter_mut() // `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(); + .for_each(|iter| { + let field = iter.next().unwrap(); + row.extend_from_slice(field); + row.push(options.delimiter); + }); + // replace last delimiter with new line + let last_byte = row.len() - 1; + row[last_byte] = b'\n'; + writer.write_all(&row)?; + row.clear(); Result::Ok(()) })?; Ok(()) } /// Writes a CSV header to `writer` -pub fn write_header(writer: &mut Writer, names: &[T]) -> Result<()> +pub fn write_header( + writer: &mut W, + names: &[T], + options: &SerializeOptions, +) -> Result<()> where T: AsRef, { - writer.write_record(names.iter().map(|x| x.as_ref().as_bytes()))?; + let names = names.iter().map(|x| x.as_ref()).collect::>(); + writer.write_all( + names + .join(std::str::from_utf8(&[options.delimiter]).unwrap()) + .as_bytes(), + )?; + writer.write_all(&[b'\n'])?; Ok(()) } diff --git a/src/io/csv/write/serialize.rs b/src/io/csv/write/serialize.rs index 660bde3da1b..22f26e5a27a 100644 --- a/src/io/csv/write/serialize.rs +++ b/src/io/csv/write/serialize.rs @@ -12,11 +12,12 @@ use crate::{ use super::super::super::iterator::{BufStreamingIterator, StreamingIterator}; use crate::array::{DictionaryArray, DictionaryKey, Offset}; +use csv_core::WriteResult; use std::any::Any; /// Options to serialize logical types to CSV /// The default is to format times and dates as `chrono` crate formats them. -#[derive(Debug, PartialEq, Eq, Hash, Clone, Default)] +#[derive(Debug, PartialEq, Eq, Hash, Clone)] pub struct SerializeOptions { /// used for [`DataType::Date32`] pub date32_format: Option, @@ -28,6 +29,24 @@ pub struct SerializeOptions { pub time64_format: Option, /// used for [`DataType::Timestamp`] pub timestamp_format: Option, + /// used as separator/delimiter + pub delimiter: u8, + /// quoting character + pub quote: u8, +} + +impl Default for SerializeOptions { + fn default() -> Self { + SerializeOptions { + date32_format: None, + date64_format: None, + time32_format: None, + time64_format: None, + timestamp_format: None, + delimiter: b',', + quote: b'"', + } + } } fn primitive_write<'a, T: NativeType + ToLexical>( @@ -187,6 +206,40 @@ fn timestamp_with_tz<'a>( } } +fn new_utf8_serializer<'a, O: Offset>( + array: &'a Utf8Array, + options: &'a SerializeOptions, +) -> Box + 'a> { + let mut local_buf = vec![0u8; 64]; + let mut ser_writer = csv_core::WriterBuilder::new().quote(options.quote).build(); + + Box::new(BufStreamingIterator::new( + array.iter(), + move |x, buf| { + match x { + // Empty strings are quoted. + // This will ensure a csv parser will not read them as missing + // in a delimited field + Some("") => buf.extend_from_slice(b"\"\""), + Some(s) => loop { + match ser_writer.field(s.as_bytes(), &mut local_buf) { + (WriteResult::OutputFull, _, _) => { + let additional = local_buf.len(); + local_buf.extend(std::iter::repeat(0u8).take(additional)) + } + (WriteResult::InputEmpty, _, n_out) => { + buf.extend_from_slice(&local_buf[..n_out]); + break; + } + } + }, + _ => {} + } + }, + vec![], + )) +} + /// Returns a [`StreamingIterator`] that yields `&[u8]` serialized from `array` according to `options`. /// For numeric types, this serializes as usual. For dates, times and timestamps, it uses `options` to /// Supported types: @@ -337,27 +390,11 @@ pub fn new_serializer<'a>( } DataType::Utf8 => { let array = array.as_any().downcast_ref::>().unwrap(); - Box::new(BufStreamingIterator::new( - array.iter(), - |x, buf| { - if let Some(x) = x { - buf.extend_from_slice(x.as_bytes()); - } - }, - vec![], - )) + new_utf8_serializer(array, options) } DataType::LargeUtf8 => { let array = array.as_any().downcast_ref::>().unwrap(); - Box::new(BufStreamingIterator::new( - array.iter(), - |x, buf| { - if let Some(x) = x { - buf.extend_from_slice(x.as_bytes()); - } - }, - vec![], - )) + new_utf8_serializer(array, options) } DataType::Binary => { let array = array.as_any().downcast_ref::>().unwrap(); diff --git a/src/io/ipc/read/common.rs b/src/io/ipc/read/common.rs index 94090b7a1a1..97f1f4e03fd 100644 --- a/src/io/ipc/read/common.rs +++ b/src/io/ipc/read/common.rs @@ -121,8 +121,7 @@ pub fn read_record_batch( Ok(None) } }) - .map(|x| x.transpose()) - .flatten() + .filter_map(|x| x.transpose()) .collect::>>()? } else { fields diff --git a/src/io/ipc/write/serialize.rs b/src/io/ipc/write/serialize.rs index 0ff8c5939e4..9c0c7de8a51 100644 --- a/src/io/ipc/write/serialize.rs +++ b/src/io/ipc/write/serialize.rs @@ -657,15 +657,9 @@ fn write_bytes( } } else { arrow_data.extend_from_slice(bytes); - pad_buffer_to_8(arrow_data, arrow_data.len() - start); }; - let total_len = (arrow_data.len() - start) as i64; - buffers.push(ipc::Buffer { - offset: *offset, - length: total_len, - }); - *offset += total_len; + buffers.push(finish_buffer(arrow_data, start, offset)); } fn write_bitmap( @@ -712,15 +706,9 @@ fn write_buffer( _write_compressed_buffer(buffer, arrow_data, is_little_endian, compression); } else { _write_buffer(buffer, arrow_data, is_little_endian); - pad_buffer_to_8(arrow_data, arrow_data.len() - start); }; - let total_len = (arrow_data.len() - start) as i64; - buffers.push(ipc::Buffer { - offset: *offset, - length: total_len, - }); - *offset += total_len; + buffers.push(finish_buffer(arrow_data, start, offset)); } #[inline] @@ -819,13 +807,21 @@ fn write_buffer_from_iter>( _write_compressed_buffer_from_iter(buffer, arrow_data, is_little_endian, compression); } else { _write_buffer_from_iter(buffer, arrow_data, is_little_endian); - pad_buffer_to_8(arrow_data, arrow_data.len() - start); } + buffers.push(finish_buffer(arrow_data, start, offset)); +} + +fn finish_buffer(arrow_data: &mut Vec, start: usize, offset: &mut i64) -> ipc::Buffer { + let buffer_len = (arrow_data.len() - start) as i64; + + pad_buffer_to_8(arrow_data, arrow_data.len() - start); let total_len = (arrow_data.len() - start) as i64; - buffers.push(ipc::Buffer { + + let buffer = ipc::Buffer { offset: *offset, - length: total_len, - }); + length: buffer_len, + }; *offset += total_len; + buffer } diff --git a/src/io/json/read/deserialize.rs b/src/io/json/read/deserialize.rs index 4a1b2255b29..7f5844ab1a3 100644 --- a/src/io/json/read/deserialize.rs +++ b/src/io/json/read/deserialize.rs @@ -10,8 +10,7 @@ use serde_json::Value; use crate::{ array::*, bitmap::MutableBitmap, - chunk::Chunk, - datatypes::{DataType, Field, IntervalUnit}, + datatypes::{DataType, IntervalUnit}, error::ArrowError, types::NativeType, }; @@ -203,7 +202,7 @@ fn deserialize_dictionary>( DictionaryArray::::from_data(keys, values) } -fn _deserialize>(rows: &[A], data_type: DataType) -> Arc { +pub(crate) fn _deserialize>(rows: &[A], data_type: DataType) -> Arc { match &data_type { DataType::Null => Arc::new(NullArray::from_data(data_type, rows.len())), DataType::Boolean => Arc::new(deserialize_boolean(rows)), @@ -251,30 +250,20 @@ fn _deserialize>(rows: &[A], data_type: DataType) -> Arc>( - rows: &[A], - fields: &[Field], -) -> Result>, ArrowError> { - let data_type = DataType::Struct(fields.to_vec()); - - // convert rows to `Value` - let rows = rows - .iter() - .map(|row| { - let row: Value = serde_json::from_str(row.as_ref()).map_err(ArrowError::from)?; - Ok(row) - }) - .collect::, ArrowError>>()?; - - let (_, columns, _) = deserialize_struct(&rows, data_type).into_data(); - Ok(Chunk::new(columns)) -} - -/// Deserializes a slice of [`Value`] to an Array of logical type [`DataType`]. -/// -/// This function allows consuming deserialized JSON to Arrow. -pub fn deserialize_json(rows: &[Value], data_type: DataType) -> Arc { - _deserialize(rows, data_type) +/// # Error +/// This function errors iff either: +/// * `json` is not a [`Value::Array`] +/// * `data_type` is neither [`DataType::List`] nor [`DataType::LargeList`] +pub fn deserialize(json: &Value, data_type: DataType) -> Result, ArrowError> { + match json { + Value::Array(rows) => match data_type { + DataType::List(inner) | DataType::LargeList(inner) => { + Ok(_deserialize(rows, inner.data_type)) + } + _ => Err(ArrowError::nyi("read an Array from a non-Array data type")), + }, + _ => Err(ArrowError::nyi("read an Array from a non-Array JSON")), + } } diff --git a/src/io/json/read/infer_schema.rs b/src/io/json/read/infer_schema.rs index 81f48335d01..0bfbaeb31c1 100644 --- a/src/io/json/read/infer_schema.rs +++ b/src/io/json/read/infer_schema.rs @@ -1,144 +1,64 @@ use std::borrow::Borrow; -use std::io::{BufRead, Seek, SeekFrom}; use indexmap::map::IndexMap as HashMap; use indexmap::set::IndexSet as HashSet; use serde_json::Value; use crate::datatypes::*; -use crate::error::{ArrowError, Result}; - -use super::iterator::ValueIter; - -type Tracker = HashMap>; +use crate::error::Result; const ITEM_NAME: &str = "item"; -/// Infers the fields of a JSON file by reading the first `number_of_rows` rows. -/// # Examples -/// ``` -/// use std::io::Cursor; -/// use arrow2::io::json::read::infer; -/// -/// let data = r#"{"a":1, "b":[2.0, 1.3, -6.1], "c":[false, true], "d":4.1} -/// {"a":-10, "b":[2.0, 1.3, -6.1], "c":null, "d":null} -/// {"a":2, "b":[2.0, null, -6.1], "c":[false, null], "d":"text"} -/// {"a":3, "b":4, "c": true, "d":[1, false, "array", 2.4]} -/// "#; -/// -/// // file's cursor's offset at 0 -/// let mut reader = Cursor::new(data); -/// let fields = infer(&mut reader, None).unwrap(); -/// ``` -pub fn infer(reader: &mut R, number_of_rows: Option) -> Result> { - infer_iterator(ValueIter::new(reader, number_of_rows)) -} - -/// Infer [`Field`]s from an iterator of [`Value`]. -pub fn infer_iterator(value_iter: I) -> Result> -where - I: Iterator>, - A: Borrow, -{ - let mut values: Tracker = Tracker::new(); - - for record in value_iter { - match record?.borrow() { - Value::Object(map) => map.iter().try_for_each(|(k, v)| { - let data_type = infer_value(v)?; - add_or_insert(&mut values, k, data_type); - Result::Ok(()) - }), - value => Err(ArrowError::ExternalFormat(format!( - "Expected JSON record to be an object, found {:?}", - value - ))), - }?; - } - - Ok(resolve_fields(values)) -} - -/// Infer the fields of a JSON file from `number_of_rows` in `reader`. -/// -/// This function seeks back to the start of the `reader`. -/// -/// # Examples -/// ``` -/// use std::fs::File; -/// use std::io::Cursor; -/// use arrow2::io::json::read::infer_and_reset; -/// -/// let data = r#"{"a":1, "b":[2.0, 1.3, -6.1], "c":[false, true], "d":4.1} -/// {"a":-10, "b":[2.0, 1.3, -6.1], "c":null, "d":null} -/// {"a":2, "b":[2.0, null, -6.1], "c":[false, null], "d":"text"} -/// {"a":3, "b":4, "c": true, "d":[1, false, "array", 2.4]} -/// "#; -/// let mut reader = Cursor::new(data); -/// let fields = infer_and_reset(&mut reader, None).unwrap(); -/// // cursor's position automatically set at 0 -/// ``` -pub fn infer_and_reset( - reader: &mut R, - number_of_rows: Option, -) -> Result> { - let fields = infer(reader, number_of_rows); - reader.seek(SeekFrom::Start(0))?; - fields -} - -fn infer_value(value: &Value) -> Result { - Ok(match value { +/// Infers [`DataType`] from [`Value`]. +pub fn infer(json: &Value) -> Result { + Ok(match json { Value::Bool(_) => DataType::Boolean, Value::Array(array) => infer_array(array)?, Value::Null => DataType::Null, Value::Number(number) => infer_number(number), Value::String(_) => DataType::Utf8, - Value::Object(inner) => { - let fields = inner - .iter() - .map(|(key, value)| infer_value(value).map(|dt| Field::new(key, dt, true))) - .collect::>>()?; - DataType::Struct(fields) - } + Value::Object(inner) => infer_object(inner)?, }) } -/// Infers a [`DataType`] from a list of JSON values -pub fn infer_rows(rows: &[Value]) -> Result { - let types = rows.iter().map(|a| { - Ok(match a { - Value::Null => None, - Value::Number(n) => Some(infer_number(n)), - Value::Bool(_) => Some(DataType::Boolean), - Value::String(_) => Some(DataType::Utf8), - Value::Array(array) => Some(infer_array(array)?), - Value::Object(inner) => { - let fields = inner - .iter() - .map(|(key, value)| infer_value(value).map(|dt| Field::new(key, dt, true))) - .collect::>>()?; - Some(DataType::Struct(fields)) - } +fn filter_map_nulls(dt: DataType) -> Option { + if dt == DataType::Null { + None + } else { + Some(dt) + } +} + +fn infer_object(inner: &serde_json::Map) -> Result { + let fields = inner + .iter() + .filter_map(|(key, value)| { + infer(value) + .map(|dt| filter_map_nulls(dt).map(|dt| (key, dt))) + .transpose() + }) + .map(|maybe_dt| { + let (key, dt) = maybe_dt?; + Ok(Field::new(key, dt, true)) }) - }); - // discard None values and deduplicate entries - let types = types - .into_iter() - .map(|x| x.transpose()) - .flatten() + .collect::>>()?; + Ok(DataType::Struct(fields)) +} + +fn infer_array(values: &[Value]) -> Result { + let types = values + .iter() + .map(infer) + .filter_map(|x| x.map(filter_map_nulls).transpose()) + // deduplicate entries .collect::>>()?; - Ok(if !types.is_empty() { + let dt = if !types.is_empty() { let types = types.into_iter().collect::>(); coerce_data_type(&types) } else { DataType::Null - }) -} - -fn infer_array(values: &[Value]) -> Result { - let dt = infer_rows(values)?; + }; // if a record contains only nulls, it is not // added to values @@ -157,36 +77,12 @@ fn infer_number(n: &serde_json::Number) -> DataType { } } -fn add_or_insert(values: &mut Tracker, key: &str, data_type: DataType) { - if data_type == DataType::Null { - return; - } - if values.contains_key(key) { - let x = values.get_mut(key).unwrap(); - x.insert(data_type); - } else { - // create hashset and add value type - let mut hs = HashSet::new(); - hs.insert(data_type); - values.insert(key.to_string(), hs); - } -} - -fn resolve_fields(spec: HashMap>) -> Vec { - spec.iter() - .map(|(k, hs)| { - let v: Vec<&DataType> = hs.iter().collect(); - Field::new(k, coerce_data_type(&v), true) - }) - .collect() -} - /// Coerce an heterogeneous set of [`DataType`] into a single one. Rules: /// * `Int64` and `Float64` are `Float64` /// * Lists and scalars are coerced to a list of a compatible scalar /// * Structs contain the union of all fields /// * All other types are coerced to `Utf8` -fn coerce_data_type>(datatypes: &[A]) -> DataType { +pub(crate) fn coerce_data_type>(datatypes: &[A]) -> DataType { use DataType::*; let are_all_equal = datatypes.windows(2).all(|w| w[0].borrow() == w[1].borrow()); @@ -207,14 +103,16 @@ fn coerce_data_type>(datatypes: &[A]) -> DataType { }); // group fields by unique let fields = fields.iter().fold( - HashMap::<&String, Vec<&DataType>>::new(), + HashMap::<&String, HashSet<&DataType>>::new(), |mut acc, field| { match acc.entry(&field.name) { indexmap::map::Entry::Occupied(mut v) => { - v.get_mut().push(&field.data_type); + v.get_mut().insert(&field.data_type); } indexmap::map::Entry::Vacant(v) => { - v.insert(vec![&field.data_type]); + let mut a = HashSet::new(); + a.insert(&field.data_type); + v.insert(a); } } acc @@ -223,7 +121,10 @@ fn coerce_data_type>(datatypes: &[A]) -> DataType { // and finally, coerce each of the fields within the same name let fields = fields .into_iter() - .map(|(name, dts)| Field::new(name, coerce_data_type(&dts), true)) + .map(|(name, dts)| { + let dts = dts.into_iter().collect::>(); + Field::new(name, coerce_data_type(&dts), true) + }) .collect(); return Struct(fields); } else if datatypes.len() > 2 { diff --git a/src/io/json/read/iterator.rs b/src/io/json/read/iterator.rs deleted file mode 100644 index 7670fa32e8f..00000000000 --- a/src/io/json/read/iterator.rs +++ /dev/null @@ -1,56 +0,0 @@ -use std::io::BufRead; - -use serde_json::Value; - -use crate::error::{ArrowError, Result}; - -#[derive(Debug)] -pub struct ValueIter<'a, R: BufRead> { - reader: &'a mut R, - remaining: usize, - // reuse line buffer to avoid allocation on each record - line_buf: String, -} - -impl<'a, R: BufRead> ValueIter<'a, R> { - pub fn new(reader: &'a mut R, number_of_rows: Option) -> Self { - Self { - reader, - remaining: number_of_rows.unwrap_or(usize::MAX), - line_buf: String::new(), - } - } -} - -impl<'a, R: BufRead> Iterator for ValueIter<'a, R> { - type Item = Result; - - fn next(&mut self) -> Option { - if self.remaining == 0 { - return None; - } - - loop { - self.line_buf.truncate(0); - match self.reader.read_line(&mut self.line_buf) { - Ok(0) => { - // read_line returns 0 when stream reached EOF - return None; - } - Err(e) => { - return Some(Err(ArrowError::from(e))); - } - _ => { - let trimmed_s = self.line_buf.trim(); - if trimmed_s.is_empty() { - // ignore empty lines - continue; - } - - self.remaining -= 1; - return Some(serde_json::from_str(trimmed_s).map_err(ArrowError::from)); - } - } - } - } -} diff --git a/src/io/json/read/mod.rs b/src/io/json/read/mod.rs index 741f375480d..908da67b51f 100644 --- a/src/io/json/read/mod.rs +++ b/src/io/json/read/mod.rs @@ -1,34 +1,8 @@ //! APIs to read and deserialize from JSON mod deserialize; mod infer_schema; -mod iterator; -use crate::error::{ArrowError, Result}; - -pub use deserialize::{deserialize, deserialize_json}; -pub use infer_schema::*; - -/// Reads rows from `reader` into `rows`. Returns the number of read items. -/// IO-bounded. -pub fn read_rows(reader: &mut R, rows: &mut [String]) -> Result { - let mut row_number = 0; - for row in rows.iter_mut() { - loop { - row.truncate(0); - let _ = reader.read_line(row).map_err(|e| { - ArrowError::External(format!(" at line {}", row_number), Box::new(e)) - })?; - if row.is_empty() { - break; - } - if !row.trim().is_empty() { - break; - } - } - if row.is_empty() { - break; - } - row_number += 1; - } - Ok(row_number) -} +pub(crate) use deserialize::_deserialize; +pub use deserialize::deserialize; +pub(crate) use infer_schema::coerce_data_type; +pub use infer_schema::infer; diff --git a/src/io/json/write/format.rs b/src/io/json/write/format.rs deleted file mode 100644 index 66d85100d3e..00000000000 --- a/src/io/json/write/format.rs +++ /dev/null @@ -1,77 +0,0 @@ -use std::{fmt::Debug, io::Write}; - -use crate::error::Result; - -/// Trait defining how to format a sequence of JSON objects to a byte stream. -pub trait JsonFormat: Debug + Default + Copy { - #[inline] - /// write any bytes needed at the start of the file to the writer - fn start_stream(&self, _writer: &mut W) -> Result<()> { - Ok(()) - } - - #[inline] - /// write any bytes needed for the start of each row - fn start_row(&self, _writer: &mut W, _is_first_row: bool) -> Result<()> { - Ok(()) - } - - #[inline] - /// write any bytes needed for the end of each row - fn end_row(&self, _writer: &mut W) -> Result<()> { - Ok(()) - } - - /// write any bytes needed for the start of each row - fn end_stream(&self, _writer: &mut W) -> Result<()> { - Ok(()) - } -} - -/// Produces JSON output with one record per line. For example -/// -/// ```json -/// {"foo":1} -/// {"bar":1} -/// -/// ``` -#[derive(Debug, Default, Clone, Copy)] -pub struct LineDelimited {} - -impl JsonFormat for LineDelimited { - #[inline] - fn end_row(&self, writer: &mut W) -> Result<()> { - writer.write_all(b"\n")?; - Ok(()) - } -} - -/// Produces JSON output as a single JSON array. For example -/// -/// ```json -/// [{"foo":1},{"bar":1}] -/// ``` -#[derive(Debug, Default, Clone, Copy)] -pub struct JsonArray {} - -impl JsonFormat for JsonArray { - #[inline] - fn start_stream(&self, writer: &mut W) -> Result<()> { - writer.write_all(b"[")?; - Ok(()) - } - - #[inline] - fn start_row(&self, writer: &mut W, is_first_row: bool) -> Result<()> { - if !is_first_row { - writer.write_all(b",")?; - } - Ok(()) - } - - #[inline] - fn end_stream(&self, writer: &mut W) -> Result<()> { - writer.write_all(b"]")?; - Ok(()) - } -} diff --git a/src/io/json/write/mod.rs b/src/io/json/write/mod.rs index fcf1c435148..ca941a76ebe 100644 --- a/src/io/json/write/mod.rs +++ b/src/io/json/write/mod.rs @@ -1,84 +1,50 @@ //! APIs to write to JSON -mod format; mod serialize; -pub use fallible_streaming_iterator::*; -pub use format::*; -pub use serialize::serialize; -use crate::{ - array::Array, - chunk::Chunk, - error::{ArrowError, Result}, -}; +pub use fallible_streaming_iterator::*; +pub(crate) use serialize::new_serializer; +use serialize::serialize; -/// Writes blocks of JSON-encoded data into `writer`, ensuring that the written -/// JSON has the expected `format` -pub fn write(writer: &mut W, format: F, mut blocks: I) -> Result<()> -where - W: std::io::Write, - F: JsonFormat, - I: FallibleStreamingIterator, -{ - format.start_stream(writer)?; - let mut is_first_row = true; - while let Some(block) = blocks.next()? { - format.start_row(writer, is_first_row)?; - is_first_row = false; - writer.write_all(block)?; - } - format.end_stream(writer)?; - Ok(()) -} +use crate::{array::Array, error::ArrowError}; -/// [`FallibleStreamingIterator`] that serializes a [`Chunk`] to bytes. -/// Advancing it is CPU-bounded -pub struct Serializer +/// [`FallibleStreamingIterator`] that serializes an [`Array`] to bytes of valid JSON +/// # Implementation +/// Advancing this iterator CPU-bounded +#[derive(Debug, Clone)] +pub struct Serializer where - F: JsonFormat, A: AsRef, - I: Iterator>>, + I: Iterator>, { - batches: I, - names: Vec, + arrays: I, buffer: Vec, - format: F, } -impl Serializer +impl Serializer where - F: JsonFormat, A: AsRef, - I: Iterator>>, + I: Iterator>, { /// Creates a new [`Serializer`]. - pub fn new(batches: I, names: Vec, buffer: Vec, format: F) -> Self { - Self { - batches, - names, - buffer, - format, - } + pub fn new(arrays: I, buffer: Vec) -> Self { + Self { arrays, buffer } } } -impl FallibleStreamingIterator for Serializer +impl FallibleStreamingIterator for Serializer where - F: JsonFormat, A: AsRef, - I: Iterator>>, + I: Iterator>, { type Item = [u8]; type Error = ArrowError; - fn advance(&mut self) -> Result<()> { + fn advance(&mut self) -> Result<(), ArrowError> { self.buffer.clear(); - self.batches + self.arrays .next() - .map(|maybe_chunk| { - maybe_chunk - .map(|columns| serialize(&self.names, &columns, self.format, &mut self.buffer)) - }) + .map(|maybe_array| maybe_array.map(|array| serialize(array.as_ref(), &mut self.buffer))) .transpose()?; Ok(()) } @@ -91,3 +57,22 @@ where } } } + +/// Writes valid JSON from an iterator of (assumed JSON-encoded) bytes to `writer` +pub fn write(writer: &mut W, mut blocks: I) -> Result<(), ArrowError> +where + W: std::io::Write, + I: FallibleStreamingIterator, +{ + writer.write_all(&[b'['])?; + let mut is_first_row = true; + while let Some(block) = blocks.next()? { + if !is_first_row { + writer.write_all(&[b','])?; + } + is_first_row = false; + writer.write_all(block)?; + } + writer.write_all(&[b']'])?; + Ok(()) +} diff --git a/src/io/json/write/serialize.rs b/src/io/json/write/serialize.rs index 59bab0bf646..3b1f9e087b5 100644 --- a/src/io/json/write/serialize.rs +++ b/src/io/json/write/serialize.rs @@ -4,7 +4,6 @@ use std::io::Write; use streaming_iterator::StreamingIterator; use crate::bitmap::utils::zip_validity; -use crate::chunk::Chunk; use crate::datatypes::TimeUnit; use crate::io::iterator::BufStreamingIterator; use crate::temporal_conversions::{ @@ -14,8 +13,6 @@ use crate::temporal_conversions::{ use crate::util::lexical_to_bytes_mut; use crate::{array::*, datatypes::DataType, types::NativeType}; -use super::{JsonArray, JsonFormat}; - fn boolean_serializer<'a>( array: &'a BooleanArray, ) -> Box + 'a + Send + Sync> { @@ -94,7 +91,7 @@ fn struct_serializer<'a>( let item = iter.next().unwrap(); record.push((name, item)); }); - serialize_item(buf, &record, JsonArray::default(), true); + serialize_item(buf, &record, true); } else { serializers.iter_mut().for_each(|iter| { let _ = iter.next(); @@ -156,7 +153,7 @@ where move |x, buf| { if let Some(x) = x { let nd = convert(*x); - write!(buf, "{}", nd).unwrap(); + write!(buf, "\"{}\"", nd).unwrap(); } else { buf.extend_from_slice(b"null") } @@ -177,7 +174,7 @@ where move |x, buf| { if let Some(x) = x { let ndt = convert(*x); - write!(buf, "{}", ndt).unwrap(); + write!(buf, "\"{}\"", ndt).unwrap(); } else { buf.extend_from_slice(b"null") } @@ -186,7 +183,7 @@ where )) } -fn new_serializer<'a>( +pub(crate) fn new_serializer<'a>( array: &'a dyn Array, ) -> Box + 'a + Send + Sync> { match array.data_type().to_logical_type() { @@ -225,13 +222,10 @@ fn new_serializer<'a>( } } -fn serialize_item( - buffer: &mut Vec, - record: &[(&str, &[u8])], - format: F, - is_first_row: bool, -) { - format.start_row(buffer, is_first_row).unwrap(); +fn serialize_item(buffer: &mut Vec, record: &[(&str, &[u8])], is_first_row: bool) { + if !is_first_row { + buffer.push(b','); + } buffer.push(b'{'); let mut first_item = true; for (key, value) in record { @@ -244,37 +238,18 @@ fn serialize_item( buffer.extend(*value); } buffer.push(b'}'); - format.end_row(buffer).unwrap(); } -/// Serializes a (name, array) to a valid JSON to `buffer` -/// This is CPU-bounded -pub fn serialize(names: &[N], columns: &Chunk, format: F, buffer: &mut Vec) -where - N: AsRef, - A: AsRef, - F: JsonFormat, -{ - let num_rows = columns.len(); - - let mut serializers: Vec<_> = columns - .arrays() - .iter() - .map(|array| new_serializer(array.as_ref())) - .collect(); +/// Serializes `array` to a valid JSON to `buffer` +/// # Implementation +/// This operation is CPU-bounded +pub(crate) fn serialize(array: &dyn Array, buffer: &mut Vec) { + let mut serializer = new_serializer(array); - let mut is_first_row = true; - (0..num_rows).for_each(|_| { - let mut record: Vec<(&str, &[u8])> = Default::default(); - serializers - .iter_mut() - .zip(names.iter()) - // `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)); - }); - serialize_item(buffer, &record, format, is_first_row); - is_first_row = false; - }) + (0..array.len()).for_each(|i| { + if i != 0 { + buffer.push(b','); + } + buffer.extend_from_slice(serializer.next().unwrap()); + }); } diff --git a/src/io/json_integration/read/array.rs b/src/io/json_integration/read/array.rs index d8210c89dfc..b73223b7488 100644 --- a/src/io/json_integration/read/array.rs +++ b/src/io/json_integration/read/array.rs @@ -170,8 +170,7 @@ fn to_binary(json_col: &ArrowJsonColumn, data_type: DataType) -> Arc< .as_ref() .unwrap() .iter() - .map(|value| value.as_str().map(|x| hex::decode(x).unwrap()).unwrap()) - .flatten() + .flat_map(|value| value.as_str().map(|x| hex::decode(x).unwrap()).unwrap()) .collect(); Arc::new(BinaryArray::from_data(data_type, offsets, values, validity)) } @@ -184,8 +183,7 @@ fn to_utf8(json_col: &ArrowJsonColumn, data_type: DataType) -> Arc Result, ArrowError> { + // deserialize strings to `Value`s + let rows = rows + .iter() + .map(|row| serde_json::from_str(row.as_ref()).map_err(ArrowError::from)) + .collect::, ArrowError>>()?; + + // deserialize &[Value] to Array + Ok(_deserialize(&rows, data_type)) +} diff --git a/src/io/ndjson/read/file.rs b/src/io/ndjson/read/file.rs new file mode 100644 index 00000000000..9ec9a73a8c6 --- /dev/null +++ b/src/io/ndjson/read/file.rs @@ -0,0 +1,121 @@ +use std::io::BufRead; + +use fallible_streaming_iterator::FallibleStreamingIterator; +use indexmap::set::IndexSet as HashSet; +use serde_json; +use serde_json::Value; + +use crate::{ + datatypes::DataType, + error::{ArrowError, Result}, +}; + +use super::super::super::json::read::{coerce_data_type, infer as infer_json}; + +/// Reads up to a number of lines from `reader` into `rows` bounded by `limit`. +fn read_rows(reader: &mut R, rows: &mut [String], limit: usize) -> Result { + if limit == 0 { + return Ok(0); + } + let mut row_number = 0; + for row in rows.iter_mut() { + loop { + row.clear(); + let _ = reader.read_line(row).map_err(|e| { + ArrowError::External(format!(" at line {}", row_number), Box::new(e)) + })?; + if row.is_empty() { + break; + } + if !row.trim().is_empty() { + break; + } + } + if row.is_empty() { + break; + } + row_number += 1; + if row_number == limit { + break; + } + } + Ok(row_number) +} + +/// A [`FallibleStreamingIterator`] of NDJSON rows. +/// +/// This iterator is used to read chunks of an NDJSON in batches. +/// This iterator is guaranteed to yield at least one row. +/// # Implementantion +/// Advancing this iterator is IO-bounded, but does require parsing each byte to find end of lines. +/// # Error +/// Advancing this iterator errors iff the reader errors. +pub struct FileReader { + reader: R, + rows: Vec, + number_of_rows: usize, + remaining: usize, +} + +impl FileReader { + /// Creates a new [`FileReader`] from a reader and `rows`. + /// + /// The number of items in `rows` denotes the batch size. + pub fn new(reader: R, rows: Vec, limit: Option) -> Self { + Self { + reader, + rows, + remaining: limit.unwrap_or(usize::MAX), + number_of_rows: 0, + } + } + + /// Deconstruct [`FileReader`] into the reader and the internal buffer. + pub fn into_inner(self) -> (R, Vec) { + (self.reader, self.rows) + } +} + +impl FallibleStreamingIterator for FileReader { + type Error = ArrowError; + type Item = [String]; + + fn advance(&mut self) -> Result<()> { + self.number_of_rows = read_rows(&mut self.reader, &mut self.rows, self.remaining)?; + self.remaining -= self.number_of_rows; + Ok(()) + } + + fn get(&self) -> Option<&Self::Item> { + if self.number_of_rows > 0 { + Some(&self.rows[..self.number_of_rows]) + } else { + None + } + } +} + +/// Infers the [`DataType`] from an NDJSON file, optionally only using `number_of_rows` rows. +/// +/// # Implementantion +/// This implementation reads the file line by line and infers the type of each line. +/// It performs both `O(N)` IO and CPU-bounded operations where `N` is the number of rows. +pub fn infer( + reader: &mut R, + number_of_rows: Option, +) -> Result { + let rows = vec!["".to_string(); 1]; // 1 <=> read row by row + let mut reader = FileReader::new(reader, rows, number_of_rows); + + let mut data_types = HashSet::new(); + while let Some(rows) = reader.next()? { + let value: Value = serde_json::from_str(&rows[0])?; // 0 because it is row by row + let data_type = infer_json(&value)?; + if data_type != DataType::Null { + data_types.insert(data_type); + } + } + + let v: Vec<&DataType> = data_types.iter().collect(); + Ok(coerce_data_type(&v)) +} diff --git a/src/io/ndjson/read/mod.rs b/src/io/ndjson/read/mod.rs new file mode 100644 index 00000000000..5c52bd183fc --- /dev/null +++ b/src/io/ndjson/read/mod.rs @@ -0,0 +1,8 @@ +//! APIs to read and deserialize [NDJSON](http://ndjson.org/). + +pub use fallible_streaming_iterator::FallibleStreamingIterator; + +mod deserialize; +mod file; +pub use deserialize::deserialize; +pub use file::{infer, FileReader}; diff --git a/src/io/ndjson/write/mod.rs b/src/io/ndjson/write/mod.rs new file mode 100644 index 00000000000..456e492de2a --- /dev/null +++ b/src/io/ndjson/write/mod.rs @@ -0,0 +1,119 @@ +//! APIs to serialize and write to [NDJSON](http://ndjson.org/). +use std::io::Write; + +pub use fallible_streaming_iterator::FallibleStreamingIterator; + +use crate::array::Array; +use crate::error::ArrowError; + +use super::super::json::write::new_serializer; + +fn serialize(array: &dyn Array, buffer: &mut Vec) { + let mut serializer = new_serializer(array); + (0..array.len()).for_each(|_| { + buffer.extend_from_slice(serializer.next().unwrap()); + buffer.push(b'\n'); + }); +} + +/// [`FallibleStreamingIterator`] that serializes an [`Array`] to bytes of valid NDJSON +/// where every line is an element of the array. +/// # Implementation +/// Advancing this iterator CPU-bounded +#[derive(Debug, Clone)] +pub struct Serializer +where + A: AsRef, + I: Iterator>, +{ + arrays: I, + buffer: Vec, +} + +impl Serializer +where + A: AsRef, + I: Iterator>, +{ + /// Creates a new [`Serializer`]. + pub fn new(arrays: I, buffer: Vec) -> Self { + Self { arrays, buffer } + } +} + +impl FallibleStreamingIterator for Serializer +where + A: AsRef, + I: Iterator>, +{ + type Item = [u8]; + + type Error = ArrowError; + + fn advance(&mut self) -> Result<(), ArrowError> { + self.buffer.clear(); + self.arrays + .next() + .map(|maybe_array| maybe_array.map(|array| serialize(array.as_ref(), &mut self.buffer))) + .transpose()?; + Ok(()) + } + + fn get(&self) -> Option<&Self::Item> { + if !self.buffer.is_empty() { + Some(&self.buffer) + } else { + None + } + } +} + +/// An iterator adapter that receives an implementer of [`Write`] and +/// an implementer of [`FallibleStreamingIterator`] (such as [`Serializer`]) +/// and writes a valid NDJSON +/// # Implementation +/// Advancing this iterator mixes CPU-bounded (serializing arrays) tasks and IO-bounded (write to the writer). +pub struct FileWriter +where + W: Write, + I: FallibleStreamingIterator, +{ + writer: W, + iterator: I, +} + +impl FileWriter +where + W: Write, + I: FallibleStreamingIterator, +{ + /// Creates a new [`FileWriter`]. + pub fn new(writer: W, iterator: I) -> Self { + Self { writer, iterator } + } + + /// Returns the inner content of this iterator + /// + /// There are two use-cases for this function: + /// * to continue writing to its writer + /// * to re-use an internal buffer of its iterator + pub fn into_inner(self) -> (W, I) { + (self.writer, self.iterator) + } +} + +impl Iterator for FileWriter +where + W: Write, + I: FallibleStreamingIterator, +{ + type Item = Result<(), ArrowError>; + + fn next(&mut self) -> Option { + let item = self.iterator.next().transpose()?; + Some(item.and_then(|x| { + self.writer.write_all(x)?; + Ok(()) + })) + } +} diff --git a/src/io/parquet/read/binary/basic.rs b/src/io/parquet/read/deserialize/binary/basic.rs similarity index 100% rename from src/io/parquet/read/binary/basic.rs rename to src/io/parquet/read/deserialize/binary/basic.rs diff --git a/src/io/parquet/read/binary/dictionary.rs b/src/io/parquet/read/deserialize/binary/dictionary.rs similarity index 98% rename from src/io/parquet/read/binary/dictionary.rs rename to src/io/parquet/read/deserialize/binary/dictionary.rs index 1a95b8a1dcc..4f8adc70268 100644 --- a/src/io/parquet/read/binary/dictionary.rs +++ b/src/io/parquet/read/deserialize/binary/dictionary.rs @@ -7,10 +7,10 @@ use crate::{ bitmap::MutableBitmap, datatypes::{DataType, PhysicalType}, error::Result, - io::parquet::read::utils::MaybeNext, }; use super::super::dictionary::*; +use super::super::utils::MaybeNext; use super::super::DataPages; /// An iterator adapter over [`DataPages`] assumed to be encoded as parquet's dictionary-encoded binary representation diff --git a/src/io/parquet/read/binary/mod.rs b/src/io/parquet/read/deserialize/binary/mod.rs similarity index 100% rename from src/io/parquet/read/binary/mod.rs rename to src/io/parquet/read/deserialize/binary/mod.rs diff --git a/src/io/parquet/read/binary/nested.rs b/src/io/parquet/read/deserialize/binary/nested.rs similarity index 96% rename from src/io/parquet/read/binary/nested.rs rename to src/io/parquet/read/deserialize/binary/nested.rs index 2276829e70c..17de2d93a97 100644 --- a/src/io/parquet/read/binary/nested.rs +++ b/src/io/parquet/read/deserialize/binary/nested.rs @@ -3,14 +3,12 @@ use std::collections::VecDeque; use parquet2::{encoding::Encoding, page::DataPage, schema::Repetition}; use crate::{ - array::Offset, - bitmap::MutableBitmap, - datatypes::DataType, - error::Result, - io::parquet::read::{utils::MaybeNext, DataPages}, + array::Offset, bitmap::MutableBitmap, datatypes::DataType, error::Result, + io::parquet::read::DataPages, }; use super::super::nested_utils::*; +use super::super::utils::MaybeNext; use super::utils::Binary; use super::{ super::utils, diff --git a/src/io/parquet/read/binary/utils.rs b/src/io/parquet/read/deserialize/binary/utils.rs similarity index 96% rename from src/io/parquet/read/binary/utils.rs rename to src/io/parquet/read/deserialize/binary/utils.rs index 6bf16655fac..d967417eb63 100644 --- a/src/io/parquet/read/binary/utils.rs +++ b/src/io/parquet/read/deserialize/binary/utils.rs @@ -1,4 +1,6 @@ -use crate::{array::Offset, io::parquet::read::utils::Pushable}; +use crate::array::Offset; + +use super::super::utils::Pushable; /// [`Pushable`] for variable length binary data. #[derive(Debug)] diff --git a/src/io/parquet/read/boolean/basic.rs b/src/io/parquet/read/deserialize/boolean/basic.rs similarity index 100% rename from src/io/parquet/read/boolean/basic.rs rename to src/io/parquet/read/deserialize/boolean/basic.rs diff --git a/src/io/parquet/read/boolean/mod.rs b/src/io/parquet/read/deserialize/boolean/mod.rs similarity index 100% rename from src/io/parquet/read/boolean/mod.rs rename to src/io/parquet/read/deserialize/boolean/mod.rs diff --git a/src/io/parquet/read/boolean/nested.rs b/src/io/parquet/read/deserialize/boolean/nested.rs similarity index 100% rename from src/io/parquet/read/boolean/nested.rs rename to src/io/parquet/read/deserialize/boolean/nested.rs diff --git a/src/io/parquet/read/dictionary.rs b/src/io/parquet/read/deserialize/dictionary.rs similarity index 100% rename from src/io/parquet/read/dictionary.rs rename to src/io/parquet/read/deserialize/dictionary.rs diff --git a/src/io/parquet/read/fixed_size_binary/basic.rs b/src/io/parquet/read/deserialize/fixed_size_binary/basic.rs similarity index 95% rename from src/io/parquet/read/fixed_size_binary/basic.rs rename to src/io/parquet/read/deserialize/fixed_size_binary/basic.rs index e3e8e593b7f..0aaa08bdf99 100644 --- a/src/io/parquet/read/fixed_size_binary/basic.rs +++ b/src/io/parquet/read/deserialize/fixed_size_binary/basic.rs @@ -7,19 +7,14 @@ use parquet2::{ }; use crate::{ - array::FixedSizeBinaryArray, - bitmap::MutableBitmap, - datatypes::DataType, - error::Result, - io::parquet::read::{ - utils::{ - dict_indices_decoder, extend_from_decoder, next, not_implemented, split_buffer, - Decoder, MaybeNext, OptionalPageValidity, PageState, - }, - DataPages, - }, + array::FixedSizeBinaryArray, bitmap::MutableBitmap, datatypes::DataType, error::Result, }; +use super::super::utils::{ + dict_indices_decoder, extend_from_decoder, next, not_implemented, split_buffer, Decoder, + MaybeNext, OptionalPageValidity, PageState, +}; +use super::super::DataPages; use super::utils::FixedSizeBinary; struct Optional<'a> { diff --git a/src/io/parquet/read/fixed_size_binary/dictionary.rs b/src/io/parquet/read/deserialize/fixed_size_binary/dictionary.rs similarity index 100% rename from src/io/parquet/read/fixed_size_binary/dictionary.rs rename to src/io/parquet/read/deserialize/fixed_size_binary/dictionary.rs diff --git a/src/io/parquet/read/fixed_size_binary/mod.rs b/src/io/parquet/read/deserialize/fixed_size_binary/mod.rs similarity index 100% rename from src/io/parquet/read/fixed_size_binary/mod.rs rename to src/io/parquet/read/deserialize/fixed_size_binary/mod.rs diff --git a/src/io/parquet/read/fixed_size_binary/utils.rs b/src/io/parquet/read/deserialize/fixed_size_binary/utils.rs similarity index 96% rename from src/io/parquet/read/fixed_size_binary/utils.rs rename to src/io/parquet/read/deserialize/fixed_size_binary/utils.rs index dcf20b0110f..a4b7d047606 100644 --- a/src/io/parquet/read/fixed_size_binary/utils.rs +++ b/src/io/parquet/read/deserialize/fixed_size_binary/utils.rs @@ -1,4 +1,4 @@ -use crate::io::parquet::read::utils::Pushable; +use super::super::utils::Pushable; /// A [`Pushable`] for fixed sized binary data #[derive(Debug)] diff --git a/src/io/parquet/read/deserialize/mod.rs b/src/io/parquet/read/deserialize/mod.rs index fe870d904d2..a6a2b48dc15 100644 --- a/src/io/parquet/read/deserialize/mod.rs +++ b/src/io/parquet/read/deserialize/mod.rs @@ -1,491 +1,271 @@ //! APIs to read from Parquet format. -use std::sync::Arc; - -use parquet2::{ - schema::types::{ - LogicalType, ParquetType, PhysicalType, TimeUnit as ParquetTimeUnit, TimestampType, - }, - types::int96_to_i64_ns, -}; +mod binary; +mod boolean; +mod dictionary; +mod fixed_size_binary; +mod nested_utils; +mod null; +mod primitive; +mod simple; +mod utils; use crate::{ - array::{Array, BinaryArray, DictionaryKey, PrimitiveArray, Utf8Array}, - datatypes::{DataType, IntervalUnit, TimeUnit}, + array::{Array, BinaryArray, ListArray, StructArray, Utf8Array}, + datatypes::{DataType, Field}, error::{ArrowError, Result}, }; -use super::binary; -use super::boolean; -use super::fixed_size_binary; -use super::null; -use super::primitive; -use super::{dyn_iter, iden, op, ArrayIter, DataPages}; +use self::nested_utils::{InitNested, NestedArrayIter, NestedState}; +use simple::page_iter_to_arrays; + +use super::*; + +/// Creates a new iterator of compressed pages. +pub fn get_page_iterator( + column_metadata: &ColumnChunkMetaData, + reader: R, + pages_filter: Option, + buffer: Vec, +) -> Result> { + Ok(_get_page_iterator( + column_metadata, + reader, + pages_filter, + buffer, + )?) +} -pub fn page_iter_to_arrays<'a, I: 'a + DataPages>( - pages: I, - type_: &ParquetType, +fn create_list( data_type: DataType, - chunk_size: usize, -) -> Result> { - use DataType::*; - - let (physical_type, logical_type) = if let ParquetType::PrimitiveType { - physical_type, - logical_type, - .. - } = type_ - { - (physical_type, logical_type) - } else { - return Err(ArrowError::InvalidArgumentError( - "page_iter_to_arrays can only be called with a parquet primitive type".into(), - )); - }; - - Ok(match data_type.to_logical_type() { - Null => null::iter_to_arrays(pages, data_type, chunk_size), - Boolean => dyn_iter(boolean::Iter::new(pages, data_type, chunk_size)), - UInt8 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i32| x as u8, - ))), - UInt16 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i32| x as u16, - ))), - UInt32 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i32| x as u32, - ))), - Int8 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i32| x as i8, - ))), - Int16 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i32| x as i16, - ))), - Int32 | Date32 | Time32(_) | Interval(IntervalUnit::YearMonth) => dyn_iter(iden( - primitive::Iter::new(pages, data_type, chunk_size, |x: i32| x as i32), - )), - - Timestamp(time_unit, None) => { - let time_unit = *time_unit; - return timestamp( - pages, - physical_type, - logical_type, + nested: &mut NestedState, + values: Arc, +) -> Result> { + Ok(match data_type { + DataType::List(_) => { + let (offsets, validity) = nested.nested.pop().unwrap().inner(); + + let offsets = offsets.iter().map(|x| *x as i32).collect::>(); + Arc::new(ListArray::::from_data( data_type, - chunk_size, - time_unit, - ); + offsets.into(), + values, + validity, + )) } + DataType::LargeList(_) => { + let (offsets, validity) = nested.nested.pop().unwrap().inner(); - FixedSizeBinary(_) => dyn_iter(fixed_size_binary::Iter::new(pages, data_type, chunk_size)), - - Decimal(_, _) => match physical_type { - PhysicalType::Int32 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i32| x as i128, - ))), - PhysicalType::Int64 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i64| x as i128, - ))), - &PhysicalType::FixedLenByteArray(n) if n > 16 => { - return Err(ArrowError::NotYetImplemented(format!( - "Can't decode Decimal128 type from Fixed Size Byte Array of len {:?}", - n - ))) - } - &PhysicalType::FixedLenByteArray(n) => { - let n = n as usize; - - let pages = - fixed_size_binary::Iter::new(pages, DataType::FixedSizeBinary(n), chunk_size); - - let pages = pages.map(move |maybe_array| { - let array = maybe_array?; - let values = array - .values() - .chunks_exact(n) - .map(|value: &[u8]| { - // Copy the fixed-size byte value to the start of a 16 byte stack - // allocated buffer, then use an arithmetic right shift to fill in - // MSBs, which accounts for leading 1's in negative (two's complement) - // values. - let mut bytes = [0u8; 16]; - bytes[..n].copy_from_slice(value); - i128::from_be_bytes(bytes) >> (8 * (16 - n)) - }) - .collect::>(); - let validity = array.validity().cloned(); - - Ok(PrimitiveArray::::from_data( - data_type.clone(), - values.into(), - validity, - )) - }); - - let arrays = pages.map(|x| x.map(|x| Arc::new(x) as Arc)); - - Box::new(arrays) as _ - } - _ => unreachable!(), - }, - - // INT64 - Int64 | Date64 | Time64(_) | Duration(_) | Timestamp(_, _) => dyn_iter(iden( - primitive::Iter::new(pages, data_type, chunk_size, |x: i64| x as i64), - )), - UInt64 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: i64| x as u64, - ))), - - Float32 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: f32| x, - ))), - Float64 => dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - |x: f64| x, - ))), - - Binary => dyn_iter(binary::Iter::, _>::new( - pages, data_type, chunk_size, - )), - LargeBinary => dyn_iter(binary::Iter::, _>::new( - pages, data_type, chunk_size, - )), - Utf8 => dyn_iter(binary::Iter::, _>::new( - pages, data_type, chunk_size, - )), - LargeUtf8 => dyn_iter(binary::Iter::, _>::new( - pages, data_type, chunk_size, - )), - - Dictionary(key_type, _, _) => { - return match_integer_type!(key_type, |$K| { - dict_read::<$K, _>(pages, physical_type, logical_type, data_type, chunk_size) - }) + Arc::new(ListArray::::from_data( + data_type, offsets, values, validity, + )) } - - other => { + _ => { return Err(ArrowError::NotYetImplemented(format!( - "Reading {:?} from parquet still not implemented", - other + "Read nested datatype {:?}", + data_type ))) } }) } -fn timestamp<'a, I: 'a + DataPages>( - pages: I, - physical_type: &PhysicalType, - logical_type: &Option, - data_type: DataType, - chunk_size: usize, - time_unit: TimeUnit, -) -> Result> { - if physical_type == &PhysicalType::Int96 { - if time_unit == TimeUnit::Nanosecond { - return Ok(dyn_iter(iden(primitive::Iter::new( - pages, - data_type, - chunk_size, - int96_to_i64_ns, - )))); - } else { - return Err(ArrowError::nyi( - "Can't decode int96 to timestamp other than ns", - )); - } - }; - if physical_type != &PhysicalType::Int64 { - return Err(ArrowError::nyi( - "Can't decode a timestamp from a non-int64 parquet type", - )); - } - - let iter = primitive::Iter::new(pages, data_type, chunk_size, |x: i64| x); - - let unit = if let Some(LogicalType::TIMESTAMP(TimestampType { unit, .. })) = logical_type { - unit - } else { - return Ok(dyn_iter(iden(iter))); - }; +struct StructIterator<'a> { + iters: Vec>, + fields: Vec, +} - Ok(match (unit, time_unit) { - (ParquetTimeUnit::MILLIS(_), TimeUnit::Second) => dyn_iter(op(iter, |x| x / 1_000)), - (ParquetTimeUnit::MICROS(_), TimeUnit::Second) => dyn_iter(op(iter, |x| x / 1_000_000)), - (ParquetTimeUnit::NANOS(_), TimeUnit::Second) => dyn_iter(op(iter, |x| x * 1_000_000_000)), +impl<'a> StructIterator<'a> { + pub fn new(iters: Vec>, fields: Vec) -> Self { + assert_eq!(iters.len(), fields.len()); + Self { iters, fields } + } +} - (ParquetTimeUnit::MILLIS(_), TimeUnit::Millisecond) => dyn_iter(iden(iter)), - (ParquetTimeUnit::MICROS(_), TimeUnit::Millisecond) => dyn_iter(op(iter, |x| x / 1_000)), - (ParquetTimeUnit::NANOS(_), TimeUnit::Millisecond) => dyn_iter(op(iter, |x| x / 1_000_000)), +impl<'a> Iterator for StructIterator<'a> { + type Item = Result<(NestedState, Arc)>; - (ParquetTimeUnit::MILLIS(_), TimeUnit::Microsecond) => dyn_iter(op(iter, |x| x * 1_000)), - (ParquetTimeUnit::MICROS(_), TimeUnit::Microsecond) => dyn_iter(iden(iter)), - (ParquetTimeUnit::NANOS(_), TimeUnit::Microsecond) => dyn_iter(op(iter, |x| x / 1_000)), + fn next(&mut self) -> Option { + let values = self + .iters + .iter_mut() + .map(|iter| iter.next()) + .collect::>(); - (ParquetTimeUnit::MILLIS(_), TimeUnit::Nanosecond) => dyn_iter(op(iter, |x| x * 1_000_000)), - (ParquetTimeUnit::MICROS(_), TimeUnit::Nanosecond) => dyn_iter(op(iter, |x| x * 1_000)), - (ParquetTimeUnit::NANOS(_), TimeUnit::Nanosecond) => dyn_iter(iden(iter)), - }) + if values.iter().any(|x| x.is_none()) { + return None; + } + let values = values + .into_iter() + .map(|x| x.unwrap().map(|x| x.1)) + .collect::>>(); + + match values { + Ok(values) => Some(Ok(( + NestedState::new(vec![]), // todo + Arc::new(StructArray::from_data( + DataType::Struct(self.fields.clone()), + values, + None, + )), + ))), + Err(e) => Some(Err(e)), + } + } } -fn timestamp_dict<'a, K: DictionaryKey, I: 'a + DataPages>( - pages: I, - physical_type: &PhysicalType, - logical_type: &Option, - data_type: DataType, +fn columns_to_iter_recursive<'a, I: 'a>( + mut columns: Vec, + mut types: Vec<&ParquetType>, + field: Field, + mut init: Vec, chunk_size: usize, - time_unit: TimeUnit, -) -> Result> { - if physical_type == &PhysicalType::Int96 { - if time_unit == TimeUnit::Nanosecond { - return Ok(dyn_iter(primitive::DictIter::::new( - pages, - DataType::Timestamp(TimeUnit::Nanosecond, None), +) -> Result> +where + I: DataPages, +{ + use DataType::*; + if init.len() == 1 && init[0].is_primitive() { + return Ok(Box::new( + page_iter_to_arrays( + columns.pop().unwrap(), + types.pop().unwrap(), + field.data_type, chunk_size, - int96_to_i64_ns, - ))); - } else { - return Err(ArrowError::nyi( - "Can't decode int96 to timestamp other than ns", - )); - } - }; - - let unit = if let Some(LogicalType::TIMESTAMP(TimestampType { unit, .. })) = logical_type { - unit - } else { - return Ok(dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x, - ))); - }; + )? + .map(|x| Ok((NestedState::new(vec![]), x?))), + )); + } - Ok(match (unit, time_unit) { - (ParquetTimeUnit::MILLIS(_), TimeUnit::Second) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x / 1_000, - )) + Ok(match field.data_type().to_logical_type() { + Boolean => { + types.pop(); + boolean::iter_to_arrays_nested(columns.pop().unwrap(), init.pop().unwrap(), chunk_size) } - (ParquetTimeUnit::MICROS(_), TimeUnit::Second) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x / 1_000_000, - )) - } - (ParquetTimeUnit::NANOS(_), TimeUnit::Second) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, + Int16 => { + types.pop(); + primitive::iter_to_arrays_nested( + columns.pop().unwrap(), + init.pop().unwrap(), + field.data_type().clone(), chunk_size, - |x: i64| x * 1_000_000_000, - )) + |x: i32| x as i16, + ) } - - (ParquetTimeUnit::MILLIS(_), TimeUnit::Millisecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, + Int64 => { + types.pop(); + primitive::iter_to_arrays_nested( + columns.pop().unwrap(), + init.pop().unwrap(), + field.data_type().clone(), chunk_size, |x: i64| x, - )) + ) } - (ParquetTimeUnit::MICROS(_), TimeUnit::Millisecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, + Utf8 => { + types.pop(); + binary::iter_to_arrays_nested::, _>( + columns.pop().unwrap(), + init.pop().unwrap(), + field.data_type().clone(), chunk_size, - |x: i64| x / 1_000, - )) + ) } - (ParquetTimeUnit::NANOS(_), TimeUnit::Millisecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, + LargeBinary => { + types.pop(); + binary::iter_to_arrays_nested::, _>( + columns.pop().unwrap(), + init.pop().unwrap(), + field.data_type().clone(), chunk_size, - |x: i64| x / 1_000_000, - )) + ) } - - (ParquetTimeUnit::MILLIS(_), TimeUnit::Microsecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, + List(inner) => { + let iter = columns_to_iter_recursive( + vec![columns.pop().unwrap()], + types, + inner.as_ref().clone(), + init, chunk_size, - |x: i64| x * 1_000, - )) + )?; + let iter = iter.map(move |x| { + let (mut nested, array) = x?; + let array = create_list(field.data_type().clone(), &mut nested, array)?; + Ok((nested, array)) + }); + Box::new(iter) as _ } - (ParquetTimeUnit::MICROS(_), TimeUnit::Microsecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x, - )) - } - (ParquetTimeUnit::NANOS(_), TimeUnit::Microsecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x / 1_000, - )) + Struct(fields) => { + let columns = fields + .iter() + .rev() + .map(|f| { + columns_to_iter_recursive( + vec![columns.pop().unwrap()], + vec![types.pop().unwrap()], + f.clone(), + vec![init.pop().unwrap()], + chunk_size, + ) + }) + .collect::>>()?; + let columns = columns.into_iter().rev().collect(); + Box::new(StructIterator::new(columns, fields.clone())) } + _ => todo!(), + }) +} - (ParquetTimeUnit::MILLIS(_), TimeUnit::Nanosecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x * 1_000_000, - )) +fn field_to_init(field: &Field) -> Vec { + use crate::datatypes::PhysicalType::*; + match field.data_type.to_physical_type() { + Null | Boolean | Primitive(_) | Binary | FixedSizeBinary | LargeBinary | Utf8 + | Dictionary(_) | LargeUtf8 => vec![InitNested::Primitive(field.is_nullable)], + List | FixedSizeList | LargeList => { + let a = field.data_type().to_logical_type(); + let inner = if let DataType::List(inner) = a { + field_to_init(inner) + } else if let DataType::LargeList(inner) = a { + field_to_init(inner) + } else if let DataType::FixedSizeList(inner, _) = a { + field_to_init(inner) + } else { + unreachable!() + }; + inner + .into_iter() + .map(|x| InitNested::List(Box::new(x), field.is_nullable)) + .collect() } - (ParquetTimeUnit::MICROS(_), TimeUnit::Nanosecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x * 1_000, - )) + Struct => { + let inner = if let DataType::Struct(fields) = field.data_type.to_logical_type() { + fields.iter().rev().map(field_to_init).collect::>() + } else { + unreachable!() + }; + inner + .into_iter() + .flatten() + .map(|x| InitNested::Struct(Box::new(x), field.is_nullable)) + .collect() } - (ParquetTimeUnit::NANOS(_), TimeUnit::Nanosecond) => { - dyn_iter(primitive::DictIter::::new( - pages, - data_type, - chunk_size, - |x: i64| x, - )) - } - }) + _ => todo!(), + } } -fn dict_read<'a, K: DictionaryKey, I: 'a + DataPages>( - iter: I, - physical_type: &PhysicalType, - logical_type: &Option, - data_type: DataType, +/// An iterator adapter that maps multiple iterators of [`DataPages`] into an iterator of [`Array`]s. +/// +/// The arrays are guaranteed to be at most of size `chunk_size` and data type `field.data_type`. +pub fn column_iter_to_arrays<'a, I: 'a>( + columns: Vec, + types: Vec<&ParquetType>, + field: Field, chunk_size: usize, -) -> Result> { - use DataType::*; - let values_data_type = if let Dictionary(_, v, _) = &data_type { - v.as_ref() - } else { - panic!() - }; - - Ok(match values_data_type.to_logical_type() { - UInt8 => dyn_iter(primitive::DictIter::::new( - iter, - data_type, - chunk_size, - |x: i32| x as u8, - )), - UInt16 => dyn_iter(primitive::DictIter::::new( - iter, - data_type, - chunk_size, - |x: i32| x as u16, - )), - UInt32 => dyn_iter(primitive::DictIter::::new( - iter, - data_type, - chunk_size, - |x: i32| x as u32, - )), - Int8 => dyn_iter(primitive::DictIter::::new( - iter, - data_type, - chunk_size, - |x: i32| x as i8, - )), - Int16 => dyn_iter(primitive::DictIter::::new( - iter, - data_type, - chunk_size, - |x: i32| x as i16, - )), - Int32 | Date32 | Time32(_) | Interval(IntervalUnit::YearMonth) => dyn_iter( - primitive::DictIter::::new(iter, data_type, chunk_size, |x: i32| { - x as i32 - }), - ), - - Timestamp(time_unit, None) => { - let time_unit = *time_unit; - return timestamp_dict::( - iter, - physical_type, - logical_type, - data_type, - chunk_size, - time_unit, - ); - } - - Int64 | Date64 | Time64(_) | Duration(_) | Timestamp(_, _) => dyn_iter( - primitive::DictIter::::new(iter, data_type, chunk_size, |x: i64| x), - ), - Float32 => dyn_iter(primitive::DictIter::::new( - iter, - data_type, - chunk_size, - |x: f32| x, - )), - Float64 => dyn_iter(primitive::DictIter::::new( - iter, - data_type, - chunk_size, - |x: f64| x, - )), - - Utf8 | Binary => dyn_iter(binary::DictIter::::new( - iter, data_type, chunk_size, - )), - LargeUtf8 | LargeBinary => dyn_iter(binary::DictIter::::new( - iter, data_type, chunk_size, - )), - FixedSizeBinary(_) => dyn_iter(fixed_size_binary::DictIter::::new( - iter, data_type, chunk_size, - )), - other => { - return Err(ArrowError::nyi(format!( - "Reading dictionaries of type {:?}", - other - ))) - } - }) +) -> Result> +where + I: DataPages, +{ + let init = field_to_init(&field); + + Ok(Box::new( + columns_to_iter_recursive(columns, types, field, init, chunk_size)?.map(|x| x.map(|x| x.1)), + )) } diff --git a/src/io/parquet/read/nested_utils.rs b/src/io/parquet/read/deserialize/nested_utils.rs similarity index 99% rename from src/io/parquet/read/nested_utils.rs rename to src/io/parquet/read/deserialize/nested_utils.rs index 8f9e2d32bf2..178ababd464 100644 --- a/src/io/parquet/read/nested_utils.rs +++ b/src/io/parquet/read/deserialize/nested_utils.rs @@ -11,10 +11,8 @@ use crate::{ error::Result, }; -use super::{ - utils::{split_buffer, Decoder, MaybeNext, Pushable}, - DataPages, -}; +use super::super::DataPages; +use super::utils::{split_buffer, Decoder, MaybeNext, Pushable}; /// trait describing deserialized repetition and definition levels pub trait Nested: std::fmt::Debug + Send + Sync { diff --git a/src/io/parquet/read/null.rs b/src/io/parquet/read/deserialize/null.rs similarity index 95% rename from src/io/parquet/read/null.rs rename to src/io/parquet/read/deserialize/null.rs index 5bfb11d135b..08382ca6be1 100644 --- a/src/io/parquet/read/null.rs +++ b/src/io/parquet/read/deserialize/null.rs @@ -5,8 +5,7 @@ use crate::{ datatypes::DataType, }; -use super::ArrayIter; -use super::DataPages; +use super::super::{ArrayIter, DataPages}; /// Converts [`DataPages`] to an [`Iterator`] of [`Array`] pub fn iter_to_arrays<'a, I>(mut iter: I, data_type: DataType, chunk_size: usize) -> ArrayIter<'a> diff --git a/src/io/parquet/read/primitive/basic.rs b/src/io/parquet/read/deserialize/primitive/basic.rs similarity index 100% rename from src/io/parquet/read/primitive/basic.rs rename to src/io/parquet/read/deserialize/primitive/basic.rs diff --git a/src/io/parquet/read/primitive/dictionary.rs b/src/io/parquet/read/deserialize/primitive/dictionary.rs similarity index 98% rename from src/io/parquet/read/primitive/dictionary.rs rename to src/io/parquet/read/deserialize/primitive/dictionary.rs index 7022d2b121c..bf6e4d4b9b3 100644 --- a/src/io/parquet/read/primitive/dictionary.rs +++ b/src/io/parquet/read/deserialize/primitive/dictionary.rs @@ -10,11 +10,11 @@ use crate::{ bitmap::MutableBitmap, datatypes::DataType, error::Result, - io::parquet::read::utils::MaybeNext, types::NativeType, }; use super::super::dictionary::*; +use super::super::utils::MaybeNext; use super::super::DataPages; #[inline] diff --git a/src/io/parquet/read/primitive/mod.rs b/src/io/parquet/read/deserialize/primitive/mod.rs similarity index 100% rename from src/io/parquet/read/primitive/mod.rs rename to src/io/parquet/read/deserialize/primitive/mod.rs diff --git a/src/io/parquet/read/primitive/nested.rs b/src/io/parquet/read/deserialize/primitive/nested.rs similarity index 95% rename from src/io/parquet/read/primitive/nested.rs rename to src/io/parquet/read/deserialize/primitive/nested.rs index 62b00119315..cbfc2ae778c 100644 --- a/src/io/parquet/read/primitive/nested.rs +++ b/src/io/parquet/read/deserialize/primitive/nested.rs @@ -7,7 +7,7 @@ use parquet2::{ use crate::{ array::PrimitiveArray, bitmap::MutableBitmap, datatypes::DataType, error::Result, - io::parquet::read::utils::MaybeNext, types::NativeType, + types::NativeType, }; use super::super::nested_utils::*; @@ -207,12 +207,12 @@ where &self.decoder, ); match maybe_state { - MaybeNext::Some(Ok((nested, values, validity))) => { + utils::MaybeNext::Some(Ok((nested, values, validity))) => { Some(Ok((nested, finish(&self.data_type, values, validity)))) } - MaybeNext::Some(Err(e)) => Some(Err(e)), - MaybeNext::None => None, - MaybeNext::More => self.next(), + utils::MaybeNext::Some(Err(e)) => Some(Err(e)), + utils::MaybeNext::None => None, + utils::MaybeNext::More => self.next(), } } } diff --git a/src/io/parquet/read/deserialize/simple.rs b/src/io/parquet/read/deserialize/simple.rs new file mode 100644 index 00000000000..58cc7dbca6f --- /dev/null +++ b/src/io/parquet/read/deserialize/simple.rs @@ -0,0 +1,531 @@ +use std::sync::Arc; + +use parquet2::{ + schema::types::{ + LogicalType, ParquetType, PhysicalType, TimeUnit as ParquetTimeUnit, TimestampType, + }, + types::int96_to_i64_ns, +}; + +use crate::{ + array::{Array, BinaryArray, DictionaryKey, MutablePrimitiveArray, PrimitiveArray, Utf8Array}, + datatypes::{DataType, IntervalUnit, TimeUnit}, + error::{ArrowError, Result}, + types::NativeType, +}; + +use super::super::{ArrayIter, DataPages}; +use super::binary; +use super::boolean; +use super::fixed_size_binary; +use super::null; +use super::primitive; + +/// Converts an iterator of arrays to a trait object returning trait objects +#[inline] +fn dyn_iter<'a, A, I>(iter: I) -> ArrayIter<'a> +where + A: Array + 'static, + I: Iterator> + Send + Sync + 'a, +{ + Box::new(iter.map(|x| x.map(|x| Arc::new(x) as Arc))) +} + +/// Converts an iterator of [MutablePrimitiveArray] into an iterator of [PrimitiveArray] +#[inline] +fn iden(iter: I) -> impl Iterator>> +where + T: NativeType, + I: Iterator>>, +{ + iter.map(|x| x.map(|x| x.into())) +} + +#[inline] +fn op(iter: I, op: F) -> impl Iterator>> +where + T: NativeType, + I: Iterator>>, + F: Fn(T) -> T + Copy, +{ + iter.map(move |x| { + x.map(move |mut x| { + x.values_mut_slice().iter_mut().for_each(|x| *x = op(*x)); + x.into() + }) + }) +} + +/// An iterator adapter that maps an iterator of DataPages into an iterator of Arrays +/// of [`DataType`] `data_type` and `chunk_size`. +pub fn page_iter_to_arrays<'a, I: 'a + DataPages>( + pages: I, + type_: &ParquetType, + data_type: DataType, + chunk_size: usize, +) -> Result> { + use DataType::*; + + let (physical_type, logical_type) = if let ParquetType::PrimitiveType { + physical_type, + logical_type, + .. + } = type_ + { + (physical_type, logical_type) + } else { + return Err(ArrowError::InvalidArgumentError( + "page_iter_to_arrays can only be called with a parquet primitive type".into(), + )); + }; + + Ok(match data_type.to_logical_type() { + Null => null::iter_to_arrays(pages, data_type, chunk_size), + Boolean => dyn_iter(boolean::Iter::new(pages, data_type, chunk_size)), + UInt8 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i32| x as u8, + ))), + UInt16 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i32| x as u16, + ))), + UInt32 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i32| x as u32, + ))), + Int8 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i32| x as i8, + ))), + Int16 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i32| x as i16, + ))), + Int32 | Date32 | Time32(_) | Interval(IntervalUnit::YearMonth) => dyn_iter(iden( + primitive::Iter::new(pages, data_type, chunk_size, |x: i32| x as i32), + )), + + Timestamp(time_unit, _) => { + let time_unit = *time_unit; + return timestamp( + pages, + physical_type, + logical_type, + data_type, + chunk_size, + time_unit, + ); + } + + FixedSizeBinary(_) => dyn_iter(fixed_size_binary::Iter::new(pages, data_type, chunk_size)), + + Decimal(_, _) => match physical_type { + PhysicalType::Int32 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i32| x as i128, + ))), + PhysicalType::Int64 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i64| x as i128, + ))), + &PhysicalType::FixedLenByteArray(n) if n > 16 => { + return Err(ArrowError::NotYetImplemented(format!( + "Can't decode Decimal128 type from Fixed Size Byte Array of len {:?}", + n + ))) + } + &PhysicalType::FixedLenByteArray(n) => { + let n = n as usize; + + let pages = + fixed_size_binary::Iter::new(pages, DataType::FixedSizeBinary(n), chunk_size); + + let pages = pages.map(move |maybe_array| { + let array = maybe_array?; + let values = array + .values() + .chunks_exact(n) + .map(|value: &[u8]| { + // Copy the fixed-size byte value to the start of a 16 byte stack + // allocated buffer, then use an arithmetic right shift to fill in + // MSBs, which accounts for leading 1's in negative (two's complement) + // values. + let mut bytes = [0u8; 16]; + bytes[..n].copy_from_slice(value); + i128::from_be_bytes(bytes) >> (8 * (16 - n)) + }) + .collect::>(); + let validity = array.validity().cloned(); + + Ok(PrimitiveArray::::from_data( + data_type.clone(), + values.into(), + validity, + )) + }); + + let arrays = pages.map(|x| x.map(|x| Arc::new(x) as Arc)); + + Box::new(arrays) as _ + } + _ => unreachable!(), + }, + + // INT64 + Int64 | Date64 | Time64(_) | Duration(_) => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i64| x as i64, + ))), + UInt64 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: i64| x as u64, + ))), + + Float32 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: f32| x, + ))), + Float64 => dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + |x: f64| x, + ))), + + Binary => dyn_iter(binary::Iter::, _>::new( + pages, data_type, chunk_size, + )), + LargeBinary => dyn_iter(binary::Iter::, _>::new( + pages, data_type, chunk_size, + )), + Utf8 => dyn_iter(binary::Iter::, _>::new( + pages, data_type, chunk_size, + )), + LargeUtf8 => dyn_iter(binary::Iter::, _>::new( + pages, data_type, chunk_size, + )), + + Dictionary(key_type, _, _) => { + return match_integer_type!(key_type, |$K| { + dict_read::<$K, _>(pages, physical_type, logical_type, data_type, chunk_size) + }) + } + + other => { + return Err(ArrowError::NotYetImplemented(format!( + "Reading {:?} from parquet still not implemented", + other + ))) + } + }) +} + +fn timestamp<'a, I: 'a + DataPages>( + pages: I, + physical_type: &PhysicalType, + logical_type: &Option, + data_type: DataType, + chunk_size: usize, + time_unit: TimeUnit, +) -> Result> { + if physical_type == &PhysicalType::Int96 { + if time_unit == TimeUnit::Nanosecond { + return Ok(dyn_iter(iden(primitive::Iter::new( + pages, + data_type, + chunk_size, + int96_to_i64_ns, + )))); + } else { + return Err(ArrowError::nyi( + "Can't decode int96 to timestamp other than ns", + )); + } + }; + if physical_type != &PhysicalType::Int64 { + return Err(ArrowError::nyi( + "Can't decode a timestamp from a non-int64 parquet type", + )); + } + + let iter = primitive::Iter::new(pages, data_type, chunk_size, |x: i64| x); + + let unit = if let Some(LogicalType::TIMESTAMP(TimestampType { unit, .. })) = logical_type { + unit + } else { + return Ok(dyn_iter(iden(iter))); + }; + + Ok(match (unit, time_unit) { + (ParquetTimeUnit::MILLIS(_), TimeUnit::Second) => dyn_iter(op(iter, |x| x / 1_000)), + (ParquetTimeUnit::MICROS(_), TimeUnit::Second) => dyn_iter(op(iter, |x| x / 1_000_000)), + (ParquetTimeUnit::NANOS(_), TimeUnit::Second) => dyn_iter(op(iter, |x| x * 1_000_000_000)), + + (ParquetTimeUnit::MILLIS(_), TimeUnit::Millisecond) => dyn_iter(iden(iter)), + (ParquetTimeUnit::MICROS(_), TimeUnit::Millisecond) => dyn_iter(op(iter, |x| x / 1_000)), + (ParquetTimeUnit::NANOS(_), TimeUnit::Millisecond) => dyn_iter(op(iter, |x| x / 1_000_000)), + + (ParquetTimeUnit::MILLIS(_), TimeUnit::Microsecond) => dyn_iter(op(iter, |x| x * 1_000)), + (ParquetTimeUnit::MICROS(_), TimeUnit::Microsecond) => dyn_iter(iden(iter)), + (ParquetTimeUnit::NANOS(_), TimeUnit::Microsecond) => dyn_iter(op(iter, |x| x / 1_000)), + + (ParquetTimeUnit::MILLIS(_), TimeUnit::Nanosecond) => dyn_iter(op(iter, |x| x * 1_000_000)), + (ParquetTimeUnit::MICROS(_), TimeUnit::Nanosecond) => dyn_iter(op(iter, |x| x * 1_000)), + (ParquetTimeUnit::NANOS(_), TimeUnit::Nanosecond) => dyn_iter(iden(iter)), + }) +} + +fn timestamp_dict<'a, K: DictionaryKey, I: 'a + DataPages>( + pages: I, + physical_type: &PhysicalType, + logical_type: &Option, + data_type: DataType, + chunk_size: usize, + time_unit: TimeUnit, +) -> Result> { + if physical_type == &PhysicalType::Int96 { + if time_unit == TimeUnit::Nanosecond { + return Ok(dyn_iter(primitive::DictIter::::new( + pages, + DataType::Timestamp(TimeUnit::Nanosecond, None), + chunk_size, + int96_to_i64_ns, + ))); + } else { + return Err(ArrowError::nyi( + "Can't decode int96 to timestamp other than ns", + )); + } + }; + + let unit = if let Some(LogicalType::TIMESTAMP(TimestampType { unit, .. })) = logical_type { + unit + } else { + return Ok(dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x, + ))); + }; + + Ok(match (unit, time_unit) { + (ParquetTimeUnit::MILLIS(_), TimeUnit::Second) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x / 1_000, + )) + } + (ParquetTimeUnit::MICROS(_), TimeUnit::Second) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x / 1_000_000, + )) + } + (ParquetTimeUnit::NANOS(_), TimeUnit::Second) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x * 1_000_000_000, + )) + } + + (ParquetTimeUnit::MILLIS(_), TimeUnit::Millisecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x, + )) + } + (ParquetTimeUnit::MICROS(_), TimeUnit::Millisecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x / 1_000, + )) + } + (ParquetTimeUnit::NANOS(_), TimeUnit::Millisecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x / 1_000_000, + )) + } + + (ParquetTimeUnit::MILLIS(_), TimeUnit::Microsecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x * 1_000, + )) + } + (ParquetTimeUnit::MICROS(_), TimeUnit::Microsecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x, + )) + } + (ParquetTimeUnit::NANOS(_), TimeUnit::Microsecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x / 1_000, + )) + } + + (ParquetTimeUnit::MILLIS(_), TimeUnit::Nanosecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x * 1_000_000, + )) + } + (ParquetTimeUnit::MICROS(_), TimeUnit::Nanosecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x * 1_000, + )) + } + (ParquetTimeUnit::NANOS(_), TimeUnit::Nanosecond) => { + dyn_iter(primitive::DictIter::::new( + pages, + data_type, + chunk_size, + |x: i64| x, + )) + } + }) +} + +fn dict_read<'a, K: DictionaryKey, I: 'a + DataPages>( + iter: I, + physical_type: &PhysicalType, + logical_type: &Option, + data_type: DataType, + chunk_size: usize, +) -> Result> { + use DataType::*; + let values_data_type = if let Dictionary(_, v, _) = &data_type { + v.as_ref() + } else { + panic!() + }; + + Ok(match values_data_type.to_logical_type() { + UInt8 => dyn_iter(primitive::DictIter::::new( + iter, + data_type, + chunk_size, + |x: i32| x as u8, + )), + UInt16 => dyn_iter(primitive::DictIter::::new( + iter, + data_type, + chunk_size, + |x: i32| x as u16, + )), + UInt32 => dyn_iter(primitive::DictIter::::new( + iter, + data_type, + chunk_size, + |x: i32| x as u32, + )), + Int8 => dyn_iter(primitive::DictIter::::new( + iter, + data_type, + chunk_size, + |x: i32| x as i8, + )), + Int16 => dyn_iter(primitive::DictIter::::new( + iter, + data_type, + chunk_size, + |x: i32| x as i16, + )), + Int32 | Date32 | Time32(_) | Interval(IntervalUnit::YearMonth) => dyn_iter( + primitive::DictIter::::new(iter, data_type, chunk_size, |x: i32| { + x as i32 + }), + ), + + Timestamp(time_unit, _) => { + let time_unit = *time_unit; + return timestamp_dict::( + iter, + physical_type, + logical_type, + data_type, + chunk_size, + time_unit, + ); + } + + Int64 | Date64 | Time64(_) | Duration(_) => dyn_iter( + primitive::DictIter::::new(iter, data_type, chunk_size, |x: i64| x), + ), + Float32 => dyn_iter(primitive::DictIter::::new( + iter, + data_type, + chunk_size, + |x: f32| x, + )), + Float64 => dyn_iter(primitive::DictIter::::new( + iter, + data_type, + chunk_size, + |x: f64| x, + )), + + Utf8 | Binary => dyn_iter(binary::DictIter::::new( + iter, data_type, chunk_size, + )), + LargeUtf8 | LargeBinary => dyn_iter(binary::DictIter::::new( + iter, data_type, chunk_size, + )), + FixedSizeBinary(_) => dyn_iter(fixed_size_binary::DictIter::::new( + iter, data_type, chunk_size, + )), + other => { + return Err(ArrowError::nyi(format!( + "Reading dictionaries of type {:?}", + other + ))) + } + }) +} diff --git a/src/io/parquet/read/utils.rs b/src/io/parquet/read/deserialize/utils.rs similarity index 99% rename from src/io/parquet/read/utils.rs rename to src/io/parquet/read/deserialize/utils.rs index b86c8da58f3..c9a9604992c 100644 --- a/src/io/parquet/read/utils.rs +++ b/src/io/parquet/read/deserialize/utils.rs @@ -9,7 +9,7 @@ use crate::bitmap::utils::BitmapIter; use crate::bitmap::MutableBitmap; use crate::error::ArrowError; -use super::DataPages; +use super::super::DataPages; #[derive(Debug)] pub struct BinaryIter<'a> { diff --git a/src/io/parquet/read/file.rs b/src/io/parquet/read/file.rs index 126253e6b2e..8251392c6c0 100644 --- a/src/io/parquet/read/file.rs +++ b/src/io/parquet/read/file.rs @@ -14,14 +14,13 @@ use super::{infer_schema, read_metadata, FileMetaData, RowGroupDeserializer, Row type GroupFilter = Arc bool>; -/// An iterator of [`Chunk`] coming from row groups of a paquet file. +/// An iterator of [`Chunk`]s coming from row groups of a parquet file. /// -/// This can be thought of flatten chain of [`Iterator`] - each row group is sequentially +/// This can be thought of a flatten chain of [`Iterator`] - each row group is sequentially /// mapped to an [`Iterator`] and each iterator is iterated upon until either the limit /// or the last iterator ends. -/// /// # Implementation -/// Note that because +/// This iterator mixes IO-bounded and CPU-bounded operations. pub struct FileReader { row_groups: RowGroupReader, metadata: FileMetaData, diff --git a/src/io/parquet/read/mod.rs b/src/io/parquet/read/mod.rs index 9539652a7de..14bcbef3c02 100644 --- a/src/io/parquet/read/mod.rs +++ b/src/io/parquet/read/mod.rs @@ -1,12 +1,15 @@ //! APIs to read from Parquet format. #![allow(clippy::type_complexity)] -use std::{ - io::{Read, Seek}, - sync::Arc, -}; +mod deserialize; +mod file; +mod row_group; +pub mod schema; +pub mod statistics; use futures::{AsyncRead, AsyncSeek}; + +// re-exports of parquet2's relevant APIs pub use parquet2::{ error::ParquetError, fallible_streaming_iterator, @@ -26,97 +29,32 @@ pub use parquet2::{ FallibleStreamingIterator, }; -use crate::{ - array::{ - Array, BinaryArray, ListArray, MutablePrimitiveArray, PrimitiveArray, StructArray, - Utf8Array, - }, - datatypes::{DataType, Field}, - error::{ArrowError, Result}, - types::NativeType, -}; - -mod binary; -mod boolean; -mod deserialize; -mod dictionary; -mod file; -mod fixed_size_binary; -mod nested_utils; -mod null; -mod primitive; -mod row_group; -pub mod schema; -pub mod statistics; -mod utils; - +pub use deserialize::{column_iter_to_arrays, get_page_iterator}; pub use file::{FileReader, RowGroupReader}; pub use row_group::*; pub(crate) use schema::is_type_nullable; pub use schema::{infer_schema, FileMetaData}; -use self::nested_utils::{InitNested, NestedArrayIter, NestedState}; -use deserialize::page_iter_to_arrays; +use std::{ + io::{Read, Seek}, + sync::Arc, +}; + +use crate::{array::Array, error::Result}; /// Trait describing a [`FallibleStreamingIterator`] of [`DataPage`] pub trait DataPages: FallibleStreamingIterator + Send + Sync { } + impl + Send + Sync> DataPages for I { } -/// Converts an iterator of arrays to a trait object returning trait objects -#[inline] -fn dyn_iter<'a, A, I>(iter: I) -> ArrayIter<'a> -where - A: Array + 'static, - I: Iterator> + Send + Sync + 'a, -{ - Box::new(iter.map(|x| x.map(|x| Arc::new(x) as Arc))) -} - -/// Converts an iterator of [MutablePrimitiveArray] into an iterator of [PrimitiveArray] -#[inline] -fn iden(iter: I) -> impl Iterator>> -where - T: NativeType, - I: Iterator>>, -{ - iter.map(|x| x.map(|x| x.into())) -} - -#[inline] -fn op(iter: I, op: F) -> impl Iterator>> -where - T: NativeType, - I: Iterator>>, - F: Fn(T) -> T + Copy, -{ - iter.map(move |x| { - x.map(move |mut x| { - x.values_mut_slice().iter_mut().for_each(|x| *x = op(*x)); - x.into() - }) - }) -} - -/// Creates a new iterator of compressed pages. -pub fn get_page_iterator( - column_metadata: &ColumnChunkMetaData, - reader: R, - pages_filter: Option, - buffer: Vec, -) -> Result> { - Ok(_get_page_iterator( - column_metadata, - reader, - pages_filter, - buffer, - )?) -} +/// Type def for a sharable, boxed dyn [`Iterator`] of arrays +pub type ArrayIter<'a> = Box>> + Send + Sync + 'a>; /// Reads parquets' metadata syncronously. pub fn read_metadata(reader: &mut R) -> Result { @@ -129,239 +67,3 @@ pub async fn read_metadata_async( ) -> Result { Ok(_read_metadata_async(reader).await?) } - -fn create_list( - data_type: DataType, - nested: &mut NestedState, - values: Arc, -) -> Result> { - Ok(match data_type { - DataType::List(_) => { - let (offsets, validity) = nested.nested.pop().unwrap().inner(); - - let offsets = offsets.iter().map(|x| *x as i32).collect::>(); - Arc::new(ListArray::::from_data( - data_type, - offsets.into(), - values, - validity, - )) - } - DataType::LargeList(_) => { - let (offsets, validity) = nested.nested.pop().unwrap().inner(); - - Arc::new(ListArray::::from_data( - data_type, offsets, values, validity, - )) - } - _ => { - return Err(ArrowError::NotYetImplemented(format!( - "Read nested datatype {:?}", - data_type - ))) - } - }) -} - -struct StructIterator<'a> { - iters: Vec>, - fields: Vec, -} - -impl<'a> StructIterator<'a> { - pub fn new(iters: Vec>, fields: Vec) -> Self { - assert_eq!(iters.len(), fields.len()); - Self { iters, fields } - } -} - -impl<'a> Iterator for StructIterator<'a> { - type Item = Result<(NestedState, Arc)>; - - fn next(&mut self) -> Option { - let values = self - .iters - .iter_mut() - .map(|iter| iter.next()) - .collect::>(); - - if values.iter().any(|x| x.is_none()) { - return None; - } - let values = values - .into_iter() - .map(|x| x.unwrap().map(|x| x.1)) - .collect::>>(); - - match values { - Ok(values) => Some(Ok(( - NestedState::new(vec![]), // todo - Arc::new(StructArray::from_data( - DataType::Struct(self.fields.clone()), - values, - None, - )), - ))), - Err(e) => Some(Err(e)), - } - } -} - -fn columns_to_iter_recursive<'a, I: 'a>( - mut columns: Vec, - mut types: Vec<&ParquetType>, - field: Field, - mut init: Vec, - chunk_size: usize, -) -> Result> -where - I: DataPages, -{ - use DataType::*; - if init.len() == 1 && init[0].is_primitive() { - return Ok(Box::new( - page_iter_to_arrays( - columns.pop().unwrap(), - types.pop().unwrap(), - field.data_type, - chunk_size, - )? - .map(|x| Ok((NestedState::new(vec![]), x?))), - )); - } - - Ok(match field.data_type().to_logical_type() { - Boolean => { - types.pop(); - boolean::iter_to_arrays_nested(columns.pop().unwrap(), init.pop().unwrap(), chunk_size) - } - Int16 => { - types.pop(); - primitive::iter_to_arrays_nested( - columns.pop().unwrap(), - init.pop().unwrap(), - field.data_type().clone(), - chunk_size, - |x: i32| x as i16, - ) - } - Int64 => { - types.pop(); - primitive::iter_to_arrays_nested( - columns.pop().unwrap(), - init.pop().unwrap(), - field.data_type().clone(), - chunk_size, - |x: i64| x, - ) - } - Utf8 => { - types.pop(); - binary::iter_to_arrays_nested::, _>( - columns.pop().unwrap(), - init.pop().unwrap(), - field.data_type().clone(), - chunk_size, - ) - } - LargeBinary => { - types.pop(); - binary::iter_to_arrays_nested::, _>( - columns.pop().unwrap(), - init.pop().unwrap(), - field.data_type().clone(), - chunk_size, - ) - } - List(inner) => { - let iter = columns_to_iter_recursive( - vec![columns.pop().unwrap()], - types, - inner.as_ref().clone(), - init, - chunk_size, - )?; - let iter = iter.map(move |x| { - let (mut nested, array) = x?; - let array = create_list(field.data_type().clone(), &mut nested, array)?; - Ok((nested, array)) - }); - Box::new(iter) as _ - } - Struct(fields) => { - let columns = fields - .iter() - .rev() - .map(|f| { - columns_to_iter_recursive( - vec![columns.pop().unwrap()], - vec![types.pop().unwrap()], - f.clone(), - vec![init.pop().unwrap()], - chunk_size, - ) - }) - .collect::>>()?; - let columns = columns.into_iter().rev().collect(); - Box::new(StructIterator::new(columns, fields.clone())) - } - _ => todo!(), - }) -} - -fn field_to_init(field: &Field) -> Vec { - use crate::datatypes::PhysicalType::*; - match field.data_type.to_physical_type() { - Null | Boolean | Primitive(_) | Binary | FixedSizeBinary | LargeBinary | Utf8 - | Dictionary(_) | LargeUtf8 => vec![InitNested::Primitive(field.is_nullable)], - List | FixedSizeList | LargeList => { - let a = field.data_type().to_logical_type(); - let inner = if let DataType::List(inner) = a { - field_to_init(inner) - } else if let DataType::LargeList(inner) = a { - field_to_init(inner) - } else if let DataType::FixedSizeList(inner, _) = a { - field_to_init(inner) - } else { - unreachable!() - }; - inner - .into_iter() - .map(|x| InitNested::List(Box::new(x), field.is_nullable)) - .collect() - } - Struct => { - let inner = if let DataType::Struct(fields) = field.data_type.to_logical_type() { - fields.iter().rev().map(field_to_init).collect::>() - } else { - unreachable!() - }; - inner - .into_iter() - .flatten() - .map(|x| InitNested::Struct(Box::new(x), field.is_nullable)) - .collect() - } - _ => todo!(), - } -} - -/// Returns an iterator of [`Array`] built from an iterator of column chunks. -pub fn column_iter_to_arrays<'a, I: 'static>( - columns: Vec, - types: Vec<&ParquetType>, - field: Field, - chunk_size: usize, -) -> Result> -where - I: DataPages, -{ - let init = field_to_init(&field); - - Ok(Box::new( - columns_to_iter_recursive(columns, types, field, init, chunk_size)?.map(|x| x.map(|x| x.1)), - )) -} - -/// Type def for a sharable, boxed dyn [`Iterator`] of arrays -pub type ArrayIter<'a> = Box>> + Send + Sync + 'a>; diff --git a/src/io/parquet/read/row_group.rs b/src/io/parquet/read/row_group.rs index 587544b6889..f83a65eadbe 100644 --- a/src/io/parquet/read/row_group.rs +++ b/src/io/parquet/read/row_group.rs @@ -87,7 +87,7 @@ impl Iterator for RowGroupDeserializer { } } -/// Returns all the parquet columns associated to `field_name`. +/// Returns all [`ColumnChunkMetaData`] associated to `field_name`. /// For non-nested parquet types, this returns a single column pub(super) fn get_field_columns<'a>( columns: &'a [ColumnChunkMetaData], diff --git a/src/io/parquet/read/schema/convert.rs b/src/io/parquet/read/schema/convert.rs index 4fea0db5994..ae2d66a1b9b 100644 --- a/src/io/parquet/read/schema/convert.rs +++ b/src/io/parquet/read/schema/convert.rs @@ -12,7 +12,7 @@ use crate::datatypes::{DataType, Field, IntervalUnit, TimeUnit}; /// Converts [`ParquetType`]s to a [`Field`], ignoring parquet fields that do not contain /// any physical column. pub fn parquet_to_arrow_schema(fields: &[ParquetType]) -> Vec { - fields.iter().map(to_field).flatten().collect::>() + fields.iter().filter_map(to_field).collect::>() } fn from_int32( @@ -224,11 +224,7 @@ fn non_repeated_group( /// Converts a parquet group type to an arrow [`DataType::Struct`]. /// Returns [`None`] if all its fields are empty fn to_struct(fields: &[ParquetType]) -> Option { - let fields = fields - .iter() - .map(to_field) - .flatten() - .collect::>(); + let fields = fields.iter().filter_map(to_field).collect::>(); if fields.is_empty() { None } else { diff --git a/src/io/parquet/read/statistics/mod.rs b/src/io/parquet/read/statistics/mod.rs index dcee10a2126..d42a2ad16e6 100644 --- a/src/io/parquet/read/statistics/mod.rs +++ b/src/io/parquet/read/statistics/mod.rs @@ -101,7 +101,7 @@ fn get_fields(field: &Field) -> Vec<&Field> { match field.data_type.to_logical_type() { DataType::List(inner) => get_fields(inner), DataType::LargeList(inner) => get_fields(inner), - DataType::Struct(fields) => fields.iter().map(get_fields).flatten().collect(), + DataType::Struct(fields) => fields.iter().flat_map(get_fields).collect(), _ => vec![field], } } diff --git a/src/io/parquet/write/dictionary.rs b/src/io/parquet/write/dictionary.rs index a18b654e036..521c863aac7 100644 --- a/src/io/parquet/write/dictionary.rs +++ b/src/io/parquet/write/dictionary.rs @@ -60,19 +60,15 @@ fn encode_keys( // encode indices // compute the required number of bits if let Some(validity) = validity { - let keys = array - .iter() - .flatten() - .map(|x| { - let index = x.to_usize().unwrap(); - // discard indices whose values are null, since they are part of the def levels. - if validity.get_bit(index) { - Some(index as u32) - } else { - None - } - }) - .flatten(); + let keys = array.iter().flatten().filter_map(|x| { + let index = x.to_usize().unwrap(); + // discard indices whose values are null, since they are part of the def levels. + if validity.get_bit(index) { + Some(index as u32) + } else { + None + } + }); let num_bits = utils::get_bit_width(keys.clone().max().unwrap_or(0) as u64) as u8; let keys = utils::ExactSizedIter::new(keys, array.len() - null_count); diff --git a/src/types/bit_chunk.rs b/src/types/bit_chunk.rs index eba57e5e484..22f677aae4a 100644 --- a/src/types/bit_chunk.rs +++ b/src/types/bit_chunk.rs @@ -1,123 +1,52 @@ use std::{ fmt::Binary, - ops::{BitAnd, BitAndAssign, BitOr, Not, Shl, ShlAssign, ShrAssign}, + ops::{BitAndAssign, Not, Shl, ShlAssign, ShrAssign}, }; +use num_traits::PrimInt; + use super::NativeType; /// A chunk of bits. This is used to create masks of a given length /// whose width is `1` bit. In `simd_packed` notation, this corresponds to `m1xY`. pub trait BitChunk: super::private::Sealed + + PrimInt + NativeType + Binary - + BitAnd + ShlAssign + Not + ShrAssign + ShlAssign + Shl - + Eq + BitAndAssign - + BitOr { - /// A value with a single bit set at the most right position. - fn one() -> Self; - /// A value with no bits set. - fn zero() -> Self; /// convert itself into bytes. fn to_ne_bytes(self) -> Self::Bytes; /// convert itself from bytes. fn from_ne_bytes(v: Self::Bytes) -> Self; } -impl BitChunk for u8 { - #[inline(always)] - fn zero() -> Self { - 0 - } - - #[inline(always)] - fn to_ne_bytes(self) -> Self::Bytes { - self.to_ne_bytes() - } - - #[inline(always)] - fn from_ne_bytes(v: Self::Bytes) -> Self { - Self::from_ne_bytes(v) - } - - #[inline(always)] - fn one() -> Self { - 1 - } -} - -impl BitChunk for u16 { - #[inline(always)] - fn zero() -> Self { - 0 - } - - #[inline(always)] - fn to_ne_bytes(self) -> Self::Bytes { - self.to_ne_bytes() - } - - #[inline(always)] - fn from_ne_bytes(v: Self::Bytes) -> Self { - Self::from_ne_bytes(v) - } - - #[inline(always)] - fn one() -> Self { - 1 - } -} - -impl BitChunk for u32 { - #[inline(always)] - fn zero() -> Self { - 0 - } - - #[inline(always)] - fn from_ne_bytes(v: Self::Bytes) -> Self { - Self::from_ne_bytes(v) - } - - #[inline(always)] - fn to_ne_bytes(self) -> Self::Bytes { - self.to_ne_bytes() - } - - #[inline(always)] - fn one() -> Self { - 1 - } +macro_rules! bit_chunk { + ($ty:ty) => { + impl BitChunk for $ty { + #[inline(always)] + fn to_ne_bytes(self) -> Self::Bytes { + self.to_ne_bytes() + } + + #[inline(always)] + fn from_ne_bytes(v: Self::Bytes) -> Self { + Self::from_ne_bytes(v) + } + } + }; } -impl BitChunk for u64 { - #[inline(always)] - fn zero() -> Self { - 0 - } - - #[inline(always)] - fn to_ne_bytes(self) -> Self::Bytes { - self.to_ne_bytes() - } - - #[inline(always)] - fn from_ne_bytes(v: Self::Bytes) -> Self { - Self::from_ne_bytes(v) - } - - #[inline(always)] - fn one() -> Self { - 1 - } -} +bit_chunk!(u8); +bit_chunk!(u16); +bit_chunk!(u32); +bit_chunk!(u64); /// An [`Iterator`] over a [`BitChunk`]. This iterator is often /// compiled to SIMD. @@ -170,6 +99,62 @@ impl Iterator for BitChunkIter { } } +// # Safety +// a mathematical invariant of this iterator +unsafe impl crate::trusted_len::TrustedLen for BitChunkIter {} + +/// An [`Iterator`] over a [`BitChunk`]. +/// This iterator returns the postion of bit set. +/// Refer: https://lemire.me/blog/2018/03/08/iterating-over-set-bits-quickly-simd-edition/ +/// # Example +/// ``` +/// use arrow2::types::BitChunkOnes; +/// let a = 0b00010000u8; +/// let iter = BitChunkOnes::new(a); +/// let r = iter.collect::>(); +/// assert_eq!(r, vec![4]); +/// ``` +pub struct BitChunkOnes { + value: T, + remaining: usize, +} + +impl BitChunkOnes { + /// Creates a new [`BitChunkOnes`] with `len` bits. + #[inline] + pub fn new(value: T) -> Self { + Self { + value, + remaining: value.count_ones() as usize, + } + } +} + +impl Iterator for BitChunkOnes { + type Item = usize; + + #[inline] + fn next(&mut self) -> Option { + if self.remaining == 0 { + return None; + } + let v = self.value.trailing_zeros() as usize; + self.value &= self.value - T::one(); + + self.remaining -= 1; + Some(v) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + (self.remaining, Some(self.remaining)) + } +} + +// # Safety +// a mathematical invariant of this iterator +unsafe impl crate::trusted_len::TrustedLen for BitChunkOnes {} + #[cfg(test)] mod tests { use super::*; @@ -182,4 +167,14 @@ mod tests { let r = iter.collect::>(); assert_eq!(r, (0..16).map(|x| x == 0 || x == 12).collect::>(),); } + + #[test] + fn test_ones() { + let a = [0b00000001, 0b00010000]; // 0th and 13th entry + let a = u16::from_ne_bytes(a); + let mut iter = BitChunkOnes::new(a); + assert_eq!(iter.size_hint(), (2, Some(2))); + assert_eq!(iter.next(), Some(0)); + assert_eq!(iter.next(), Some(12)); + } } diff --git a/src/types/mod.rs b/src/types/mod.rs index 4ba1584acf9..93e5b0667ce 100644 --- a/src/types/mod.rs +++ b/src/types/mod.rs @@ -21,7 +21,7 @@ //! for SIMD, at [`mod@simd`]. mod bit_chunk; -pub use bit_chunk::{BitChunk, BitChunkIter}; +pub use bit_chunk::{BitChunk, BitChunkIter, BitChunkOnes}; mod index; pub mod simd; pub use index::*; @@ -30,8 +30,12 @@ pub use native::*; mod offset; pub use offset::*; +#[cfg(feature = "serde_types")] +use serde_derive::{Deserialize, Serialize}; + /// The set of all implementations of the sealed trait [`NativeType`]. #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] pub enum PrimitiveType { /// A signed 8-bit integer. Int8, diff --git a/tests/it/array/list/mod.rs b/tests/it/array/list/mod.rs index 78bc8dcd719..2a4e1feca4e 100644 --- a/tests/it/array/list/mod.rs +++ b/tests/it/array/list/mod.rs @@ -23,7 +23,7 @@ fn debug() { } #[test] -#[should_panic(expected = "The child's datatype must match the inner type of the \'data_type\'")] +#[should_panic] fn test_nested_panic() { let values = Buffer::from_slice([1, 2, 3, 4, 5]); let values = PrimitiveArray::::from_data(DataType::Int32, values, None); diff --git a/tests/it/compute/lower.rs b/tests/it/compute/lower.rs deleted file mode 100644 index d8f594174fb..00000000000 --- a/tests/it/compute/lower.rs +++ /dev/null @@ -1,186 +0,0 @@ -use arrow2::{array::*, compute::lower::*, error::Result}; - -fn with_nulls_utf8() -> Result<()> { - let cases = vec![ - // identity - ( - vec![Some("hello"), None, Some("world")], - vec![Some("hello"), None, Some("world")], - ), - // part of input - ( - vec![Some("Hello"), None, Some("wOrld")], - vec![Some("hello"), None, Some("world")], - ), - // all input - ( - vec![Some("HELLO"), None, Some("WORLD")], - vec![Some("hello"), None, Some("world")], - ), - // UTF8 characters - ( - vec![ - None, - Some("السلام عليكم"), - Some("Dobrý den"), - Some("שָׁלוֹם"), - Some("नमस्ते"), - Some("こんにちは"), - Some("안녕하세요"), - Some("你好"), - Some("Olá"), - Some("Здравствуйте"), - Some("Hola"), - ], - vec![ - None, - Some("السلام عليكم"), - Some("dobrý den"), - Some("שָׁלוֹם"), - Some("नमस्ते"), - Some("こんにちは"), - Some("안녕하세요"), - Some("你好"), - Some("olá"), - Some("здравствуйте"), - Some("hola"), - ], - ), - ]; - - cases - .into_iter() - .try_for_each::<_, Result<()>>(|(array, expected)| { - let array = Utf8Array::::from(&array); - let result = lower(&array)?; - assert_eq!(array.len(), result.len()); - - let result = result.as_any().downcast_ref::>().unwrap(); - let expected = Utf8Array::::from(&expected); - - assert_eq!(&expected, result); - Ok(()) - })?; - - Ok(()) -} - -#[test] -fn with_nulls_string() -> Result<()> { - with_nulls_utf8::() -} - -#[test] -fn with_nulls_large_string() -> Result<()> { - with_nulls_utf8::() -} - -fn without_nulls_utf8() -> Result<()> { - let cases = vec![ - // identity - (vec!["hello", "world"], vec!["hello", "world"]), - // part of input - (vec!["Hello", "wOrld"], vec!["hello", "world"]), - // all input - (vec!["HELLO", "WORLD"], vec!["hello", "world"]), - // UTF8 characters - ( - vec![ - "السلام عليكم", - "Dobrý den", - "שָׁלוֹם", - "नमस्ते", - "こんにちは", - "안녕하세요", - "你好", - "Olá", - "Здравствуйте", - "Hola", - ], - vec![ - "السلام عليكم", - "dobrý den", - "שָׁלוֹם", - "नमस्ते", - "こんにちは", - "안녕하세요", - "你好", - "olá", - "здравствуйте", - "hola", - ], - ), - ]; - - cases - .into_iter() - .try_for_each::<_, Result<()>>(|(array, expected)| { - let array = Utf8Array::::from_slice(&array); - let result = lower(&array)?; - assert_eq!(array.len(), result.len()); - - let result = result.as_any().downcast_ref::>().unwrap(); - let expected = Utf8Array::::from_slice(&expected); - assert_eq!(&expected, result); - Ok(()) - })?; - - Ok(()) -} - -#[test] -fn without_nulls_string() -> Result<()> { - without_nulls_utf8::() -} - -#[test] -fn without_nulls_large_string() -> Result<()> { - without_nulls_utf8::() -} - -#[test] -fn consistency() { - use arrow2::datatypes::DataType::*; - use arrow2::datatypes::TimeUnit; - let datatypes = vec![ - Null, - Boolean, - UInt8, - UInt16, - UInt32, - UInt64, - Int8, - Int16, - Int32, - Int64, - Float32, - Float64, - Timestamp(TimeUnit::Second, None), - Timestamp(TimeUnit::Millisecond, None), - Timestamp(TimeUnit::Microsecond, None), - Timestamp(TimeUnit::Nanosecond, None), - Time64(TimeUnit::Microsecond), - Time64(TimeUnit::Nanosecond), - Date32, - Time32(TimeUnit::Second), - Time32(TimeUnit::Millisecond), - Date64, - Utf8, - LargeUtf8, - Binary, - LargeBinary, - Duration(TimeUnit::Second), - Duration(TimeUnit::Millisecond), - Duration(TimeUnit::Microsecond), - Duration(TimeUnit::Nanosecond), - ]; - - datatypes.into_iter().for_each(|d1| { - let array = new_null_array(d1.clone(), 10); - if can_lower(&d1) { - assert!(lower(array.as_ref()).is_ok()); - } else { - assert!(lower(array.as_ref()).is_err()); - } - }); -} diff --git a/tests/it/compute/mod.rs b/tests/it/compute/mod.rs index 9b5079be712..f4972c21940 100644 --- a/tests/it/compute/mod.rs +++ b/tests/it/compute/mod.rs @@ -28,8 +28,6 @@ mod length; mod like; #[cfg(feature = "compute_limit")] mod limit; -#[cfg(feature = "compute_lower")] -mod lower; #[cfg(feature = "compute_merge_sort")] mod merge_sort; #[cfg(feature = "compute_partition")] @@ -44,7 +42,7 @@ mod substring; mod take; #[cfg(feature = "compute_temporal")] mod temporal; -#[cfg(feature = "compute_upper")] -mod upper; +#[cfg(feature = "compute_utf8")] +mod utf8; #[cfg(feature = "compute_window")] mod window; diff --git a/tests/it/compute/sort/mod.rs b/tests/it/compute/sort/mod.rs index a29d99e2986..7618c041352 100644 --- a/tests/it/compute/sort/mod.rs +++ b/tests/it/compute/sort/mod.rs @@ -27,15 +27,15 @@ fn primitive_arrays( } fn to_indices_string_arrays(data: &[Option<&str>], options: SortOptions, expected_data: &[i32]) { - let input = Utf8Array::::from(&data.to_vec()); + let input = Utf8Array::::from(data); let expected = Int32Array::from_slice(expected_data); let output = sort_to_indices(&input, &options, None).unwrap(); assert_eq!(output, expected) } fn string_arrays(data: &[Option<&str>], options: SortOptions, expected_data: &[Option<&str>]) { - let input = Utf8Array::::from(&data.to_vec()); - let expected = Utf8Array::::from(&expected_data.to_vec()); + let input = Utf8Array::::from(data); + let expected = Utf8Array::::from(expected_data); let output = sort(&input, &options, None).unwrap(); assert_eq!(expected, output.as_ref()) } diff --git a/tests/it/compute/upper.rs b/tests/it/compute/upper.rs deleted file mode 100644 index 4daf35626a9..00000000000 --- a/tests/it/compute/upper.rs +++ /dev/null @@ -1,186 +0,0 @@ -use arrow2::{array::*, compute::upper::*, error::Result}; - -fn with_nulls_utf8() -> Result<()> { - let cases = vec![ - // identity - ( - vec![Some("hello"), None, Some("world")], - vec![Some("HELLO"), None, Some("WORLD")], - ), - // part of input - ( - vec![Some("Hello"), None, Some("wOrld")], - vec![Some("HELLO"), None, Some("WORLD")], - ), - // all input - ( - vec![Some("hello"), None, Some("world")], - vec![Some("HELLO"), None, Some("WORLD")], - ), - // UTF8 characters - ( - vec![ - None, - Some("السلام عليكم"), - Some("Dobrý den"), - Some("שָׁלוֹם"), - Some("नमस्ते"), - Some("こんにちは"), - Some("안녕하세요"), - Some("你好"), - Some("Olá"), - Some("Здравствуйте"), - Some("Hola"), - ], - vec![ - None, - Some("السلام عليكم"), - Some("DOBRÝ DEN"), - Some("שָׁלוֹם"), - Some("नमस्ते"), - Some("こんにちは"), - Some("안녕하세요"), - Some("你好"), - Some("OLÁ"), - Some("ЗДРАВСТВУЙТЕ"), - Some("HOLA"), - ], - ), - ]; - - cases - .into_iter() - .try_for_each::<_, Result<()>>(|(array, expected)| { - let array = Utf8Array::::from(&array); - let result = upper(&array)?; - assert_eq!(array.len(), result.len()); - - let result = result.as_any().downcast_ref::>().unwrap(); - let expected = Utf8Array::::from(&expected); - - assert_eq!(&expected, result); - Ok(()) - })?; - - Ok(()) -} - -#[test] -fn with_nulls_string() -> Result<()> { - with_nulls_utf8::() -} - -#[test] -fn with_nulls_large_string() -> Result<()> { - with_nulls_utf8::() -} - -fn without_nulls_utf8() -> Result<()> { - let cases = vec![ - // identity - (vec!["hello", "world"], vec!["HELLO", "WORLD"]), - // part of input - (vec!["Hello", "wOrld"], vec!["HELLO", "WORLD"]), - // all input - (vec!["HELLO", "WORLD"], vec!["HELLO", "WORLD"]), - // UTF8 characters - ( - vec![ - "السلام عليكم", - "Dobrý den", - "שָׁלוֹם", - "नमस्ते", - "こんにちは", - "안녕하세요", - "你好", - "Olá", - "Здравствуйте", - "Hola", - ], - vec![ - "السلام عليكم", - "DOBRÝ DEN", - "שָׁלוֹם", - "नमस्ते", - "こんにちは", - "안녕하세요", - "你好", - "OLÁ", - "ЗДРАВСТВУЙТЕ", - "HOLA", - ], - ), - ]; - - cases - .into_iter() - .try_for_each::<_, Result<()>>(|(array, expected)| { - let array = Utf8Array::::from_slice(&array); - let result = upper(&array)?; - assert_eq!(array.len(), result.len()); - - let result = result.as_any().downcast_ref::>().unwrap(); - let expected = Utf8Array::::from_slice(&expected); - assert_eq!(&expected, result); - Ok(()) - })?; - - Ok(()) -} - -#[test] -fn without_nulls_string() -> Result<()> { - without_nulls_utf8::() -} - -#[test] -fn without_nulls_large_string() -> Result<()> { - without_nulls_utf8::() -} - -#[test] -fn consistency() { - use arrow2::datatypes::DataType::*; - use arrow2::datatypes::TimeUnit; - let datatypes = vec![ - Null, - Boolean, - UInt8, - UInt16, - UInt32, - UInt64, - Int8, - Int16, - Int32, - Int64, - Float32, - Float64, - Timestamp(TimeUnit::Second, None), - Timestamp(TimeUnit::Millisecond, None), - Timestamp(TimeUnit::Microsecond, None), - Timestamp(TimeUnit::Nanosecond, None), - Time64(TimeUnit::Microsecond), - Time64(TimeUnit::Nanosecond), - Date32, - Time32(TimeUnit::Second), - Time32(TimeUnit::Millisecond), - Date64, - Utf8, - LargeUtf8, - Binary, - LargeBinary, - Duration(TimeUnit::Second), - Duration(TimeUnit::Millisecond), - Duration(TimeUnit::Microsecond), - Duration(TimeUnit::Nanosecond), - ]; - - datatypes.into_iter().for_each(|d1| { - let array = new_null_array(d1.clone(), 10); - if can_upper(&d1) { - assert!(upper(array.as_ref()).is_ok()); - } else { - assert!(upper(array.as_ref()).is_err()); - } - }); -} diff --git a/tests/it/compute/utf8.rs b/tests/it/compute/utf8.rs new file mode 100644 index 00000000000..864dc0eca27 --- /dev/null +++ b/tests/it/compute/utf8.rs @@ -0,0 +1,371 @@ +use arrow2::{array::*, compute::utf8::*, error::Result}; + +fn with_nulls_utf8_lower() -> Result<()> { + let cases = vec![ + // identity + ( + vec![Some("hello"), None, Some("world")], + vec![Some("hello"), None, Some("world")], + ), + // part of input + ( + vec![Some("Hello"), None, Some("wOrld")], + vec![Some("hello"), None, Some("world")], + ), + // all input + ( + vec![Some("HELLO"), None, Some("WORLD")], + vec![Some("hello"), None, Some("world")], + ), + // UTF8 characters + ( + vec![ + None, + Some("السلام عليكم"), + Some("Dobrý den"), + Some("שָׁלוֹם"), + Some("नमस्ते"), + Some("こんにちは"), + Some("안녕하세요"), + Some("你好"), + Some("Olá"), + Some("Здравствуйте"), + Some("Hola"), + ], + vec![ + None, + Some("السلام عليكم"), + Some("dobrý den"), + Some("שָׁלוֹם"), + Some("नमस्ते"), + Some("こんにちは"), + Some("안녕하세요"), + Some("你好"), + Some("olá"), + Some("здравствуйте"), + Some("hola"), + ], + ), + ]; + + cases + .into_iter() + .try_for_each::<_, Result<()>>(|(array, expected)| { + let array = Utf8Array::::from(&array); + let result = lower(&array)?; + assert_eq!(array.len(), result.len()); + + let result = result.as_any().downcast_ref::>().unwrap(); + let expected = Utf8Array::::from(&expected); + + assert_eq!(&expected, result); + Ok(()) + })?; + + Ok(()) +} + +#[test] +fn test_lower() -> Result<()> { + with_nulls_utf8_lower::() +} + +#[test] +fn test_large_lower() -> Result<()> { + with_nulls_utf8_lower::() +} + +fn without_nulls_utf8_lower() -> Result<()> { + let cases = vec![ + // identity + (vec!["hello", "world"], vec!["hello", "world"]), + // part of input + (vec!["Hello", "wOrld"], vec!["hello", "world"]), + // all input + (vec!["HELLO", "WORLD"], vec!["hello", "world"]), + // UTF8 characters + ( + vec![ + "السلام عليكم", + "Dobrý den", + "שָׁלוֹם", + "नमस्ते", + "こんにちは", + "안녕하세요", + "你好", + "Olá", + "Здравствуйте", + "Hola", + ], + vec![ + "السلام عليكم", + "dobrý den", + "שָׁלוֹם", + "नमस्ते", + "こんにちは", + "안녕하세요", + "你好", + "olá", + "здравствуйте", + "hola", + ], + ), + ]; + + cases + .into_iter() + .try_for_each::<_, Result<()>>(|(array, expected)| { + let array = Utf8Array::::from_slice(&array); + let result = lower(&array)?; + assert_eq!(array.len(), result.len()); + + let result = result.as_any().downcast_ref::>().unwrap(); + let expected = Utf8Array::::from_slice(&expected); + assert_eq!(&expected, result); + Ok(()) + })?; + + Ok(()) +} + +#[test] +fn without_nulls_string_lower() -> Result<()> { + without_nulls_utf8_lower::() +} + +#[test] +fn without_nulls_large_string_lower() -> Result<()> { + without_nulls_utf8_lower::() +} + +#[test] +fn consistency_lower() { + use arrow2::datatypes::DataType::*; + use arrow2::datatypes::TimeUnit; + let datatypes = vec![ + Null, + Boolean, + UInt8, + UInt16, + UInt32, + UInt64, + Int8, + Int16, + Int32, + Int64, + Float32, + Float64, + Timestamp(TimeUnit::Second, None), + Timestamp(TimeUnit::Millisecond, None), + Timestamp(TimeUnit::Microsecond, None), + Timestamp(TimeUnit::Nanosecond, None), + Time64(TimeUnit::Microsecond), + Time64(TimeUnit::Nanosecond), + Date32, + Time32(TimeUnit::Second), + Time32(TimeUnit::Millisecond), + Date64, + Utf8, + LargeUtf8, + Binary, + LargeBinary, + Duration(TimeUnit::Second), + Duration(TimeUnit::Millisecond), + Duration(TimeUnit::Microsecond), + Duration(TimeUnit::Nanosecond), + ]; + + datatypes.into_iter().for_each(|d1| { + let array = new_null_array(d1.clone(), 10); + if can_lower(&d1) { + assert!(lower(array.as_ref()).is_ok()); + } else { + assert!(lower(array.as_ref()).is_err()); + } + }); +} + +fn with_nulls_utf8() -> Result<()> { + let cases = vec![ + // identity + ( + vec![Some("hello"), None, Some("world")], + vec![Some("HELLO"), None, Some("WORLD")], + ), + // part of input + ( + vec![Some("Hello"), None, Some("wOrld")], + vec![Some("HELLO"), None, Some("WORLD")], + ), + // all input + ( + vec![Some("hello"), None, Some("world")], + vec![Some("HELLO"), None, Some("WORLD")], + ), + // UTF8 characters + ( + vec![ + None, + Some("السلام عليكم"), + Some("Dobrý den"), + Some("שָׁלוֹם"), + Some("नमस्ते"), + Some("こんにちは"), + Some("안녕하세요"), + Some("你好"), + Some("Olá"), + Some("Здравствуйте"), + Some("Hola"), + ], + vec![ + None, + Some("السلام عليكم"), + Some("DOBRÝ DEN"), + Some("שָׁלוֹם"), + Some("नमस्ते"), + Some("こんにちは"), + Some("안녕하세요"), + Some("你好"), + Some("OLÁ"), + Some("ЗДРАВСТВУЙТЕ"), + Some("HOLA"), + ], + ), + ]; + + cases + .into_iter() + .try_for_each::<_, Result<()>>(|(array, expected)| { + let array = Utf8Array::::from(&array); + let result = upper(&array)?; + assert_eq!(array.len(), result.len()); + + let result = result.as_any().downcast_ref::>().unwrap(); + let expected = Utf8Array::::from(&expected); + + assert_eq!(&expected, result); + Ok(()) + })?; + + Ok(()) +} + +#[test] +fn with_nulls_string() -> Result<()> { + with_nulls_utf8::() +} + +#[test] +fn with_nulls_large_string() -> Result<()> { + with_nulls_utf8::() +} + +fn without_nulls_utf8() -> Result<()> { + let cases = vec![ + // identity + (vec!["hello", "world"], vec!["HELLO", "WORLD"]), + // part of input + (vec!["Hello", "wOrld"], vec!["HELLO", "WORLD"]), + // all input + (vec!["HELLO", "WORLD"], vec!["HELLO", "WORLD"]), + // UTF8 characters + ( + vec![ + "السلام عليكم", + "Dobrý den", + "שָׁלוֹם", + "नमस्ते", + "こんにちは", + "안녕하세요", + "你好", + "Olá", + "Здравствуйте", + "Hola", + ], + vec![ + "السلام عليكم", + "DOBRÝ DEN", + "שָׁלוֹם", + "नमस्ते", + "こんにちは", + "안녕하세요", + "你好", + "OLÁ", + "ЗДРАВСТВУЙТЕ", + "HOLA", + ], + ), + ]; + + cases + .into_iter() + .try_for_each::<_, Result<()>>(|(array, expected)| { + let array = Utf8Array::::from_slice(&array); + let result = upper(&array)?; + assert_eq!(array.len(), result.len()); + + let result = result.as_any().downcast_ref::>().unwrap(); + let expected = Utf8Array::::from_slice(&expected); + assert_eq!(&expected, result); + Ok(()) + })?; + + Ok(()) +} + +#[test] +fn without_nulls_string() -> Result<()> { + without_nulls_utf8::() +} + +#[test] +fn without_nulls_large_string() -> Result<()> { + without_nulls_utf8::() +} + +#[test] +fn consistency_upper() { + use arrow2::datatypes::DataType::*; + use arrow2::datatypes::TimeUnit; + let datatypes = vec![ + Null, + Boolean, + UInt8, + UInt16, + UInt32, + UInt64, + Int8, + Int16, + Int32, + Int64, + Float32, + Float64, + Timestamp(TimeUnit::Second, None), + Timestamp(TimeUnit::Millisecond, None), + Timestamp(TimeUnit::Microsecond, None), + Timestamp(TimeUnit::Nanosecond, None), + Time64(TimeUnit::Microsecond), + Time64(TimeUnit::Nanosecond), + Date32, + Time32(TimeUnit::Second), + Time32(TimeUnit::Millisecond), + Date64, + Utf8, + LargeUtf8, + Binary, + LargeBinary, + Duration(TimeUnit::Second), + Duration(TimeUnit::Millisecond), + Duration(TimeUnit::Microsecond), + Duration(TimeUnit::Nanosecond), + ]; + + datatypes.into_iter().for_each(|d1| { + let array = new_null_array(d1.clone(), 10); + if can_upper(&d1) { + assert!(upper(array.as_ref()).is_ok()); + } else { + assert!(upper(array.as_ref()).is_err()); + } + }); +} diff --git a/tests/it/ffi.rs b/tests/it/ffi/data.rs similarity index 94% rename from tests/it/ffi.rs rename to tests/it/ffi/data.rs index dfc5b5102cb..45c2f66cc1a 100644 --- a/tests/it/ffi.rs +++ b/tests/it/ffi/data.rs @@ -8,8 +8,8 @@ use std::sync::Arc; fn _test_round_trip(array: Arc, expected: Box) -> Result<()> { let field = Field::new("a", array.data_type().clone(), true); - let array_ptr = Box::new(ffi::Ffi_ArrowArray::empty()); - let schema_ptr = Box::new(ffi::Ffi_ArrowSchema::empty()); + let array_ptr = Box::new(ffi::ArrowArray::empty()); + let schema_ptr = Box::new(ffi::ArrowSchema::empty()); let array_ptr = Box::into_raw(array_ptr); let schema_ptr = Box::into_raw(schema_ptr); @@ -24,7 +24,8 @@ fn _test_round_trip(array: Arc, expected: Box) -> Result<( // import references let result_field = unsafe { ffi::import_field_from_c(schema_ptr.as_ref())? }; - let result_array = unsafe { ffi::import_array_from_c(array_ptr, &result_field)? }; + let result_array = + unsafe { ffi::import_array_from_c(array_ptr, result_field.data_type.clone())? }; assert_eq!(&result_array, &expected); assert_eq!(result_field, field); @@ -41,8 +42,8 @@ fn test_round_trip(expected: impl Array + Clone + 'static) -> Result<()> { } fn test_round_trip_schema(field: Field) -> Result<()> { - // create a `ArrowArray` from the data. - let schema_ptr = Box::new(ffi::Ffi_ArrowSchema::empty()); + // create a `InternalArrowArray` from the data. + let schema_ptr = Box::new(ffi::ArrowSchema::empty()); let schema_ptr = Box::into_raw(schema_ptr); diff --git a/tests/it/ffi/mod.rs b/tests/it/ffi/mod.rs new file mode 100644 index 00000000000..af49381f138 --- /dev/null +++ b/tests/it/ffi/mod.rs @@ -0,0 +1,2 @@ +mod data; +mod stream; diff --git a/tests/it/ffi/stream.rs b/tests/it/ffi/stream.rs new file mode 100644 index 00000000000..9d99c77e08b --- /dev/null +++ b/tests/it/ffi/stream.rs @@ -0,0 +1,33 @@ +use std::sync::Arc; + +use arrow2::array::*; +use arrow2::datatypes::Field; +use arrow2::{error::Result, ffi}; + +fn _test_round_trip(arrays: Vec>) -> Result<()> { + let field = Field::new("a", arrays[0].data_type().clone(), true); + let iter = Box::new(arrays.clone().into_iter().map(Ok)) as _; + + let mut stream = Box::new(ffi::ArrowArrayStream::empty()); + + unsafe { ffi::export_iterator(iter, field.clone(), &mut *stream) } + + let mut stream = unsafe { ffi::ArrowArrayStreamReader::try_new(stream)? }; + + let mut produced_arrays: Vec> = vec![]; + while let Some(array) = unsafe { stream.next() } { + produced_arrays.push(array?.into()); + } + + assert_eq!(produced_arrays, arrays); + assert_eq!(stream.field(), &field); + Ok(()) +} + +#[test] +fn round_trip() -> Result<()> { + let array = Int32Array::from(&[Some(2), None, Some(1), None]); + let array: Arc = Arc::new(array); + + _test_round_trip(vec![array.clone(), array.clone(), array]) +} diff --git a/tests/it/io/csv/write.rs b/tests/it/io/csv/write.rs index 656a04f7839..6d8e27b9660 100644 --- a/tests/it/io/csv/write.rs +++ b/tests/it/io/csv/write.rs @@ -34,15 +34,18 @@ fn data() -> Chunk> { fn write_csv() -> Result<()> { let columns = data(); - let write = Cursor::new(Vec::::new()); - let mut writer = WriterBuilder::new().from_writer(write); - - write_header(&mut writer, &["c1", "c2", "c3", "c4", "c5", "c6", "c7"])?; + let mut writer = Cursor::new(Vec::::new()); let options = SerializeOptions::default(); + + write_header( + &mut writer, + &["c1", "c2", "c3", "c4", "c5", "c6", "c7"], + &options, + )?; write_chunk(&mut writer, &columns, &options)?; // check - let buffer = writer.into_inner().unwrap().into_inner(); + let buffer = writer.into_inner(); assert_eq!( r#"c1,c2,c3,c4,c5,c6,c7 a b,123.564532,3,true,,00:20:34,d @@ -59,18 +62,18 @@ d,-556132.25,1,,2019-04-18 02:45:55.555,23:46:03,c fn write_csv_custom_options() -> Result<()> { let batch = data(); - let write = Cursor::new(Vec::::new()); - let mut writer = WriterBuilder::new().delimiter(b'|').from_writer(write); + let mut writer = Cursor::new(Vec::::new()); let options = SerializeOptions { time32_format: Some("%r".to_string()), time64_format: Some("%r".to_string()), + delimiter: b'|', ..Default::default() }; write_chunk(&mut writer, &batch, &options)?; // check - let buffer = writer.into_inner().unwrap().into_inner(); + let buffer = writer.into_inner(); assert_eq!( r#"a b|123.564532|3|true||12:20:34 AM|d c||2|false|2019-04-18 10:54:47.378|06:51:20 AM|a b @@ -230,14 +233,13 @@ fn test_array( data: Vec<&'static str>, options: SerializeOptions, ) -> Result<()> { - let write = Cursor::new(Vec::::new()); - let mut writer = WriterBuilder::new().delimiter(b'|').from_writer(write); + let mut writer = Cursor::new(Vec::::new()); - write_header(&mut writer, &["c1"])?; + write_header(&mut writer, &["c1"], &options)?; write_chunk(&mut writer, &columns, &options)?; // check - let buffer = writer.into_inner().unwrap().into_inner(); + let buffer = writer.into_inner(); let mut expected = "c1\n".to_owned(); expected.push_str(&data.join("\n")); @@ -314,3 +316,20 @@ fn write_tz_timezone_formatted_tz() -> Result<()> { }, ) } + +#[test] +fn write_empty_and_missing() { + let a = Utf8Array::::from(&[Some(""), None]); + let b = Utf8Array::::from(&[None, Some("")]); + let columns = Chunk::new(vec![ + Arc::new(a) as Arc, + Arc::new(b) as Arc, + ]); + + let mut writer = vec![]; + let options = SerializeOptions::default(); + write_chunk(&mut writer, &columns, &options).unwrap(); + let csv = std::str::from_utf8(&writer).unwrap(); + + assert_eq!(csv, "\"\",\n,\"\"\n"); +} diff --git a/tests/it/io/json/mod.rs b/tests/it/io/json/mod.rs index 03b61c3b5ad..785789ae0e3 100644 --- a/tests/it/io/json/mod.rs +++ b/tests/it/io/json/mod.rs @@ -1,310 +1,16 @@ mod read; mod write; -use std::io::Cursor; use std::sync::Arc; use arrow2::array::*; -use arrow2::bitmap::Bitmap; -use arrow2::buffer::Buffer; -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; -fn read_batch(data: String, fields: &[Field]) -> Result>> { - let mut reader = Cursor::new(data); +fn write_batch(array: Box) -> Result> { + let mut serializer = json_write::Serializer::new(vec![Ok(array)].into_iter(), vec![]); - let mut rows = vec![String::default(); 1024]; - let read = json_read::read_rows(&mut reader, &mut rows)?; - let rows = &rows[..read]; - json_read::deserialize(rows, fields) -} - -fn write_batch>( - batch: Chunk, - names: Vec, - format: F, -) -> Result> { - let batches = vec![Ok(batch)].into_iter(); - - let blocks = json_write::Serializer::new(batches, names, vec![], format); - - let mut buf = Vec::new(); - json_write::write(&mut buf, format, blocks)?; + let mut buf = vec![]; + json_write::write(&mut buf, &mut serializer)?; Ok(buf) } - -fn round_trip(data: String) -> Result<()> { - let mut reader = Cursor::new(data); - let fields = json_read::infer(&mut reader, None)?; - let data = reader.into_inner(); - - let columns = read_batch(data, &fields)?; - - let buf = write_batch( - columns.clone(), - fields.iter().map(|x| x.name.clone()).collect(), - json_write::LineDelimited::default(), - )?; - - let new_chunk = read_batch(String::from_utf8(buf).unwrap(), &fields)?; - - assert_eq!(columns, new_chunk); - Ok(()) -} - -#[test] -fn round_trip_basics() -> Result<()> { - let (data, _, _) = case_basics(); - round_trip(data) -} - -#[test] -fn round_trip_list() -> Result<()> { - let (data, _, _) = case_list(); - round_trip(data) -} - -fn case_list() -> (String, Vec, Vec>) { - let data = r#"{"a":1, "b":[2.0, 1.3, -6.1], "c":[false, true], "d":"4"} - {"a":-10, "b":null, "c":[true, true]} - {"a":null, "b":[2.1, null, -6.2], "c":[false, null], "d":"text"} - "# - .to_string(); - - let fields = vec![ - Field::new("a", DataType::Int64, true), - Field::new( - "b", - DataType::List(Box::new(Field::new("item", DataType::Float64, true))), - true, - ), - Field::new( - "c", - DataType::List(Box::new(Field::new("item", DataType::Boolean, true))), - true, - ), - Field::new("d", DataType::Utf8, true), - ]; - - let a = Int64Array::from(&[Some(1), Some(-10), None]); - let mut b = MutableListArray::>::new(); - b.try_extend(vec![ - Some(vec![Some(2.0), Some(1.3), Some(-6.1)]), - None, - Some(vec![Some(2.1), None, Some(-6.2)]), - ]) - .unwrap(); - let b: ListArray = b.into(); - - let mut c = MutableListArray::::new(); - c.try_extend(vec![ - Some(vec![Some(false), Some(true)]), - Some(vec![Some(true), Some(true)]), - Some(vec![Some(false), None]), - ]) - .unwrap(); - let c: ListArray = c.into(); - - let d = Utf8Array::::from(&[Some("4"), None, Some("text")]); - - let columns = vec![ - Box::new(a) as Box, - Box::new(b), - Box::new(c), - Box::new(d), - ]; - - (data, fields, columns) -} - -fn case_dict() -> (String, Vec, Vec>) { - let data = r#"{"machine": "a", "events": [null, "Elect Leader", "Do Ballot"]} - {"machine": "b", "events": ["Do Ballot", null, "Send Data", "Elect Leader"]} - {"machine": "c", "events": ["Send Data"]} - {"machine": "c"} - {"machine": "c", "events": null} - "# - .to_string(); - - let data_type = DataType::List(Box::new(Field::new( - "item", - DataType::Dictionary(u64::KEY_TYPE, Box::new(DataType::Utf8), false), - true, - ))); - - let fields = vec![Field::new("events", data_type, true)]; - - type A = MutableDictionaryArray>; - - let mut array = MutableListArray::::new(); - array - .try_extend(vec![ - Some(vec![None, Some("Elect Leader"), Some("Do Ballot")]), - Some(vec![ - Some("Do Ballot"), - None, - Some("Send Data"), - Some("Elect Leader"), - ]), - Some(vec![Some("Send Data")]), - None, - None, - ]) - .unwrap(); - - let array: ListArray = array.into(); - - (data, fields, vec![Box::new(array) as Box]) -} - -fn case_basics() -> (String, Vec, Vec>) { - let data = r#"{"a":1, "b":2.0, "c":false, "d":"4"} - {"a":-10, "b":-3.5, "c":true, "d":null} - {"a":100000000, "b":0.6, "d":"text"}"# - .to_string(); - let fields = vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Float64, true), - Field::new("c", DataType::Boolean, true), - Field::new("d", DataType::Utf8, true), - ]; - let columns = vec![ - Box::new(Int64Array::from_slice(&[1, -10, 100000000])) as Box, - Box::new(Float64Array::from_slice(&[2.0, -3.5, 0.6])), - Box::new(BooleanArray::from(&[Some(false), Some(true), None])), - Box::new(Utf8Array::::from(&[Some("4"), None, Some("text")])), - ]; - (data, fields, columns) -} - -fn case_projection() -> (String, Vec, Vec>) { - let data = r#"{"a":1, "b":2.0, "c":false, "d":"4", "e":"4"} - {"a":10, "b":-3.5, "c":true, "d":null, "e":"text"} - {"a":100000000, "b":0.6, "d":"text"}"# - .to_string(); - let fields = vec![ - Field::new("a", DataType::UInt32, true), - Field::new("b", DataType::Float32, true), - Field::new("c", DataType::Boolean, true), - // note how "d" is not here - Field::new("e", DataType::Binary, true), - ]; - let columns = vec![ - Box::new(UInt32Array::from_slice(&[1, 10, 100000000])) as Box, - Box::new(Float32Array::from_slice(&[2.0, -3.5, 0.6])), - Box::new(BooleanArray::from(&[Some(false), Some(true), None])), - Box::new(BinaryArray::::from(&[ - Some(b"4".as_ref()), - Some(b"text".as_ref()), - None, - ])), - ]; - (data, fields, columns) -} - -fn case_struct() -> (String, Vec, Vec>) { - let data = r#"{"a": {"b": true, "c": {"d": "text"}}} - {"a": {"b": false, "c": null}} - {"a": {"b": true, "c": {"d": "text"}}} - {"a": 1}"# - .to_string(); - - let d_field = Field::new("d", DataType::Utf8, true); - let c_field = Field::new("c", DataType::Struct(vec![d_field.clone()]), true); - let a_field = Field::new( - "a", - DataType::Struct(vec![ - Field::new("b", DataType::Boolean, true), - c_field.clone(), - ]), - true, - ); - let fields = vec![a_field]; - - // build expected output - let d = Utf8Array::::from(&vec![Some("text"), None, Some("text"), None]); - let c = StructArray::from_data(DataType::Struct(vec![d_field]), vec![Arc::new(d)], None); - - let b = BooleanArray::from(vec![Some(true), Some(false), Some(true), None]); - let expected = StructArray::from_data( - DataType::Struct(vec![Field::new("b", DataType::Boolean, true), c_field]), - vec![Arc::new(b), Arc::new(c)], - None, - ); - - (data, fields, vec![Box::new(expected) as Box]) -} - -fn case_nested_list() -> (String, Vec, Vec>) { - let d_field = Field::new("d", DataType::Utf8, true); - let c_field = Field::new("c", DataType::Struct(vec![d_field.clone()]), true); - let b_field = Field::new("b", DataType::Boolean, true); - let a_struct_field = Field::new( - "a", - DataType::Struct(vec![b_field.clone(), c_field.clone()]), - true, - ); - 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 data = r#" - {"a": [{"b": true, "c": {"d": "a_text"}}, {"b": false, "c": {"d": "b_text"}}]} - {"a": [{"b": false, "c": null}]} - {"a": [{"b": true, "c": {"d": "c_text"}}, {"b": null, "c": {"d": "d_text"}}, {"b": true, "c": {"d": null}}]} - {"a": null} - {"a": []} - "#.to_string(); - - // build expected output - let d = Utf8Array::::from(&vec![ - Some("a_text"), - Some("b_text"), - None, - Some("c_text"), - Some("d_text"), - None, - ]); - - let c = StructArray::from_data(DataType::Struct(vec![d_field]), vec![Arc::new(d)], None); - - let b = BooleanArray::from(vec![ - Some(true), - Some(false), - Some(false), - Some(true), - None, - Some(true), - ]); - let a_struct = StructArray::from_data( - DataType::Struct(vec![b_field, c_field]), - vec![Arc::new(b) as Arc, Arc::new(c) as Arc], - None, - ); - let expected = ListArray::from_data( - a_list_data_type, - Buffer::from_slice([0i32, 2, 3, 6, 6, 6]), - Arc::new(a_struct) as Arc, - Some(Bitmap::from_u8_slice([0b00010111], 5)), - ); - - ( - data, - vec![a_field], - vec![Box::new(expected) as Box], - ) -} - -fn case(case: &str) -> (String, Vec, Vec>) { - match case { - "basics" => case_basics(), - "projection" => case_projection(), - "list" => case_list(), - "dict" => case_dict(), - "struct" => case_struct(), - "nested_list" => case_nested_list(), - _ => todo!(), - } -} diff --git a/tests/it/io/json/read.rs b/tests/it/io/json/read.rs index 66caf83a302..db1b067bb42 100644 --- a/tests/it/io/json/read.rs +++ b/tests/it/io/json/read.rs @@ -1,178 +1,10 @@ -use std::io::Cursor; - use arrow2::array::*; use arrow2::datatypes::*; -use arrow2::error::ArrowError; use arrow2::error::Result; use arrow2::io::json::read; use super::*; -fn test_case(case_: &str) -> Result<()> { - let (data, fields, columns) = case(case_); - - let batch = read_batch(data, &fields)?; - - columns - .iter() - .zip(batch.columns()) - .for_each(|(expected, result)| assert_eq!(expected.as_ref(), result.as_ref())); - Ok(()) -} - -#[test] -fn basic() -> Result<()> { - test_case("basics") -} - -#[test] -fn projection() -> Result<()> { - test_case("projection") -} - -#[test] -fn dictionary() -> Result<()> { - test_case("dict") -} - -#[test] -fn list() -> Result<()> { - test_case("list") -} - -#[test] -fn nested_struct() -> Result<()> { - test_case("struct") -} - -#[test] -fn nested_list() -> Result<()> { - test_case("nested_list") -} - -#[test] -fn line_break_in_values() -> Result<()> { - let data = r#" - {"a":"aa\n\n"} - {"a":"aa\n"} - {"a":null} - "#; - - 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]); - - assert_eq!(expected, batch.columns()[0].as_ref()); - Ok(()) -} - -#[test] -fn invalid_infer_schema() -> Result<()> { - let re = read::infer(&mut Cursor::new("city,lat,lng"), None); - assert_eq!( - re.err().unwrap().to_string(), - "External error: expected value at line 1 column 1", - ); - Ok(()) -} - -#[test] -fn invalid_read_record() -> Result<()> { - let fields = vec![Field::new( - "a", - DataType::Struct(vec![Field::new("a", DataType::Utf8, true)]), - true, - )]; - let batch = read_batch("city,lat,lng".to_string(), &fields); - - assert_eq!( - batch.err().unwrap().to_string(), - "External error: expected value at line 1 column 1", - ); - Ok(()) -} - -#[test] -fn skip_empty_lines() { - let data = " - {\"a\": 1} - - {\"a\": 2} - - {\"a\": 3}"; - - let batch = read_batch(data.to_string(), &[Field::new("a", DataType::Int64, true)]).unwrap(); - - assert_eq!(1, batch.arrays().len()); - assert_eq!(3, batch.len()); -} - -#[test] -fn row_type_validation() { - let data = " - [1, \"hello\"] - \"world\""; - - let batch = read::infer(&mut Cursor::new(data.to_string()), None); - assert_eq!( - batch.err().unwrap().to_string(), - r#"External format error: Expected JSON record to be an object, found Array([Number(1), String("hello")])"#, - ); -} - -#[test] -fn infer_schema_mixed_list() -> Result<()> { - let data = r#"{"a":1, "b":[2.0, 1.3, -6.1], "c":[false, true], "d":4.1} - {"a":-10, "b":[2.0, 1.3, -6.1], "c":null, "d":null} - {"a":2, "b":[2.0, null, -6.1], "c":[false, null], "d":"text"} - {"a":3, "b":4, "c": true, "d":[1, false, "array", 2.4]} - "#; - - let fields = vec![ - Field::new("a", DataType::Int64, true), - Field::new( - "b", - DataType::List(Box::new(Field::new("item", DataType::Float64, true))), - true, - ), - Field::new( - "c", - DataType::List(Box::new(Field::new("item", DataType::Boolean, true))), - true, - ), - Field::new("d", DataType::Utf8, true), - ]; - - let result = read::infer(&mut Cursor::new(data), None)?; - - assert_eq!(result, fields); - Ok(()) -} - -#[test] -fn infer_nested_struct() -> Result<()> { - let data = r#"{"a": {"a": 2.0, "b": 2}} - {"a": {"b": 2}} - {"a": {"a": 2.0, "b": 2, "c": true}} - {"a": {"a": 2.0, "b": 2}} - "#; - - let fields = vec![Field::new( - "a", - DataType::Struct(vec![ - Field::new("a", DataType::Float64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Boolean, true), - ]), - true, - )]; - - let result = read::infer(&mut Cursor::new(data), None)?; - - assert_eq!(result, fields); - Ok(()) -} - #[test] fn read_json() -> Result<()> { let data = r#"[ @@ -187,17 +19,11 @@ fn read_json() -> Result<()> { } ]"#; - let data = serde_json::from_slice(data.as_bytes())?; - - let values = if let serde_json::Value::Array(values) = data { - Ok(values) - } else { - Err(ArrowError::InvalidArgumentError("".to_string())) - }?; + let json = serde_json::from_slice(data.as_bytes())?; - let data_type = read::infer_rows(&values)?; + let data_type = read::infer(&json)?; - let result = read::deserialize_json(&values, data_type); + let result = read::deserialize(&json, data_type)?; let expected = StructArray::from_data( DataType::Struct(vec![Field::new("a", DataType::Int64, true)]), diff --git a/tests/it/io/json/write.rs b/tests/it/io/json/write.rs index 6145fa0ddbc..69e4bfa9d94 100644 --- a/tests/it/io/json/write.rs +++ b/tests/it/io/json/write.rs @@ -4,59 +4,57 @@ use arrow2::{ array::*, bitmap::Bitmap, buffer::Buffer, - datatypes::{DataType, Field}, + datatypes::{DataType, Field, TimeUnit}, error::Result, }; use super::*; +macro_rules! test { + ($array:expr, $expected:expr) => {{ + let buf = write_batch(Box::new($array))?; + assert_eq!(String::from_utf8(buf).unwrap(), $expected); + Ok(()) + }}; +} + #[test] -fn write_simple_rows() -> Result<()> { - 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 = Chunk::try_new(vec![&a as &dyn Array, &b]).unwrap(); - - let buf = write_batch( - batch, - vec!["c1".to_string(), "c2".to_string()], - json_write::LineDelimited::default(), - )?; - - assert_eq!( - String::from_utf8(buf).unwrap(), - r#"{"c1":1,"c2":"a"} -{"c1":2,"c2":"b"} -{"c1":3,"c2":"c"} -{"c1":null,"c2":"d"} -{"c1":5,"c2":null} -"# - ); - Ok(()) +fn int32() -> Result<()> { + let array = 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 expected = r#"[1,2,3,null,5]"#; + + test!(array, expected) } #[test] -fn write_simple_rows_array() -> Result<()> { - 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]); +fn utf8() -> Result<()> { + let array = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c"), Some("d"), None]); - let batch = Chunk::try_new(vec![&a as &dyn Array, &b]).unwrap(); + let expected = r#"["a","b","c","d",null]"#; - let buf = write_batch( - batch, - vec!["c1".to_string(), "c2".to_string()], - json_write::JsonArray::default(), - )?; + test!(array, expected) +} - assert_eq!( - String::from_utf8(buf).unwrap(), - r#"[{"c1":1,"c2":"a"},{"c1":2,"c2":"b"},{"c1":3,"c2":"c"},{"c1":null,"c2":"d"},{"c1":5,"c2":null}]"# - ); - Ok(()) +#[test] +fn struct_() -> Result<()> { + let c1 = Int32Array::from([Some(1), Some(2), Some(3), None, Some(5)]); + let c2 = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c"), Some("d"), None]); + + let data_type = DataType::Struct(vec![ + Field::new("c1", c1.data_type().clone(), true), + Field::new("c2", c2.data_type().clone(), true), + ]); + let array = StructArray::from_data(data_type, vec![Arc::new(c1) as _, Arc::new(c2)], None); + + let expected = r#"[{"c1":1,"c2":"a"},{"c1":2,"c2":"b"},{"c1":3,"c2":"c"},{"c1":null,"c2":"d"},{"c1":5,"c2":null}]"#; + + test!(array, expected) } #[test] -fn write_nested_struct_with_validity() -> Result<()> { +fn nested_struct_with_validity() -> Result<()> { let inner = vec![ Field::new("c121", DataType::Utf8, false), Field::new("c122", DataType::Int32, false), @@ -83,26 +81,19 @@ fn write_nested_struct_with_validity() -> Result<()> { ); let c2 = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c")]); - let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); + let data_type = DataType::Struct(vec![ + Field::new("c1", c1.data_type().clone(), true), + Field::new("c2", c2.data_type().clone(), true), + ]); + let array = StructArray::from_data(data_type, vec![Arc::new(c1) as _, Arc::new(c2)], None); - let buf = write_batch( - batch, - vec!["c1".to_string(), "c2".to_string()], - json_write::LineDelimited::default(), - )?; + let expected = r#"[{"c1":{"c11":1,"c12":null},"c2":"a"},{"c1":{"c11":null,"c12":{"c121":"f","c122":null}},"c2":"b"},{"c1":null,"c2":"c"}]"#; - assert_eq!( - String::from_utf8(buf).unwrap(), - r#"{"c1":{"c11":1,"c12":null},"c2":"a"} -{"c1":{"c11":null,"c12":{"c121":"f","c122":null}},"c2":"b"} -{"c1":null,"c2":"c"} -"# - ); - Ok(()) + test!(array, expected) } #[test] -fn write_nested_structs() -> Result<()> { +fn nested_struct() -> Result<()> { let c121 = Field::new("c121", DataType::Utf8, false); let fields = vec![ Field::new("c11", DataType::Int32, false), @@ -128,26 +119,19 @@ fn write_nested_structs() -> Result<()> { let c2 = Utf8Array::::from(&vec![Some("a"), Some("b"), Some("c")]); - let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); + let data_type = DataType::Struct(vec![ + Field::new("c1", c1.data_type().clone(), true), + Field::new("c2", c2.data_type().clone(), true), + ]); + let array = StructArray::from_data(data_type, vec![Arc::new(c1) as _, Arc::new(c2)], None); - let buf = write_batch( - batch, - vec!["c1".to_string(), "c2".to_string()], - json_write::LineDelimited::default(), - )?; + let expected = r#"[{"c1":{"c11":1,"c12":{"c121":"e"}},"c2":"a"},{"c1":{"c11":null,"c12":{"c121":"f"}},"c2":"b"},{"c1":{"c11":5,"c12":{"c121":"g"}},"c2":"c"}]"#; - assert_eq!( - String::from_utf8(buf).unwrap(), - r#"{"c1":{"c11":1,"c12":{"c121":"e"}},"c2":"a"} -{"c1":{"c11":null,"c12":{"c121":"f"}},"c2":"b"} -{"c1":{"c11":5,"c12":{"c121":"g"}},"c2":"c"} -"# - ); - Ok(()) + test!(array, expected) } #[test] -fn write_struct_with_list_field() -> Result<()> { +fn struct_with_list_field() -> Result<()> { let iter = vec![vec!["a", "a1"], vec!["b"], vec!["c"], vec!["d"], vec!["e"]]; let iter = iter @@ -160,32 +144,23 @@ fn write_struct_with_list_field() -> Result<()> { false, ); a.try_extend(iter).unwrap(); - let a: ListArray = a.into(); - - let b = PrimitiveArray::from_slice([1, 2, 3, 4, 5]); - - let batch = Chunk::try_new(vec![&a as &dyn Array, &b]).unwrap(); - - let buf = write_batch( - batch, - vec!["c1".to_string(), "c2".to_string()], - json_write::LineDelimited::default(), - )?; - - assert_eq!( - String::from_utf8(buf).unwrap(), - r#"{"c1":["a","a1"],"c2":1} -{"c1":["b"],"c2":2} -{"c1":["c"],"c2":3} -{"c1":["d"],"c2":4} -{"c1":["e"],"c2":5} -"# - ); - Ok(()) + let c1: ListArray = a.into(); + + let c2 = PrimitiveArray::from_slice([1, 2, 3, 4, 5]); + + let data_type = DataType::Struct(vec![ + Field::new("c1", c1.data_type().clone(), true), + Field::new("c2", c2.data_type().clone(), true), + ]); + let array = StructArray::from_data(data_type, vec![Arc::new(c1) as _, Arc::new(c2)], None); + + let expected = r#"[{"c1":["a","a1"],"c2":1},{"c1":["b"],"c2":2},{"c1":["c"],"c2":3},{"c1":["d"],"c2":4},{"c1":["e"],"c2":5}]"#; + + test!(array, expected) } #[test] -fn write_nested_list() -> Result<()> { +fn nested_list() -> Result<()> { let iter = vec![ vec![Some(vec![Some(1), Some(2)]), Some(vec![Some(3)])], vec![], @@ -208,26 +183,20 @@ fn write_nested_list() -> Result<()> { let c2 = Utf8Array::::from(&vec![Some("foo"), Some("bar"), None]); - let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); + let data_type = DataType::Struct(vec![ + Field::new("c1", c1.data_type().clone(), true), + Field::new("c2", c2.data_type().clone(), true), + ]); + let array = StructArray::from_data(data_type, vec![Arc::new(c1) as _, Arc::new(c2)], None); - let buf = write_batch( - batch, - vec!["c1".to_string(), "c2".to_string()], - json_write::LineDelimited::default(), - )?; + let expected = + r#"[{"c1":[[1,2],[3]],"c2":"foo"},{"c1":[],"c2":"bar"},{"c1":[[4,5,6]],"c2":null}]"#; - assert_eq!( - String::from_utf8(buf).unwrap(), - r#"{"c1":[[1,2],[3]],"c2":"foo"} -{"c1":[],"c2":"bar"} -{"c1":[[4,5,6]],"c2":null} -"# - ); - Ok(()) + test!(array, expected) } #[test] -fn write_list_of_struct() -> Result<()> { +fn list_of_struct() -> Result<()> { let inner = vec![Field::new("c121", DataType::Utf8, false)]; let fields = vec![ Field::new("c11", DataType::Int32, false), @@ -269,99 +238,54 @@ fn write_list_of_struct() -> Result<()> { let c2 = Int32Array::from_slice(&[1, 2, 3]); - let batch = Chunk::try_new(vec![&c1 as &dyn Array, &c2]).unwrap(); + let data_type = DataType::Struct(vec![ + Field::new("c1", c1.data_type().clone(), true), + Field::new("c2", c2.data_type().clone(), true), + ]); + let array = StructArray::from_data(data_type, vec![Arc::new(c1) as _, Arc::new(c2)], None); - let buf = write_batch( - batch, - vec!["c1".to_string(), "c2".to_string()], - json_write::LineDelimited::default(), - )?; + let expected = r#"[{"c1":[{"c11":1,"c12":null},{"c11":null,"c12":{"c121":"f"}}],"c2":1},{"c1":null,"c2":2},{"c1":[null],"c2":3}]"#; - assert_eq!( - String::from_utf8(buf).unwrap(), - r#"{"c1":[{"c11":1,"c12":null},{"c11":null,"c12":{"c121":"f"}}],"c2":1} -{"c1":null,"c2":2} -{"c1":[null],"c2":3} -"# - ); - Ok(()) + test!(array, expected) } #[test] -fn write_escaped_utf8() -> Result<()> { - let a = Utf8Array::::from(&vec![Some("a\na"), None]); - - let batch = Chunk::try_new(vec![&a as &dyn Array]).unwrap(); +fn escaped_end_of_line_in_utf8() -> Result<()> { + let array = Utf8Array::::from(&vec![Some("a\na"), None]); - let buf = write_batch( - batch, - vec!["c1".to_string()], - json_write::LineDelimited::default(), - )?; + let expected = r#"["a\na",null]"#; - assert_eq!( - String::from_utf8(buf).unwrap().as_bytes(), - b"{\"c1\":\"a\\na\"}\n{\"c1\":null}\n" - ); - Ok(()) + test!(array, expected) } #[test] -fn write_quotation_marks_in_utf8() -> Result<()> { - let a = Utf8Array::::from(&vec![Some("a\"a"), None]); +fn escaped_quotation_marks_in_utf8() -> Result<()> { + let array = Utf8Array::::from(&vec![Some("a\"a"), None]); - let batch = Chunk::try_new(vec![&a as &dyn Array]).unwrap(); + let expected = r#"["a\"a",null]"#; - let buf = write_batch( - batch, - vec!["c1".to_string()], - json_write::LineDelimited::default(), - )?; - - assert_eq!( - String::from_utf8(buf).unwrap().as_bytes(), - b"{\"c1\":\"a\\\"a\"}\n{\"c1\":null}\n" - ); - Ok(()) + test!(array, expected) } #[test] fn write_date32() -> Result<()> { - let a = PrimitiveArray::from_data(DataType::Date32, vec![1000i32, 8000, 10000].into(), None); + let array = + PrimitiveArray::from_data(DataType::Date32, vec![1000i32, 8000, 10000].into(), None); - let batch = Chunk::try_new(vec![&a as &dyn Array]).unwrap(); + let expected = r#"["1972-09-27","1991-11-27","1997-05-19"]"#; - let buf = write_batch( - batch, - vec!["c1".to_string()], - json_write::LineDelimited::default(), - )?; - - assert_eq!( - String::from_utf8(buf).unwrap().as_bytes(), - b"{\"c1\":1972-09-27}\n{\"c1\":1991-11-27}\n{\"c1\":1997-05-19}\n" - ); - Ok(()) + test!(array, expected) } + #[test] fn write_timestamp() -> Result<()> { - let a = PrimitiveArray::from_data( + let array = PrimitiveArray::from_data( DataType::Timestamp(TimeUnit::Second, None), vec![10i64, 1 << 32, 1 << 33].into(), None, ); - let batch = Chunk::try_new(vec![&a as &dyn Array]).unwrap(); - - let buf = write_batch( - batch, - vec!["c1".to_string()], - json_write::LineDelimited::default(), - )?; + let expected = r#"["1970-01-01 00:00:10","2106-02-07 06:28:16","2242-03-16 12:56:32"]"#; - assert_eq!( - String::from_utf8(buf).unwrap().as_bytes(), - b"{\"c1\":1970-01-01 00:00:10}\n{\"c1\":2106-02-07 06:28:16}\n{\"c1\":2242-03-16 12:56:32}\n" - ); - Ok(()) + test!(array, expected) } diff --git a/tests/it/io/mod.rs b/tests/it/io/mod.rs index 75296ecf766..cfd49263a20 100644 --- a/tests/it/io/mod.rs +++ b/tests/it/io/mod.rs @@ -4,6 +4,9 @@ mod print; #[cfg(feature = "io_json")] mod json; +#[cfg(feature = "io_json")] +mod ndjson; + #[cfg(feature = "io_ipc")] mod ipc; diff --git a/tests/it/io/ndjson/mod.rs b/tests/it/io/ndjson/mod.rs new file mode 100644 index 00000000000..9f9cd0ab6b6 --- /dev/null +++ b/tests/it/io/ndjson/mod.rs @@ -0,0 +1,312 @@ +mod read; + +use std::sync::Arc; + +use arrow2::array::*; +use arrow2::bitmap::Bitmap; +use arrow2::buffer::Buffer; +use arrow2::datatypes::*; +use arrow2::error::Result; +use arrow2::io::ndjson::write as ndjson_write; + +use read::{infer, read_and_deserialize}; + +fn round_trip(ndjson: String) -> Result<()> { + let data_type = infer(&ndjson)?; + + let expected = read_and_deserialize(&ndjson, &data_type, 1000)?; + + let arrays = expected.clone().into_iter().map(Ok); + + let serializer = ndjson_write::Serializer::new(arrays, vec![]); + + let mut writer = ndjson_write::FileWriter::new(vec![], serializer); + writer.by_ref().collect::>()?; // write + let buf = writer.into_inner().0; + + let new_chunk = read_and_deserialize(std::str::from_utf8(&buf).unwrap(), &data_type, 1000)?; + + assert_eq!(expected, new_chunk); + Ok(()) +} + +#[test] +fn round_trip_basics() -> Result<()> { + let (data, _) = case_basics(); + round_trip(data) +} + +#[test] +fn round_trip_list() -> Result<()> { + let (data, _) = case_list(); + round_trip(data) +} + +fn case_list() -> (String, Arc) { + let data = r#"{"a":1, "b":[2.0, 1.3, -6.1], "c":[false, true], "d":"4"} + {"a":-10, "b":null, "c":[true, true]} + {"a":null, "b":[2.1, null, -6.2], "c":[false, null], "d":"text"} + "# + .to_string(); + + let data_type = DataType::Struct(vec![ + Field::new("a", DataType::Int64, true), + Field::new( + "b", + DataType::List(Box::new(Field::new("item", DataType::Float64, true))), + true, + ), + Field::new( + "c", + DataType::List(Box::new(Field::new("item", DataType::Boolean, true))), + true, + ), + Field::new("d", DataType::Utf8, true), + ]); + + let a = Int64Array::from(&[Some(1), Some(-10), None]); + let mut b = MutableListArray::>::new(); + b.try_extend(vec![ + Some(vec![Some(2.0), Some(1.3), Some(-6.1)]), + None, + Some(vec![Some(2.1), None, Some(-6.2)]), + ]) + .unwrap(); + let b: ListArray = b.into(); + + let mut c = MutableListArray::::new(); + c.try_extend(vec![ + Some(vec![Some(false), Some(true)]), + Some(vec![Some(true), Some(true)]), + Some(vec![Some(false), None]), + ]) + .unwrap(); + let c: ListArray = c.into(); + + let d = Utf8Array::::from(&[Some("4"), None, Some("text")]); + + let array = StructArray::from_data( + data_type, + vec![ + Arc::new(a) as Arc, + Arc::new(b), + Arc::new(c), + Arc::new(d), + ], + None, + ); + + (data, Arc::new(array)) +} + +fn case_dict() -> (String, Arc) { + let data = r#"{"machine": "a", "events": [null, "Elect Leader", "Do Ballot"]} + {"machine": "b", "events": ["Do Ballot", null, "Send Data", "Elect Leader"]} + {"machine": "c", "events": ["Send Data"]} + {"machine": "c"} + {"machine": "c", "events": null} + "# + .to_string(); + + let data_type = DataType::List(Box::new(Field::new( + "item", + DataType::Dictionary(u64::KEY_TYPE, Box::new(DataType::Utf8), false), + true, + ))); + + let fields = vec![Field::new("events", data_type, true)]; + + type A = MutableDictionaryArray>; + + let mut array = MutableListArray::::new(); + array + .try_extend(vec![ + Some(vec![None, Some("Elect Leader"), Some("Do Ballot")]), + Some(vec![ + Some("Do Ballot"), + None, + Some("Send Data"), + Some("Elect Leader"), + ]), + Some(vec![Some("Send Data")]), + None, + None, + ]) + .unwrap(); + + let array: ListArray = array.into(); + + ( + data, + Arc::new(StructArray::from_data( + DataType::Struct(fields), + vec![Arc::new(array) as Arc], + None, + )), + ) +} + +fn case_basics() -> (String, Arc) { + let data = r#"{"a":1, "b":2.0, "c":false, "d":"4"} + {"a":-10, "b":-3.5, "c":true, "d":null} + {"a":100000000, "b":0.6, "d":"text"}"# + .to_string(); + let data_type = DataType::Struct(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Float64, true), + Field::new("c", DataType::Boolean, true), + Field::new("d", DataType::Utf8, true), + ]); + let array = StructArray::from_data( + data_type, + vec![ + Arc::new(Int64Array::from_slice(&[1, -10, 100000000])) as Arc, + Arc::new(Float64Array::from_slice(&[2.0, -3.5, 0.6])), + Arc::new(BooleanArray::from(&[Some(false), Some(true), None])), + Arc::new(Utf8Array::::from(&[Some("4"), None, Some("text")])), + ], + None, + ); + (data, Arc::new(array)) +} + +fn case_projection() -> (String, Arc) { + let data = r#"{"a":1, "b":2.0, "c":false, "d":"4", "e":"4"} + {"a":10, "b":-3.5, "c":true, "d":null, "e":"text"} + {"a":100000000, "b":0.6, "d":"text"}"# + .to_string(); + let data_type = DataType::Struct(vec![ + Field::new("a", DataType::UInt32, true), + Field::new("b", DataType::Float32, true), + Field::new("c", DataType::Boolean, true), + // note how "d" is not here + Field::new("e", DataType::Binary, true), + ]); + let array = StructArray::from_data( + data_type, + vec![ + Arc::new(UInt32Array::from_slice(&[1, 10, 100000000])) as Arc, + Arc::new(Float32Array::from_slice(&[2.0, -3.5, 0.6])), + Arc::new(BooleanArray::from(&[Some(false), Some(true), None])), + Arc::new(BinaryArray::::from(&[ + Some(b"4".as_ref()), + Some(b"text".as_ref()), + None, + ])), + ], + None, + ); + (data, Arc::new(array)) +} + +fn case_struct() -> (String, Arc) { + let data = r#"{"a": {"b": true, "c": {"d": "text"}}} + {"a": {"b": false, "c": null}} + {"a": {"b": true, "c": {"d": "text"}}} + {"a": 1}"# + .to_string(); + + let d_field = Field::new("d", DataType::Utf8, true); + let c_field = Field::new("c", DataType::Struct(vec![d_field.clone()]), true); + let a_field = Field::new( + "a", + DataType::Struct(vec![ + Field::new("b", DataType::Boolean, true), + c_field.clone(), + ]), + true, + ); + let fields = vec![a_field]; + + // build expected output + let d = Utf8Array::::from(&vec![Some("text"), None, Some("text"), None]); + let c = StructArray::from_data(DataType::Struct(vec![d_field]), vec![Arc::new(d)], None); + + let b = BooleanArray::from(vec![Some(true), Some(false), Some(true), None]); + let inner = DataType::Struct(vec![Field::new("b", DataType::Boolean, true), c_field]); + let expected = StructArray::from_data(inner, vec![Arc::new(b), Arc::new(c)], None); + + let data_type = DataType::Struct(fields); + + ( + data, + Arc::new(StructArray::from_data( + data_type, + vec![Arc::new(expected) as Arc], + None, + )), + ) +} + +fn case_nested_list() -> (String, Arc) { + let d_field = Field::new("d", DataType::Utf8, true); + let c_field = Field::new("c", DataType::Struct(vec![d_field.clone()]), true); + let b_field = Field::new("b", DataType::Boolean, true); + let a_struct_field = Field::new( + "a", + DataType::Struct(vec![b_field.clone(), c_field.clone()]), + true, + ); + 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 data = r#" + {"a": [{"b": true, "c": {"d": "a_text"}}, {"b": false, "c": {"d": "b_text"}}]} + {"a": [{"b": false, "c": null}]} + {"a": [{"b": true, "c": {"d": "c_text"}}, {"b": null, "c": {"d": "d_text"}}, {"b": true, "c": {"d": null}}]} + {"a": null} + {"a": []} + "#.to_string(); + + // build expected output + let d = Utf8Array::::from(&vec![ + Some("a_text"), + Some("b_text"), + None, + Some("c_text"), + Some("d_text"), + None, + ]); + + let c = StructArray::from_data(DataType::Struct(vec![d_field]), vec![Arc::new(d)], None); + + let b = BooleanArray::from(vec![ + Some(true), + Some(false), + Some(false), + Some(true), + None, + Some(true), + ]); + let a_struct = StructArray::from_data( + DataType::Struct(vec![b_field, c_field]), + vec![Arc::new(b) as Arc, Arc::new(c) as Arc], + None, + ); + let expected = ListArray::from_data( + a_list_data_type, + Buffer::from_slice([0i32, 2, 3, 6, 6, 6]), + Arc::new(a_struct) as Arc, + Some(Bitmap::from_u8_slice([0b00010111], 5)), + ); + + let array = Arc::new(StructArray::from_data( + DataType::Struct(vec![a_field]), + vec![Arc::new(expected)], + None, + )); + + (data, array) +} + +fn case(case: &str) -> (String, Arc) { + match case { + "basics" => case_basics(), + "projection" => case_projection(), + "list" => case_list(), + "dict" => case_dict(), + "struct" => case_struct(), + "nested_list" => case_nested_list(), + _ => todo!(), + } +} diff --git a/tests/it/io/ndjson/read.rs b/tests/it/io/ndjson/read.rs new file mode 100644 index 00000000000..489fd548503 --- /dev/null +++ b/tests/it/io/ndjson/read.rs @@ -0,0 +1,246 @@ +use std::io::Cursor; +use std::sync::Arc; + +use arrow2::array::*; +use arrow2::datatypes::{DataType, Field}; +use arrow2::error::Result; +use arrow2::io::ndjson::read as ndjson_read; +use arrow2::io::ndjson::read::FallibleStreamingIterator; + +use super::*; + +fn test_case(case_: &str) -> Result<()> { + let (ndjson, expected) = case(case_); + + let data_type = expected.data_type().clone(); + + let mut arrays = read_and_deserialize(&ndjson, &data_type, 1000)?; + + assert_eq!(arrays.len(), 1); + assert_eq!(expected, arrays.pop().unwrap()); + Ok(()) +} + +pub fn infer(ndjson: &str) -> Result { + ndjson_read::infer(&mut Cursor::new(ndjson), None) +} + +pub fn read_and_deserialize( + ndjson: &str, + data_type: &DataType, + batch_size: usize, +) -> Result>> { + let reader = Cursor::new(ndjson); + + let mut reader = ndjson_read::FileReader::new(reader, vec!["".to_string(); batch_size], None); + + let mut chunks = vec![]; + while let Some(rows) = reader.next()? { + chunks.push(ndjson_read::deserialize(rows, data_type.clone())?); + } + + Ok(chunks) +} + +#[test] +fn infer_nullable() -> Result<()> { + let ndjson = r#"true + false + null + true + "#; + let expected = DataType::Boolean; + + let result = infer(ndjson)?; + + assert_eq!(result, expected); + Ok(()) +} + +fn case_nested_struct() -> (String, Arc) { + let ndjson = r#"{"a": {"a": 2.0, "b": 2}} + {"a": {"b": 2}} + {"a": {"a": 2.0, "b": 2, "c": true}} + {"a": {"a": 2.0, "b": 2}} + "#; + + let inner = DataType::Struct(vec![ + Field::new("a", DataType::Float64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Boolean, true), + ]); + + let data_type = DataType::Struct(vec![Field::new("a", inner.clone(), true)]); + + let values = vec![ + Arc::new(Float64Array::from([Some(2.0), None, Some(2.0), Some(2.0)])) as Arc, + Arc::new(Int64Array::from([Some(2), Some(2), Some(2), Some(2)])), + Arc::new(BooleanArray::from([None, None, Some(true), None])), + ]; + + let values = vec![Arc::new(StructArray::from_data(inner, values, None)) as Arc]; + + let array = Arc::new(StructArray::from_data(data_type, values, None)); + + (ndjson.to_string(), array) +} + +#[test] +fn infer_nested_struct() -> Result<()> { + let (ndjson, array) = case_nested_struct(); + + let result = infer(&ndjson)?; + + assert_eq!(&result, array.data_type()); + Ok(()) +} + +#[test] +fn read_nested_struct() -> Result<()> { + let (ndjson, expected) = case_nested_struct(); + + let data_type = infer(&ndjson)?; + + let result = read_and_deserialize(&ndjson, &data_type, 100)?; + + assert_eq!(result, vec![expected]); + Ok(()) +} + +#[test] +fn read_nested_struct_batched() -> Result<()> { + let (ndjson, expected) = case_nested_struct(); + let batch_size = 2; + + // create a chunked array by batch_size from the (un-chunked) expected + let expected: Vec> = (0..(expected.len() + batch_size - 1) / batch_size) + .map(|offset| expected.slice(offset * batch_size, batch_size).into()) + .collect(); + + let data_type = infer(&ndjson)?; + + let result = read_and_deserialize(&ndjson, &data_type, batch_size)?; + + assert_eq!(result, expected,); + Ok(()) +} + +#[test] +fn invalid_infer_schema() -> Result<()> { + let re = ndjson_read::infer(&mut Cursor::new("city,lat,lng"), None); + assert_eq!( + re.err().unwrap().to_string(), + "External error: expected value at line 1 column 1", + ); + Ok(()) +} + +#[test] +fn infer_schema_mixed_list() -> Result<()> { + let ndjson = r#"{"a":1, "b":[2.0, 1.3, -6.1], "c":[false, true], "d":4.1} + {"a":-10, "b":[2.0, 1.3, -6.1], "c":null, "d":null} + {"a":2, "b":[2.0, null, -6.1], "c":[false, null], "d":"text"} + {"a":3, "b":4, "c": true, "d":[1, false, "array", 2.4]} + "#; + + let expected = DataType::Struct(vec![ + Field::new("a", DataType::Int64, true), + Field::new( + "b", + DataType::List(Box::new(Field::new("item", DataType::Float64, true))), + true, + ), + Field::new( + "c", + DataType::List(Box::new(Field::new("item", DataType::Boolean, true))), + true, + ), + Field::new("d", DataType::Utf8, true), + ]); + + let result = infer(ndjson)?; + + assert_eq!(result, expected); + Ok(()) +} + +#[test] +fn basic() -> Result<()> { + test_case("basics") +} + +#[test] +fn projection() -> Result<()> { + test_case("projection") +} + +#[test] +fn dictionary() -> Result<()> { + test_case("dict") +} + +#[test] +fn list() -> Result<()> { + test_case("list") +} + +#[test] +fn nested_struct() -> Result<()> { + test_case("struct") +} + +#[test] +fn nested_list() -> Result<()> { + test_case("nested_list") +} + +#[test] +fn line_break_in_values() -> Result<()> { + let ndjson = r#" + "aa\n\n" + "aa\n" + null + "#; + + let data_type = DataType::Utf8; + let arrays = read_and_deserialize(ndjson, &data_type, 1000)?; + + let expected = Utf8Array::::from(&[Some("aa\n\n"), Some("aa\n"), None]); + + assert_eq!(expected, arrays[0].as_ref()); + Ok(()) +} + +#[test] +fn invalid_read_record() -> Result<()> { + let fields = vec![Field::new( + "a", + DataType::Struct(vec![Field::new("a", DataType::Utf8, true)]), + true, + )]; + let data_type = DataType::Struct(fields); + let arrays = read_and_deserialize("city,lat,lng", &data_type, 1000); + + assert_eq!( + arrays.err().unwrap().to_string(), + "External error: expected value at line 1 column 1", + ); + Ok(()) +} + +#[test] +fn skip_empty_lines() -> Result<()> { + let ndjson = " + {\"a\": 1} + + {\"a\": 2} + + {\"a\": 3}"; + + let data_type = DataType::Struct(vec![Field::new("a", DataType::Int64, true)]); + let arrays = read_and_deserialize(ndjson, &data_type, 1000)?; + + assert_eq!(1, arrays.len()); + assert_eq!(3, arrays[0].len()); + Ok(()) +} diff --git a/tests/it/io/parquet/mod.rs b/tests/it/io/parquet/mod.rs index d5446a8c8b9..8eb009204b2 100644 --- a/tests/it/io/parquet/mod.rs +++ b/tests/it/io/parquet/mod.rs @@ -330,6 +330,12 @@ pub fn pyarrow_nullable(column: usize) -> Box { 11 => Box::new( PrimitiveArray::::from(i64_values).to(DataType::Timestamp(TimeUnit::Second, None)), ), + 13 => Box::new( + PrimitiveArray::::from(i64_values).to(DataType::Timestamp( + TimeUnit::Second, + Some("UTC".to_string()), + )), + ), _ => unreachable!(), } } @@ -415,6 +421,13 @@ pub fn pyarrow_nullable_statistics(column: usize) -> Option> min_value: Some(0), max_value: Some(9), }), + 13 => Box::new(PrimitiveStatistics:: { + data_type: DataType::Timestamp(TimeUnit::Second, Some("UTC".to_string())), + distinct_count: None, + null_count: Some(3), + min_value: Some(0), + max_value: Some(9), + }), _ => unreachable!(), }) } diff --git a/tests/it/io/parquet/read.rs b/tests/it/io/parquet/read.rs index 166becd593f..90d61f0a97c 100644 --- a/tests/it/io/parquet/read.rs +++ b/tests/it/io/parquet/read.rs @@ -355,6 +355,11 @@ fn v1_timestamp_s_nullable_dict() -> Result<()> { test_pyarrow_integration(11, 1, "basic", true, false, None) } +#[test] +fn v1_timestamp_s_utc_nullable() -> Result<()> { + test_pyarrow_integration(13, 1, "basic", false, false, None) +} + #[test] fn v2_decimal_26_required() -> Result<()> { test_pyarrow_integration(8, 2, "basic", false, true, None)