diff --git a/Cargo.toml b/Cargo.toml index 17ddddab439..549965c2c2c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,6 +17,7 @@ bench = false [dependencies] either = "1.6" num-traits = "0.2" +dyn-clone = "1" bytemuck = { version = "1", features = ["derive"] } chrono = { version = "0.4", default_features = false, features = ["std"] } chrono-tz = { version = "0.6", optional = true } diff --git a/arrow-parquet-integration-testing/src/main.rs b/arrow-parquet-integration-testing/src/main.rs index 887ba77d45b..c7e9a03431d 100644 --- a/arrow-parquet-integration-testing/src/main.rs +++ b/arrow-parquet-integration-testing/src/main.rs @@ -1,3 +1,6 @@ +use std::fs::File; +use std::{collections::HashMap, io::Read}; + use arrow2::array::Array; use arrow2::io::ipc::IpcField; use arrow2::{ @@ -15,15 +18,12 @@ use arrow2::{ }; use clap::Parser; use flate2::read::GzDecoder; -use std::fs::File; -use std::sync::Arc; -use std::{collections::HashMap, io::Read}; /// Read gzipped JSON file pub fn read_gzip_json( version: &str, file_name: &str, -) -> Result<(Schema, Vec, Vec>>)> { +) -> Result<(Schema, Vec, Vec>>)> { let path = format!( "../testing/arrow-testing/data/arrow-ipc-stream/integration/{}/{}.json.gz", version, file_name diff --git a/arrow-pyarrow-integration-testing/src/c_stream.rs b/arrow-pyarrow-integration-testing/src/c_stream.rs index 027c0d624f2..8998f4812ea 100644 --- a/arrow-pyarrow-integration-testing/src/c_stream.rs +++ b/arrow-pyarrow-integration-testing/src/c_stream.rs @@ -34,15 +34,13 @@ pub fn from_rust_iterator(py: Python) -> PyResult { 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)], + vec![array.boxed()], None, - ); + ) + .boxed(); // 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 _; diff --git a/arrow-pyarrow-integration-testing/src/lib.rs b/arrow-pyarrow-integration-testing/src/lib.rs index 4e8c1e8a73d..ad6a16bbadb 100644 --- a/arrow-pyarrow-integration-testing/src/lib.rs +++ b/arrow-pyarrow-integration-testing/src/lib.rs @@ -4,7 +4,6 @@ mod c_stream; use std::error; use std::fmt; -use std::sync::Arc; use pyo3::exceptions::PyOSError; use pyo3::ffi::Py_uintptr_t; @@ -50,7 +49,7 @@ impl From for PyErr { } } -fn to_rust_array(ob: PyObject, py: Python) -> PyResult> { +fn to_rust_array(ob: PyObject, py: Python) -> PyResult> { // prepare a pointer to receive the Array struct let array = Box::new(ffi::ArrowArray::empty()); let schema = Box::new(ffi::ArrowSchema::empty()); @@ -73,7 +72,7 @@ fn to_rust_array(ob: PyObject, py: Python) -> PyResult> { Ok(array.into()) } -fn to_py_array(array: Arc, py: Python) -> PyResult { +fn to_py_array(array: Box, py: Python) -> PyResult { let array_ptr = Box::new(ffi::ArrowArray::empty()); let schema_ptr = Box::new(ffi::ArrowSchema::empty()); diff --git a/benches/iter_list.rs b/benches/iter_list.rs index 24af30591c7..ba576cb67b0 100644 --- a/benches/iter_list.rs +++ b/benches/iter_list.rs @@ -1,5 +1,4 @@ use std::iter::FromIterator; -use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; @@ -28,7 +27,7 @@ fn add_benchmark(c: &mut Criterion) { let array = ListArray::::from_data( data_type, offsets.into(), - Arc::new(values), + Box::new(values), Some(validity), ); diff --git a/benches/write_csv.rs b/benches/write_csv.rs index faba0701c65..ee30d9ec3a7 100644 --- a/benches/write_csv.rs +++ b/benches/write_csv.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::*; @@ -8,9 +6,9 @@ use arrow2::error::Result; use arrow2::io::csv::write; use arrow2::util::bench_util::*; -type ChunkArc = Chunk>; +type ChunkBox = Chunk>; -fn write_batch(columns: &ChunkArc) -> Result<()> { +fn write_batch(columns: &ChunkBox) -> Result<()> { let mut writer = vec![]; assert_eq!(columns.arrays().len(), 1); @@ -20,8 +18,8 @@ fn write_batch(columns: &ChunkArc) -> Result<()> { write::write_chunk(&mut writer, columns, &options) } -fn make_chunk(array: impl Array + 'static) -> Chunk> { - Chunk::new(vec![Arc::new(array)]) +fn make_chunk(array: impl Array + 'static) -> Chunk> { + Chunk::new(vec![Box::new(array)]) } fn add_benchmark(c: &mut Criterion) { diff --git a/benches/write_ipc.rs b/benches/write_ipc.rs index 68d7b9882cc..8b35c7b6d11 100644 --- a/benches/write_ipc.rs +++ b/benches/write_ipc.rs @@ -11,7 +11,7 @@ use arrow2::util::bench_util::{create_boolean_array, create_primitive_array, cre fn write(array: &dyn Array) -> Result<()> { let field = Field::new("c1", array.data_type().clone(), true); let schema = vec![field].into(); - let columns = Chunk::try_new(vec![clone(array).into()])?; + let columns = Chunk::try_new(vec![clone(array)])?; let writer = Cursor::new(vec![]); let mut writer = FileWriter::try_new(writer, &schema, None, Default::default())?; diff --git a/benches/write_parquet.rs b/benches/write_parquet.rs index b1f5ecfd5ef..f4053c92791 100644 --- a/benches/write_parquet.rs +++ b/benches/write_parquet.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use criterion::{criterion_group, criterion_main, Criterion}; use arrow2::array::{clone, Array}; @@ -9,11 +7,11 @@ use arrow2::error::Result; use arrow2::io::parquet::write::*; use arrow2::util::bench_util::{create_boolean_array, create_primitive_array, create_string_array}; -type ChunkArc = Chunk>; +type ChunkBox = Chunk>; fn write(array: &dyn Array, encoding: Encoding) -> Result<()> { let schema = Schema::from(vec![Field::new("c1", array.data_type().clone(), true)]); - let columns: ChunkArc = Chunk::new(vec![clone(array).into()]); + let columns: ChunkBox = Chunk::new(vec![clone(array)]); let options = WriteOptions { write_statistics: false, diff --git a/examples/avro_read_async.rs b/examples/avro_read_async.rs index 6370f6aeca5..d3c51720496 100644 --- a/examples/avro_read_async.rs +++ b/examples/avro_read_async.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use futures::pin_mut; use futures::StreamExt; use tokio::fs::File; @@ -19,8 +17,8 @@ async fn main() -> Result<()> { let mut reader = File::open(file_path).await?.compat(); let (avro_schemas, schema, compression, marker) = read_metadata(&mut reader).await?; - let avro_schemas = Arc::new(avro_schemas); - let projection = Arc::new(schema.fields.iter().map(|_| true).collect::>()); + let avro_schemas = Box::new(avro_schemas); + let projection = Box::new(schema.fields.iter().map(|_| true).collect::>()); let blocks = block_stream(&mut reader, marker).await; diff --git a/examples/cow.rs b/examples/cow.rs new file mode 100644 index 00000000000..4b76972a795 --- /dev/null +++ b/examples/cow.rs @@ -0,0 +1,20 @@ +// This example demos how to operate on arrays in-place. +use arrow2::array::{Array, PrimitiveArray}; + +fn main() { + // say we have have received an array + let mut array: Box = PrimitiveArray::from_vec(vec![1i32, 2]).boxed(); + + // we can apply a transformation to its values without allocating a new array as follows: + // 1. downcast it to the correct type (known via `array.data_type().to_physical_type()`) + let array = array + .as_any_mut() + .downcast_mut::>() + .unwrap(); + + // 2. call `apply_values` with the function to apply over the values + array.apply_values(|x| x.iter_mut().for_each(|x| *x *= 10)); + + // confirm that it gives the right result :) + assert_eq!(array, &PrimitiveArray::from_vec(vec![10i32, 20])); +} diff --git a/examples/csv_read.rs b/examples/csv_read.rs index 988110c45fa..c2661f8a6a9 100644 --- a/examples/csv_read.rs +++ b/examples/csv_read.rs @@ -1,11 +1,9 @@ -use std::sync::Arc; - use arrow2::array::Array; use arrow2::chunk::Chunk; use arrow2::error::Result; use arrow2::io::csv::read; -fn read_path(path: &str, projection: Option<&[usize]>) -> Result>> { +fn read_path(path: &str, projection: Option<&[usize]>) -> Result>> { // Create a CSV reader. This is typically created on the thread that reads the file and // thus owns the read head. let mut reader = read::ReaderBuilder::new().from_path(path)?; diff --git a/examples/csv_read_parallel.rs b/examples/csv_read_parallel.rs index fd71f081cfb..9d84d979ffc 100644 --- a/examples/csv_read_parallel.rs +++ b/examples/csv_read_parallel.rs @@ -1,6 +1,5 @@ use crossbeam_channel::unbounded; -use std::sync::Arc; use std::thread; use std::time::SystemTime; @@ -8,7 +7,7 @@ use arrow2::array::Array; use arrow2::chunk::Chunk; use arrow2::{error::Result, io::csv::read}; -fn parallel_read(path: &str) -> Result>>> { +fn parallel_read(path: &str) -> Result>>> { let batch_size = 100; let has_header = true; let projection = None; @@ -19,7 +18,7 @@ fn parallel_read(path: &str) -> Result>>> { let mut reader = read::ReaderBuilder::new().from_path(path)?; let (fields, _) = read::infer_schema(&mut reader, Some(batch_size * 10), has_header, &read::infer)?; - let fields = Arc::new(fields); + let fields = Box::new(fields); let start = SystemTime::now(); // spawn a thread to produce `Vec` (IO bounded) diff --git a/examples/csv_write_parallel.rs b/examples/csv_write_parallel.rs index 4db8683c285..65a7e74e5c3 100644 --- a/examples/csv_write_parallel.rs +++ b/examples/csv_write_parallel.rs @@ -1,7 +1,6 @@ use std::io::Write; use std::sync::mpsc; use std::sync::mpsc::{Receiver, Sender}; -use std::sync::Arc; use std::thread; use arrow2::{ @@ -11,7 +10,7 @@ use arrow2::{ io::csv::write, }; -fn parallel_write(path: &str, batches: [Chunk>; 2]) -> Result<()> { +fn parallel_write(path: &str, batches: [Chunk>; 2]) -> Result<()> { let options = write::SerializeOptions::default(); // write a header @@ -59,7 +58,7 @@ fn main() -> Result<()> { Some(5), Some(6), ]); - let columns = Chunk::new(vec![array.arced()]); + let columns = Chunk::new(vec![array.boxed()]); parallel_write("example.csv", [columns.clone(), columns]) } diff --git a/examples/extension.rs b/examples/extension.rs index 70b9e076029..158b9bbd438 100644 --- a/examples/extension.rs +++ b/examples/extension.rs @@ -1,5 +1,4 @@ use std::io::{Cursor, Seek, Write}; -use std::sync::Arc; use arrow2::array::*; use arrow2::chunk::Chunk; @@ -40,7 +39,7 @@ fn write_ipc(writer: W, array: impl Array + 'static) -> Result< let options = write::WriteOptions { compression: None }; let mut writer = write::FileWriter::new(writer, schema, None, options); - let batch = Chunk::try_new(vec![Arc::new(array) as Arc])?; + let batch = Chunk::try_new(vec![Box::new(array) as Box])?; writer.start()?; writer.write(&batch, None)?; @@ -49,7 +48,7 @@ fn write_ipc(writer: W, array: impl Array + 'static) -> Result< Ok(writer.into_inner()) } -fn read_ipc(buf: &[u8]) -> Result>> { +fn read_ipc(buf: &[u8]) -> Result>> { let mut cursor = Cursor::new(buf); let metadata = read::read_file_metadata(&mut cursor)?; let mut reader = read::FileReader::new(cursor, metadata, None); diff --git a/examples/ffi.rs b/examples/ffi.rs index 3356e554cdb..84a65599e4a 100644 --- a/examples/ffi.rs +++ b/examples/ffi.rs @@ -2,10 +2,9 @@ use arrow2::array::{Array, PrimitiveArray}; use arrow2::datatypes::Field; use arrow2::error::Result; use arrow2::ffi; -use std::sync::Arc; unsafe fn export( - array: Arc, + array: Box, array_ptr: *mut ffi::ArrowArray, schema_ptr: *mut ffi::ArrowSchema, ) { @@ -22,7 +21,7 @@ unsafe fn import(array: Box, schema: &ffi::ArrowSchema) -> Resu fn main() -> Result<()> { // let's assume that we have an array: - let array = PrimitiveArray::::from([Some(1), None, Some(123)]).arced(); + let array = PrimitiveArray::::from([Some(1), None, Some(123)]).boxed(); // the goal is to export this array and import it back via FFI. // to import, we initialize the structs that will receive the data diff --git a/examples/ipc_file_read.rs b/examples/ipc_file_read.rs index cdaba030d8c..23d179adb47 100644 --- a/examples/ipc_file_read.rs +++ b/examples/ipc_file_read.rs @@ -1,5 +1,4 @@ use std::fs::File; -use std::sync::Arc; use arrow2::array::Array; use arrow2::chunk::Chunk; @@ -9,7 +8,7 @@ use arrow2::io::ipc::read; use arrow2::io::print; /// Simplest way: read all record batches from the file. This can be used e.g. for random access. -fn read_batches(path: &str) -> Result<(Schema, Vec>>)> { +fn read_batches(path: &str) -> Result<(Schema, Vec>>)> { let mut file = File::open(path)?; // read the files' metadata. At this point, we can distribute the read whatever we like. @@ -25,7 +24,7 @@ fn read_batches(path: &str) -> Result<(Schema, Vec>>)> { } /// Random access way: read a single record batch from the file. This can be used e.g. for random access. -fn read_batch(path: &str) -> Result<(Schema, Chunk>)> { +fn read_batch(path: &str) -> Result<(Schema, Chunk>)> { let mut file = File::open(path)?; // read the files' metadata. At this point, we can distribute the read whatever we like. diff --git a/examples/ipc_file_write.rs b/examples/ipc_file_write.rs index bea0ef85122..74c7ffeedff 100644 --- a/examples/ipc_file_write.rs +++ b/examples/ipc_file_write.rs @@ -1,5 +1,4 @@ use std::fs::File; -use std::sync::Arc; use arrow2::array::{Array, Int32Array, Utf8Array}; use arrow2::chunk::Chunk; @@ -7,7 +6,7 @@ use arrow2::datatypes::{DataType, Field, Schema}; use arrow2::error::Result; use arrow2::io::ipc::write; -fn write_batches(path: &str, schema: Schema, columns: &[Chunk>]) -> Result<()> { +fn write_batches(path: &str, schema: Schema, columns: &[Chunk>]) -> Result<()> { let file = File::create(path)?; let options = write::WriteOptions { compression: None }; @@ -35,7 +34,7 @@ fn main() -> Result<()> { let a = Int32Array::from_slice(&[1, 2, 3, 4, 5]); let b = Utf8Array::::from_slice(&["a", "b", "c", "d", "e"]); - let batch = Chunk::try_new(vec![a.arced(), b.arced()])?; + let batch = Chunk::try_new(vec![a.boxed(), b.boxed()])?; // write it write_batches(file_path, schema, &[batch])?; diff --git a/examples/json_read.rs b/examples/json_read.rs index 608980a62de..b674f4a9966 100644 --- a/examples/json_read.rs +++ b/examples/json_read.rs @@ -1,12 +1,11 @@ /// Example of reading a JSON file. use std::fs; -use std::sync::Arc; use arrow2::array::Array; use arrow2::error::Result; use arrow2::io::json::read; -fn read_path(path: &str) -> Result> { +fn read_path(path: &str) -> Result> { // read the file into memory (IO-bounded) let data = fs::read(path)?; diff --git a/examples/ndjson_read.rs b/examples/ndjson_read.rs index 0490b23e2f1..93b7b64ba25 100644 --- a/examples/ndjson_read.rs +++ b/examples/ndjson_read.rs @@ -1,13 +1,12 @@ use std::fs::File; use std::io::{BufReader, Seek}; -use std::sync::Arc; use arrow2::array::Array; use arrow2::error::Result; use arrow2::io::ndjson::read; use arrow2::io::ndjson::read::FallibleStreamingIterator; -fn read_path(path: &str) -> Result>> { +fn read_path(path: &str) -> Result>> { let batch_size = 1024; // number of rows per array let mut reader = BufReader::new(File::open(path)?); diff --git a/examples/parquet_read_async.rs b/examples/parquet_read_async.rs index 65a802b0008..2428b368869 100644 --- a/examples/parquet_read_async.rs +++ b/examples/parquet_read_async.rs @@ -1,4 +1,3 @@ -use std::sync::Arc; use std::time::SystemTime; use futures::future::BoxFuture; @@ -15,7 +14,7 @@ async fn main() -> Result<()> { use std::env; let args: Vec = env::args().collect(); - let file_path = Arc::new(args[1].clone()); + let file_path = Box::new(args[1].clone()); // # Read metadata let mut reader = BufReader::new(File::open(file_path.as_ref()).await?).compat(); diff --git a/examples/parquet_read_parallel/src/main.rs b/examples/parquet_read_parallel/src/main.rs index 3b6eb243bc8..313205bbfd2 100644 --- a/examples/parquet_read_parallel/src/main.rs +++ b/examples/parquet_read_parallel/src/main.rs @@ -1,7 +1,6 @@ //! Example demonstrating how to read from parquet in parallel using rayon use std::fs::File; use std::io::BufReader; -use std::sync::Arc; use std::time::SystemTime; use log::trace; @@ -18,7 +17,7 @@ mod logger; /// # Panic /// If the iterators are empty -fn deserialize_parallel(columns: &mut [ArrayIter<'static>]) -> Result>> { +fn deserialize_parallel(columns: &mut [ArrayIter<'static>]) -> Result>> { // CPU-bounded let columns = columns .par_iter_mut() diff --git a/examples/parquet_write.rs b/examples/parquet_write.rs index 5237baf246b..fef11079ac9 100644 --- a/examples/parquet_write.rs +++ b/examples/parquet_write.rs @@ -1,5 +1,4 @@ use std::fs::File; -use std::sync::Arc; use arrow2::{ array::{Array, Int32Array}, @@ -12,7 +11,7 @@ use arrow2::{ }, }; -fn write_batch(path: &str, schema: Schema, columns: Chunk>) -> Result<()> { +fn write_batch(path: &str, schema: Schema, columns: Chunk>) -> Result<()> { let options = WriteOptions { write_statistics: true, compression: CompressionOptions::Uncompressed, @@ -53,7 +52,7 @@ fn main() -> Result<()> { ]); let field = Field::new("c1", array.data_type().clone(), true); let schema = Schema::from(vec![field]); - let columns = Chunk::new(vec![array.arced()]); + let columns = Chunk::new(vec![array.boxed()]); write_batch("test.parquet", schema, columns) } diff --git a/examples/parquet_write_parallel/src/main.rs b/examples/parquet_write_parallel/src/main.rs index 4939905a76f..869e0f65838 100644 --- a/examples/parquet_write_parallel/src/main.rs +++ b/examples/parquet_write_parallel/src/main.rs @@ -1,6 +1,5 @@ //! Example demonstrating how to write to parquet in parallel. use std::collections::VecDeque; -use std::sync::Arc; use rayon::prelude::*; @@ -12,7 +11,7 @@ use arrow2::{ io::parquet::{read::ParquetError, write::*}, }; -type Chunk = AChunk>; +type Chunk = AChunk>; struct Bla { columns: VecDeque, @@ -134,7 +133,14 @@ fn create_batch(size: usize) -> Result { }) .collect(); +<<<<<<< HEAD Chunk::try_new(vec![c1.arced(), c2.arced()]) +======= + Chunk::try_new(vec![ + Box::new(c1) as Box, + Box::new(c2) as Box, + ]) +>>>>>>> 619252ff0 (Arc->Box) } fn main() -> Result<()> { diff --git a/guide/src/high_level.md b/guide/src/high_level.md index 8c5b93c55f0..b460e44f5c2 100644 --- a/guide/src/high_level.md +++ b/guide/src/high_level.md @@ -268,3 +268,16 @@ Some notes: and cloned its validity. This approach is suitable for operations whose branching off is more expensive than operating over all values. If the operation is expensive, then using `PrimitiveArray::::from_trusted_len_iter` is likely faster. + +## Clone on write semantics + +We support the mutation of arrays in-place via clone-on-write semantics. +Essentially, all data is under an `Arc`, but it can be taken via `Arc::get_mut` +and operated in place. + +Below is a complete example of how to operate on a `Box` without +extra allocations. + +```rust,ignore +{{#include ../examples/cow.rs}} +``` diff --git a/integration-testing/src/flight_client_scenarios/integration_test.rs b/integration-testing/src/flight_client_scenarios/integration_test.rs index ea220f197a0..94fd19c1629 100644 --- a/integration-testing/src/flight_client_scenarios/integration_test.rs +++ b/integration-testing/src/flight_client_scenarios/integration_test.rs @@ -41,14 +41,12 @@ use arrow_format::{ use futures::{channel::mpsc, sink::SinkExt, stream, StreamExt}; use tonic::{Request, Streaming}; -use std::sync::Arc; - type Error = Box; type Result = std::result::Result; type Client = FlightServiceClient; -type ChunkArc = Chunk>; +type ChunkBox = Chunk>; pub async fn run_scenario(host: &str, port: u16, path: &str) -> Result { let url = format!("http://{}:{}", host, port); @@ -66,7 +64,7 @@ pub async fn run_scenario(host: &str, port: u16, path: &str) -> Result { is_little_endian: true, }; - let schema = Arc::new(schema); + let schema = Box::new(schema); let mut descriptor = FlightDescriptor::default(); descriptor.set_type(DescriptorType::Path); @@ -90,7 +88,7 @@ async fn upload_data( schema: &Schema, fields: &[IpcField], descriptor: FlightDescriptor, - original_data: Vec, + original_data: Vec, ) -> Result { let (mut upload_tx, upload_rx) = mpsc::channel(10); @@ -145,7 +143,7 @@ async fn upload_data( async fn send_batch( upload_tx: &mut mpsc::Sender, metadata: &[u8], - batch: &ChunkArc, + batch: &ChunkBox, fields: &[IpcField], options: &write::WriteOptions, ) -> Result { @@ -166,7 +164,7 @@ async fn verify_data( descriptor: FlightDescriptor, expected_schema: &Schema, ipc_schema: &IpcSchema, - expected_data: &[ChunkArc], + expected_data: &[ChunkBox], ) -> Result { let resp = client.get_flight_info(Request::new(descriptor)).await?; let info = resp.into_inner(); @@ -202,7 +200,7 @@ async fn verify_data( async fn consume_flight_location( location: Location, ticket: Ticket, - expected_data: &[ChunkArc], + expected_data: &[ChunkBox], schema: &Schema, ipc_schema: &IpcSchema, ) -> Result { diff --git a/integration-testing/src/flight_server_scenarios/integration_test.rs b/integration-testing/src/flight_server_scenarios/integration_test.rs index 5c3129dcc8c..87d182928ca 100644 --- a/integration-testing/src/flight_server_scenarios/integration_test.rs +++ b/integration-testing/src/flight_server_scenarios/integration_test.rs @@ -62,7 +62,7 @@ pub async fn scenario_setup(port: u16) -> Result { struct IntegrationDataset { schema: Schema, ipc_schema: IpcSchema, - chunks: Vec>>, + chunks: Vec>>, } #[derive(Clone, Default)] @@ -282,7 +282,7 @@ async fn record_batch_from_message( fields: &[Field], ipc_schema: &IpcSchema, dictionaries: &mut Dictionaries, -) -> Result>, Status> { +) -> Result>, Status> { let mut reader = std::io::Cursor::new(data_body); let arrow_batch_result = ipc::read::read_record_batch( diff --git a/integration-testing/src/lib.rs b/integration-testing/src/lib.rs index 0fa3f9e1e0b..ed70a32f95b 100644 --- a/integration-testing/src/lib.rs +++ b/integration-testing/src/lib.rs @@ -29,7 +29,6 @@ use arrow2::io::json_integration::{read, ArrowJsonBatch, ArrowJsonDictionaryBatc use std::collections::HashMap; use std::fs::File; use std::io::BufReader; -use std::sync::Arc; /// The expected username for the basic auth integration test. pub const AUTH_USERNAME: &str = "arrow"; @@ -45,7 +44,7 @@ pub struct ArrowFile { // we can evolve this into a concrete Arrow type // this is temporarily not being read from pub _dictionaries: HashMap, - pub batches: Vec>>, + pub batches: Vec>>, } pub fn read_json_file(json_name: &str) -> Result { diff --git a/src/array/README.md b/src/array/README.md index f5e71a15802..497348610aa 100644 --- a/src/array/README.md +++ b/src/array/README.md @@ -20,7 +20,7 @@ This document describes the overall design of this module. * 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. +* Every child array on the struct MUST be `Box`. * 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. diff --git a/src/array/binary/mod.rs b/src/array/binary/mod.rs index ff6fb609ab8..6b0668865b8 100644 --- a/src/array/binary/mod.rs +++ b/src/array/binary/mod.rs @@ -423,6 +423,11 @@ impl Array for BinaryArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/boolean/mod.rs b/src/array/boolean/mod.rs index 977529cf5ca..6faef21186f 100644 --- a/src/array/boolean/mod.rs +++ b/src/array/boolean/mod.rs @@ -257,6 +257,11 @@ impl Array for BooleanArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/dictionary/ffi.rs b/src/array/dictionary/ffi.rs index b1159ebd685..6bc467630a2 100644 --- a/src/array/dictionary/ffi.rs +++ b/src/array/dictionary/ffi.rs @@ -33,7 +33,7 @@ impl FromFfi for DictionaryArray let data_type = K::PRIMITIVE.into(); let keys = PrimitiveArray::::try_new(data_type, values, validity)?; let values = array.dictionary()?.unwrap(); - let values = ffi::try_from(values)?.into(); + let values = ffi::try_from(values)?; Ok(DictionaryArray::::from_data(keys, values)) } diff --git a/src/array/dictionary/mod.rs b/src/array/dictionary/mod.rs index 8bda284c1ec..83993644836 100644 --- a/src/array/dictionary/mod.rs +++ b/src/array/dictionary/mod.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{ bitmap::Bitmap, datatypes::{DataType, IntegerType}, @@ -54,14 +52,14 @@ impl DictionaryKey for u64 { pub struct DictionaryArray { data_type: DataType, keys: PrimitiveArray, - values: Arc, + values: Box, } impl DictionaryArray { /// Returns a new empty [`DictionaryArray`]. pub fn new_empty(data_type: DataType) -> Self { let values = Self::get_child(&data_type); - let values = new_empty_array(values.clone()).into(); + let values = new_empty_array(values.clone()); let data_type = K::PRIMITIVE.into(); Self::from_data(PrimitiveArray::::new_empty(data_type), values) } @@ -73,12 +71,12 @@ impl DictionaryArray { let data_type = K::PRIMITIVE.into(); Self::from_data( PrimitiveArray::::new_null(data_type, length), - new_empty_array(values.clone()).into(), + new_empty_array(values.clone()), ) } /// The canonical method to create a new [`DictionaryArray`]. - pub fn from_data(keys: PrimitiveArray, values: Arc) -> Self { + pub fn from_data(keys: PrimitiveArray, values: Box) -> Self { let data_type = DataType::Dictionary(K::KEY_TYPE, Box::new(values.data_type().clone()), false); @@ -119,7 +117,7 @@ impl DictionaryArray { panic!("validity should be as least as large as the array") } let mut arr = self.clone(); - arr.values = Arc::from(arr.values.with_validity(validity)); + arr.values = arr.values.with_validity(validity); arr } } @@ -147,7 +145,7 @@ impl DictionaryArray { /// Returns the values of the [`DictionaryArray`]. #[inline] - pub fn values(&self) -> &Arc { + pub fn values(&self) -> &Box { &self.values } @@ -189,6 +187,11 @@ impl Array for DictionaryArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/dictionary/mutable.rs b/src/array/dictionary/mutable.rs index a77e6e37e68..ba0d393f973 100644 --- a/src/array/dictionary/mutable.rs +++ b/src/array/dictionary/mutable.rs @@ -37,7 +37,7 @@ pub struct MutableDictionaryArray { impl From> for DictionaryArray { fn from(mut other: MutableDictionaryArray) -> Self { - DictionaryArray::::from_data(other.keys.into(), other.values.as_arc()) + DictionaryArray::::from_data(other.keys.into(), other.values.as_box()) } } @@ -155,14 +155,14 @@ impl MutableArray for MutableDictio fn as_box(&mut self) -> Box { Box::new(DictionaryArray::::from_data( std::mem::take(&mut self.keys).into(), - self.values.as_arc(), + self.values.as_box(), )) } fn as_arc(&mut self) -> Arc { Arc::new(DictionaryArray::::from_data( std::mem::take(&mut self.keys).into(), - self.values.as_arc(), + self.values.as_box(), )) } diff --git a/src/array/equal/mod.rs b/src/array/equal/mod.rs index 932ccd67bbe..737edb35e76 100644 --- a/src/array/equal/mod.rs +++ b/src/array/equal/mod.rs @@ -21,7 +21,7 @@ impl PartialEq for dyn Array + '_ { } } -impl PartialEq for Arc { +impl PartialEq for std::sync::Arc { fn eq(&self, that: &dyn Array) -> bool { equal(&**self, that) } @@ -51,6 +51,12 @@ impl PartialEq<&dyn Array> for PrimitiveArray { } } +impl PartialEq> for &dyn Array { + fn eq(&self, other: &PrimitiveArray) -> bool { + equal(*self, other) + } +} + impl PartialEq> for PrimitiveArray { fn eq(&self, other: &Self) -> bool { primitive::equal::(self, other) diff --git a/src/array/ffi.rs b/src/array/ffi.rs index a9d0a089074..c879b7fa037 100644 --- a/src/array/ffi.rs +++ b/src/array/ffi.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::datatypes::PhysicalType; use crate::{array::*, ffi}; @@ -14,7 +12,7 @@ pub(crate) unsafe trait ToFfi { fn buffers(&self) -> Vec>>; /// The children - fn children(&self) -> Vec> { + fn children(&self) -> Vec> { vec![] } @@ -50,8 +48,8 @@ macro_rules! ffi_dyn { type BuffersChildren = ( usize, Vec>>, - Vec>, - Option>, + Vec>, + Option>, ); pub fn offset_buffers_children_dictionary(array: &dyn Array) -> BuffersChildren { diff --git a/src/array/fixed_size_binary/mod.rs b/src/array/fixed_size_binary/mod.rs index 0c25fbd2e77..f02ea0d3bb7 100644 --- a/src/array/fixed_size_binary/mod.rs +++ b/src/array/fixed_size_binary/mod.rs @@ -246,6 +246,11 @@ impl Array for FixedSizeBinaryArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/fixed_size_list/ffi.rs b/src/array/fixed_size_list/ffi.rs index 872c7459869..47367b180a5 100644 --- a/src/array/fixed_size_list/ffi.rs +++ b/src/array/fixed_size_list/ffi.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use super::FixedSizeListArray; use crate::{ array::{ @@ -15,8 +13,8 @@ unsafe impl ToFfi for FixedSizeListArray { vec![self.validity.as_ref().map(|x| x.as_ptr())] } - fn children(&self) -> Vec> { - vec![self.values().clone()] + fn children(&self) -> Vec> { + vec![self.values.clone()] } fn offset(&self) -> Option { @@ -38,7 +36,7 @@ impl FromFfi for FixedSizeListArray { 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(); + let values = ffi::try_from(child)?; Self::try_new(data_type, values, validity) } diff --git a/src/array/fixed_size_list/mod.rs b/src/array/fixed_size_list/mod.rs index f8faa3598a2..25e84f03fc5 100644 --- a/src/array/fixed_size_list/mod.rs +++ b/src/array/fixed_size_list/mod.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{ bitmap::Bitmap, datatypes::{DataType, Field}, @@ -21,7 +19,7 @@ pub use mutable::*; pub struct FixedSizeListArray { size: usize, // this is redundant with `data_type`, but useful to not have to deconstruct the data_type. data_type: DataType, - values: Arc, + values: Box, validity: Option, } @@ -36,7 +34,7 @@ impl FixedSizeListArray { /// * the validity's length is not equal to `values.len() / size`. pub fn try_new( data_type: DataType, - values: Arc, + values: Box, validity: Option, ) -> Result { let (child, size) = Self::try_child_and_size(&data_type)?; @@ -82,14 +80,14 @@ impl FixedSizeListArray { /// * 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 { + pub fn new(data_type: DataType, values: Box, validity: Option) -> Self { Self::try_new(data_type, values, validity).unwrap() } /// Alias for `new` pub fn from_data( data_type: DataType, - values: Arc, + values: Box, validity: Option, ) -> Self { Self::new(data_type, values, validity) @@ -97,8 +95,7 @@ impl FixedSizeListArray { /// Returns a new empty [`FixedSizeListArray`]. pub fn new_empty(data_type: DataType) -> Self { - let values = - new_empty_array(Self::get_child_and_size(&data_type).0.data_type().clone()).into(); + let values = new_empty_array(Self::get_child_and_size(&data_type).0.data_type().clone()); Self::new(data_type, values, None) } @@ -107,8 +104,7 @@ impl FixedSizeListArray { let values = new_null_array( Self::get_child_and_size(&data_type).0.data_type().clone(), length, - ) - .into(); + ); Self::new(data_type, values, Some(Bitmap::new_zeroed(length))) } @@ -153,8 +149,7 @@ impl FixedSizeListArray { let values = self .values .clone() - .slice_unchecked(offset * self.size as usize, length * self.size as usize) - .into(); + .slice_unchecked(offset * self.size as usize, length * self.size as usize); Self { data_type: self.data_type.clone(), size: self.size, @@ -192,7 +187,7 @@ impl FixedSizeListArray { } /// Returns the inner array. - pub fn values(&self) -> &Arc { + pub fn values(&self) -> &Box { &self.values } @@ -242,6 +237,11 @@ impl Array for FixedSizeListArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/fixed_size_list/mutable.rs b/src/array/fixed_size_list/mutable.rs index a2e8c8d1758..1ffcd15c4dd 100644 --- a/src/array/fixed_size_list/mutable.rs +++ b/src/array/fixed_size_list/mutable.rs @@ -22,7 +22,7 @@ impl From> for FixedSizeListArray fn from(mut other: MutableFixedSizeListArray) -> Self { FixedSizeListArray::new( other.data_type, - other.values.as_arc(), + other.values.as_box(), other.validity.map(|x| x.into()), ) } @@ -117,7 +117,7 @@ impl MutableArray for MutableFixedSizeListArray { fn as_box(&mut self) -> Box { Box::new(FixedSizeListArray::new( self.data_type.clone(), - self.values.as_arc(), + self.values.as_box(), std::mem::take(&mut self.validity).map(|x| x.into()), )) } @@ -125,7 +125,7 @@ impl MutableArray for MutableFixedSizeListArray { fn as_arc(&mut self) -> Arc { Arc::new(FixedSizeListArray::new( self.data_type.clone(), - self.values.as_arc(), + self.values.as_box(), std::mem::take(&mut self.validity).map(|x| x.into()), )) } diff --git a/src/array/growable/dictionary.rs b/src/array/growable/dictionary.rs index e75514c66ae..908a1394fa9 100644 --- a/src/array/growable/dictionary.rs +++ b/src/array/growable/dictionary.rs @@ -20,7 +20,7 @@ pub struct GrowableDictionary<'a, K: DictionaryKey> { key_values: Vec, key_validity: MutableBitmap, offsets: Vec, - values: Arc, + values: Box, extend_null_bits: Vec>, } @@ -28,7 +28,7 @@ fn concatenate_values( arrays_keys: &[&PrimitiveArray], arrays_values: &[&dyn Array], capacity: usize, -) -> (Arc, Vec) { +) -> (Box, Vec) { let mut mutable = make_growable(arrays_values, false, capacity); let mut offsets = Vec::with_capacity(arrays_keys.len() + 1); offsets.push(0); @@ -36,7 +36,7 @@ fn concatenate_values( mutable.extend(i, 0, values.len()); offsets.push(offsets[i] + values.len()); } - (mutable.as_arc(), offsets) + (mutable.as_box(), offsets) } impl<'a, T: DictionaryKey> GrowableDictionary<'a, T> { @@ -81,10 +81,10 @@ impl<'a, T: DictionaryKey> GrowableDictionary<'a, T> { #[inline] fn to(&mut self) -> DictionaryArray { let validity = std::mem::take(&mut self.key_validity); - let values = std::mem::take(&mut self.key_values); + let key_values = std::mem::take(&mut self.key_values); let data_type = T::PRIMITIVE.into(); - let keys = PrimitiveArray::::from_data(data_type, values.into(), validity.into()); + let keys = PrimitiveArray::::from_data(data_type, key_values.into(), validity.into()); DictionaryArray::::from_data(keys, self.values.clone()) } diff --git a/src/array/growable/fixed_size_list.rs b/src/array/growable/fixed_size_list.rs index 1986120efe6..26344f4c96a 100644 --- a/src/array/growable/fixed_size_list.rs +++ b/src/array/growable/fixed_size_list.rs @@ -67,7 +67,7 @@ impl<'a> GrowableFixedSizeList<'a> { fn to(&mut self) -> FixedSizeListArray { let validity = std::mem::take(&mut self.validity); - let values = self.values.as_arc(); + let values = self.values.as_box(); FixedSizeListArray::new(self.arrays[0].data_type().clone(), values, validity.into()) } @@ -97,7 +97,7 @@ impl<'a> Growable<'a> for GrowableFixedSizeList<'a> { impl<'a> From> for FixedSizeListArray { fn from(val: GrowableFixedSizeList<'a>) -> Self { let mut values = val.values; - let values = values.as_arc(); + let values = values.as_box(); Self::new( val.arrays[0].data_type().clone(), diff --git a/src/array/growable/list.rs b/src/array/growable/list.rs index a7eb57eb2c4..41db9ae7892 100644 --- a/src/array/growable/list.rs +++ b/src/array/growable/list.rs @@ -102,7 +102,7 @@ impl<'a, O: Offset> GrowableList<'a, O> { fn to(&mut self) -> ListArray { let validity = std::mem::take(&mut self.validity); let offsets = std::mem::take(&mut self.offsets); - let values = self.values.as_arc(); + let values = self.values.as_box(); ListArray::::new( self.arrays[0].data_type().clone(), @@ -137,7 +137,7 @@ impl<'a, O: Offset> Growable<'a> for GrowableList<'a, O> { impl<'a, O: Offset> From> for ListArray { fn from(val: GrowableList<'a, O>) -> Self { let mut values = val.values; - let values = values.as_arc(); + let values = values.as_box(); ListArray::::new( val.arrays[0].data_type().clone(), diff --git a/src/array/growable/structure.rs b/src/array/growable/structure.rs index 4a7ac0669da..e7f997024de 100644 --- a/src/array/growable/structure.rs +++ b/src/array/growable/structure.rs @@ -66,7 +66,7 @@ impl<'a> GrowableStruct<'a> { fn to(&mut self) -> StructArray { let validity = std::mem::take(&mut self.validity); let values = std::mem::take(&mut self.values); - let values = values.into_iter().map(|mut x| x.as_arc()).collect(); + let values = values.into_iter().map(|mut x| x.as_box()).collect(); StructArray::new( DataType::Struct(self.arrays[0].fields().to_vec()), @@ -118,7 +118,7 @@ impl<'a> Growable<'a> for GrowableStruct<'a> { impl<'a> From> for StructArray { fn from(val: GrowableStruct<'a>) -> Self { - let values = val.values.into_iter().map(|mut x| x.as_arc()).collect(); + let values = val.values.into_iter().map(|mut x| x.as_box()).collect(); StructArray::new( DataType::Struct(val.arrays[0].fields().to_vec()), diff --git a/src/array/growable/union.rs b/src/array/growable/union.rs index c880c05ccf7..f050d1272e7 100644 --- a/src/array/growable/union.rs +++ b/src/array/growable/union.rs @@ -53,7 +53,7 @@ impl<'a> GrowableUnion<'a> { let types = std::mem::take(&mut self.types); let fields = std::mem::take(&mut self.fields); let offsets = std::mem::take(&mut self.offsets); - let fields = fields.into_iter().map(|mut x| x.as_arc()).collect(); + let fields = fields.into_iter().map(|mut x| x.as_box()).collect(); UnionArray::new( self.arrays[0].data_type().clone(), @@ -99,7 +99,7 @@ impl<'a> Growable<'a> for GrowableUnion<'a> { impl<'a> From> for UnionArray { fn from(val: GrowableUnion<'a>) -> Self { - let fields = val.fields.into_iter().map(|mut x| x.as_arc()).collect(); + let fields = val.fields.into_iter().map(|mut x| x.as_box()).collect(); UnionArray::new( val.arrays[0].data_type().clone(), diff --git a/src/array/list/ffi.rs b/src/array/list/ffi.rs index 67349792305..717ca063835 100644 --- a/src/array/list/ffi.rs +++ b/src/array/list/ffi.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{array::FromFfi, bitmap::align, error::Result, ffi}; use super::super::{ffi::ToFfi, Array, Offset}; @@ -13,7 +11,7 @@ unsafe impl ToFfi for ListArray { ] } - fn children(&self) -> Vec> { + fn children(&self) -> Vec> { vec![self.values.clone()] } @@ -56,7 +54,7 @@ impl FromFfi for ListArray { let validity = unsafe { array.validity() }?; let offsets = unsafe { array.buffer::(1) }?; let child = unsafe { array.child(0)? }; - let values = ffi::try_from(child)?.into(); + let values = ffi::try_from(child)?; Ok(Self::from_data(data_type, offsets, values, validity)) } diff --git a/src/array/list/mod.rs b/src/array/list/mod.rs index 0ef4d882808..451d90086ae 100644 --- a/src/array/list/mod.rs +++ b/src/array/list/mod.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{ bitmap::Bitmap, buffer::Buffer, @@ -25,7 +23,7 @@ pub use mutable::*; pub struct ListArray { data_type: DataType, offsets: Buffer, - values: Arc, + values: Box, validity: Option, } @@ -44,7 +42,7 @@ impl ListArray { pub fn try_new( data_type: DataType, offsets: Buffer, - values: Arc, + values: Box, validity: Option, ) -> Result { try_check_offsets(&offsets, values.len())?; @@ -88,7 +86,7 @@ impl ListArray { pub fn new( data_type: DataType, offsets: Buffer, - values: Arc, + values: Box, validity: Option, ) -> Self { Self::try_new(data_type, offsets, values, validity).unwrap() @@ -98,7 +96,7 @@ impl ListArray { pub fn from_data( data_type: DataType, offsets: Buffer, - values: Arc, + values: Box, validity: Option, ) -> Self { Self::new(data_type, offsets, values, validity) @@ -106,7 +104,7 @@ impl ListArray { /// 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(); + let values = new_empty_array(Self::get_child_type(&data_type).clone()); Self::new(data_type, Buffer::from(vec![O::zero()]), values, None) } @@ -117,7 +115,7 @@ impl ListArray { Self::new( data_type, vec![O::default(); 1 + length].into(), - new_empty_array(child).into(), + new_empty_array(child), Some(Bitmap::new_zeroed(length)), ) } @@ -151,7 +149,7 @@ impl ListArray { pub unsafe fn try_new_unchecked( data_type: DataType, offsets: Buffer, - values: Arc, + values: Box, validity: Option, ) -> Result { try_check_offsets_bounds(&offsets, values.len())?; @@ -197,7 +195,7 @@ impl ListArray { pub unsafe fn new_unchecked( data_type: DataType, offsets: Buffer, - values: Arc, + values: Box, validity: Option, ) -> Self { Self::try_new_unchecked(data_type, offsets, values, validity).unwrap() @@ -293,7 +291,7 @@ impl ListArray { /// The values. #[inline] - pub fn values(&self) -> &Arc { + pub fn values(&self) -> &Box { &self.values } } @@ -347,6 +345,11 @@ impl Array for ListArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/list/mutable.rs b/src/array/list/mutable.rs index 26bd29476a0..8ad2d9d80d2 100644 --- a/src/array/list/mutable.rs +++ b/src/array/list/mutable.rs @@ -56,7 +56,7 @@ impl From> for ListArray { ListArray::new_unchecked( other.data_type, other.offsets.into(), - other.values.as_arc(), + other.values.as_box(), other.validity.map(|x| x.into()), ) } @@ -215,27 +215,27 @@ impl MutableArray for MutableLis fn as_box(&mut self) -> Box { // Safety: // MutableListArray has monotonically increasing offsets - unsafe { - Box::new(ListArray::new_unchecked( + Box::new(unsafe { + ListArray::new_unchecked( self.data_type.clone(), std::mem::take(&mut self.offsets).into(), - self.values.as_arc(), + self.values.as_box(), std::mem::take(&mut self.validity).map(|x| x.into()), - )) - } + ) + }) } fn as_arc(&mut self) -> Arc { // Safety: // MutableListArray has monotonically increasing offsets - unsafe { - Arc::new(ListArray::new_unchecked( + Arc::new(unsafe { + ListArray::new_unchecked( self.data_type.clone(), std::mem::take(&mut self.offsets).into(), - self.values.as_arc(), + self.values.as_box(), std::mem::take(&mut self.validity).map(|x| x.into()), - )) - } + ) + }) } fn data_type(&self) -> &DataType { diff --git a/src/array/map/ffi.rs b/src/array/map/ffi.rs index 33333596a01..538da4cbc3d 100644 --- a/src/array/map/ffi.rs +++ b/src/array/map/ffi.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{array::FromFfi, bitmap::align, error::Result, ffi}; use super::super::{ffi::ToFfi, Array}; @@ -13,7 +11,7 @@ unsafe impl ToFfi for MapArray { ] } - fn children(&self) -> Vec> { + fn children(&self) -> Vec> { vec![self.field.clone()] } @@ -56,7 +54,7 @@ impl FromFfi for MapArray { let validity = unsafe { array.validity() }?; let offsets = unsafe { array.buffer::(1) }?; let child = array.child(0)?; - let values = ffi::try_from(child)?.into(); + let values = ffi::try_from(child)?; Self::try_new(data_type, offsets, values, validity) } diff --git a/src/array/map/mod.rs b/src/array/map/mod.rs index f8b754d6189..015b9feab81 100644 --- a/src/array/map/mod.rs +++ b/src/array/map/mod.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{ bitmap::Bitmap, buffer::Buffer, @@ -20,7 +18,7 @@ pub struct MapArray { data_type: DataType, // invariant: field.len() == offsets.len() - 1 offsets: Buffer, - field: Arc, + field: Box, // invariant: offsets.len() - 1 == Bitmap::len() validity: Option, } @@ -37,7 +35,7 @@ impl MapArray { pub fn try_new( data_type: DataType, offsets: Buffer, - field: Arc, + field: Box, validity: Option, ) -> Result { try_check_offsets(&offsets, field.len())?; @@ -86,7 +84,7 @@ impl MapArray { pub fn new( data_type: DataType, offsets: Buffer, - field: Arc, + field: Box, validity: Option, ) -> Self { Self::try_new(data_type, offsets, field, validity).unwrap() @@ -96,7 +94,7 @@ impl MapArray { pub fn from_data( data_type: DataType, offsets: Buffer, - field: Arc, + field: Box, validity: Option, ) -> Self { Self::new(data_type, offsets, field, validity) @@ -104,7 +102,7 @@ impl MapArray { /// Returns a new null [`MapArray`] of `length`. pub fn new_null(data_type: DataType, length: usize) -> Self { - let field = new_empty_array(Self::get_field(&data_type).data_type().clone()).into(); + let field = new_empty_array(Self::get_field(&data_type).data_type().clone()); Self::new( data_type, vec![0i32; 1 + length].into(), @@ -115,7 +113,7 @@ impl MapArray { /// Returns a new empty [`MapArray`]. pub fn new_empty(data_type: DataType) -> Self { - let field = new_empty_array(Self::get_field(&data_type).data_type().clone()).into(); + let field = new_empty_array(Self::get_field(&data_type).data_type().clone()); Self::new(data_type, Buffer::from(vec![0i32]), field, None) } @@ -190,7 +188,7 @@ impl MapArray { /// Returns the field (guaranteed to be a `Struct`) #[inline] - pub fn field(&self) -> &Arc { + pub fn field(&self) -> &Box { &self.field } @@ -226,6 +224,11 @@ impl Array for MapArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/mod.rs b/src/array/mod.rs index 2a8806f0787..e004bb0e778 100644 --- a/src/array/mod.rs +++ b/src/array/mod.rs @@ -28,10 +28,13 @@ pub(self) mod physical_binary; /// A trait representing an immutable Arrow array. Arrow arrays are trait objects /// that are infallibly downcasted to concrete types according to the [`Array::data_type`]. -pub trait Array: Send + Sync { - /// Convert to trait object. +pub trait Array: Send + Sync + dyn_clone::DynClone + 'static { + /// Converts itself to a reference of [`Any`], which enables downcasting to concrete types. fn as_any(&self) -> &dyn Any; + /// Converts itself to a mutable reference of [`Any`], which enables mutable downcasting to concrete types. + fn as_any_mut(&mut self) -> &mut dyn Any; + /// The length of the [`Array`]. Every array has a length corresponding to the number of /// elements (slots). fn len(&self) -> usize; @@ -108,6 +111,8 @@ pub trait Array: Send + Sync { fn to_boxed(&self) -> Box; } +dyn_clone::clone_trait_object!(Array); + /// A trait describing a mutable array; i.e. an array whose values can be changed. /// Mutable arrays cannot be cloned but can be mutated in place, /// thereby making them useful to perform numeric operations without allocations. @@ -134,7 +139,7 @@ pub trait MutableArray: std::fmt::Debug + Send + Sync { // This provided implementation has an extra allocation as it first // boxes `self`, then converts the box into an `Arc`. Implementors may wish // to avoid an allocation by skipping the box completely. - fn as_arc(&mut self) -> Arc { + fn as_arc(&mut self) -> std::sync::Arc { self.as_box().into() } @@ -422,9 +427,3 @@ pub unsafe trait GenericBinaryArray: Array { /// The offsets of the array fn offsets(&self) -> &[O]; } - -// backward compatibility -use std::sync::Arc; - -/// A type def of [`Array`]. -pub type ArrayRef = Arc; diff --git a/src/array/null.rs b/src/array/null.rs index 57c9f00210a..5b0ee9fd4b5 100644 --- a/src/array/null.rs +++ b/src/array/null.rs @@ -84,6 +84,11 @@ impl Array for NullArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/primitive/mod.rs b/src/array/primitive/mod.rs index cc419c06458..b5eee572511 100644 --- a/src/array/primitive/mod.rs +++ b/src/array/primitive/mod.rs @@ -53,6 +53,28 @@ pub struct PrimitiveArray { validity: Option, } +fn check( + data_type: &DataType, + values: &[T], + validity: &Option, +) -> Result<(), Error> { + if validity + .as_ref() + .map_or(false, |validity| validity.len() != values.len()) + { + return Err(Error::oos( + "validity mask length must match the number of values", + )); + } + + if data_type.to_physical_type() != PhysicalType::Primitive(T::PRIMITIVE) { + return Err(Error::oos( + "BooleanArray can only be initialized with a DataType whose physical type is Primitive", + )); + } + Ok(()) +} + impl PrimitiveArray { /// The canonical method to create a [`PrimitiveArray`] out of its internal components. /// # Implementation @@ -67,21 +89,7 @@ impl PrimitiveArray { values: Buffer, validity: Option, ) -> Result { - if validity - .as_ref() - .map_or(false, |validity| validity.len() != values.len()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - if data_type.to_physical_type() != PhysicalType::Primitive(T::PRIMITIVE) { - return Err(Error::oos( - "BooleanArray can only be initialized with a DataType whose physical type is Primitive", - )); - } - + check(&data_type, &values, &validity)?; Ok(Self { data_type, values, @@ -109,14 +117,7 @@ impl PrimitiveArray { #[inline] #[must_use] pub fn to(self, data_type: DataType) -> Self { - if !data_type.to_physical_type().eq_primitive(T::PRIMITIVE) { - Err(Error::InvalidArgumentError(format!( - "Type {} does not support logical type {:?}", - std::any::type_name::(), - data_type - ))) - .unwrap() - } + check(&data_type, &self.values, &self.validity).unwrap(); Self { data_type, values: self.values, @@ -420,6 +421,11 @@ impl Array for PrimitiveArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.values.len() diff --git a/src/array/struct_/ffi.rs b/src/array/struct_/ffi.rs index 9f24db5253f..61b36ca1b82 100644 --- a/src/array/struct_/ffi.rs +++ b/src/array/struct_/ffi.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use super::super::{ffi::ToFfi, Array, FromFfi}; use super::StructArray; use crate::{error::Result, ffi}; @@ -9,7 +7,7 @@ unsafe impl ToFfi for StructArray { vec![self.validity.as_ref().map(|x| x.as_ptr())] } - fn children(&self) -> Vec> { + fn children(&self) -> Vec> { self.values.clone() } @@ -36,9 +34,9 @@ impl FromFfi for StructArray { let values = (0..fields.len()) .map(|index| { let child = array.child(index)?; - Ok(ffi::try_from(child)?.into()) + ffi::try_from(child) }) - .collect::>>>()?; + .collect::>>>()?; Self::try_new(data_type, values, validity) } diff --git a/src/array/struct_/mod.rs b/src/array/struct_/mod.rs index 4736a23262a..cb888ce8721 100644 --- a/src/array/struct_/mod.rs +++ b/src/array/struct_/mod.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{ bitmap::Bitmap, datatypes::{DataType, Field}, @@ -16,11 +14,10 @@ mod iterator; /// multiple [`Array`] with the same number of rows. /// # Example /// ``` -/// use std::sync::Arc; /// use arrow2::array::*; /// use arrow2::datatypes::*; -/// let boolean = BooleanArray::from_slice(&[false, false, true, true]).arced(); -/// let int = Int32Array::from_slice(&[42, 28, 19, 31]).arced(); +/// let boolean = BooleanArray::from_slice(&[false, false, true, true]).boxed(); +/// let int = Int32Array::from_slice(&[42, 28, 19, 31]).boxed(); /// /// let fields = vec![ /// Field::new("b", DataType::Boolean, false), @@ -32,7 +29,7 @@ mod iterator; #[derive(Clone)] pub struct StructArray { data_type: DataType, - values: Vec>, + values: Vec>, validity: Option, } @@ -48,7 +45,7 @@ impl StructArray { /// * the validity's length is not equal to the length of the first element pub fn try_new( data_type: DataType, - values: Vec>, + values: Vec>, validity: Option, ) -> Result { let fields = Self::try_get_fields(&data_type)?; @@ -117,14 +114,14 @@ impl StructArray { /// * 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 { + 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>, + values: Vec>, validity: Option, ) -> Self { Self::new(data_type, values, validity) @@ -135,7 +132,7 @@ impl StructArray { if let DataType::Struct(fields) = &data_type { let values = fields .iter() - .map(|field| new_empty_array(field.data_type().clone()).into()) + .map(|field| new_empty_array(field.data_type().clone())) .collect(); Self::new(data_type, values, None) } else { @@ -148,7 +145,7 @@ impl StructArray { if let DataType::Struct(fields) = &data_type { let values = fields .iter() - .map(|field| new_null_array(field.data_type().clone(), length).into()) + .map(|field| new_null_array(field.data_type().clone(), length)) .collect(); Self::new(data_type, values, Some(Bitmap::new_zeroed(length))) } else { @@ -161,7 +158,7 @@ impl StructArray { impl StructArray { /// Deconstructs the [`StructArray`] into its individual components. #[must_use] - pub fn into_data(self) -> (Vec, Vec>, Option) { + pub fn into_data(self) -> (Vec, Vec>, Option) { let Self { data_type, values, @@ -205,7 +202,7 @@ impl StructArray { values: self .values .iter() - .map(|x| x.slice_unchecked(offset, length).into()) + .map(|x| x.slice_unchecked(offset, length)) .collect(), validity, } @@ -249,7 +246,7 @@ impl StructArray { } /// Returns the values of this [`StructArray`]. - pub fn values(&self) -> &[Arc] { + pub fn values(&self) -> &[Box] { &self.values } @@ -282,6 +279,11 @@ impl Array for StructArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/array/union/ffi.rs b/src/array/union/ffi.rs index f7c9292c605..94bcdea42c8 100644 --- a/src/array/union/ffi.rs +++ b/src/array/union/ffi.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{array::FromFfi, error::Result, ffi}; use super::super::{ffi::ToFfi, Array}; @@ -17,7 +15,7 @@ unsafe impl ToFfi for UnionArray { } } - fn children(&self) -> Vec> { + fn children(&self) -> Vec> { self.fields.clone() } @@ -47,9 +45,9 @@ impl FromFfi for UnionArray { let fields = (0..fields.len()) .map(|index| { let child = array.child(index)?; - Ok(ffi::try_from(child)?.into()) + ffi::try_from(child) }) - .collect::>>>()?; + .collect::>>>()?; if offset > 0 { types = types.slice(offset, length); diff --git a/src/array/union/mod.rs b/src/array/union/mod.rs index f008f15af70..6d605da064a 100644 --- a/src/array/union/mod.rs +++ b/src/array/union/mod.rs @@ -1,4 +1,4 @@ -use std::{collections::HashMap, sync::Arc}; +use std::collections::HashMap; use crate::{ bitmap::Bitmap, @@ -14,7 +14,7 @@ mod ffi; pub(super) mod fmt; mod iterator; -type FieldEntry = (usize, Arc); +type FieldEntry = (usize, Box); type UnionComponents<'a> = (&'a [Field], Option<&'a [i32]>, UnionMode); /// [`UnionArray`] represents an array whose each slot can contain different values. @@ -32,7 +32,7 @@ pub struct UnionArray { types: Buffer, // None represents when there is no typeid fields_hash: Option>, - fields: Vec>, + fields: Vec>, offsets: Option>, data_type: DataType, offset: usize, @@ -48,7 +48,7 @@ impl UnionArray { pub fn try_new( data_type: DataType, types: Buffer, - fields: Vec>, + fields: Vec>, offsets: Option>, ) -> Result { let (f, ids, mode) = Self::try_get_all(&data_type)?; @@ -111,7 +111,7 @@ impl UnionArray { pub fn new( data_type: DataType, types: Buffer, - fields: Vec>, + fields: Vec>, offsets: Option>, ) -> Self { Self::try_new(data_type, types, fields, offsets).unwrap() @@ -121,7 +121,7 @@ impl UnionArray { pub fn from_data( data_type: DataType, types: Buffer, - fields: Vec>, + fields: Vec>, offsets: Option>, ) -> Self { Self::new(data_type, types, fields, offsets) @@ -132,7 +132,7 @@ impl UnionArray { if let DataType::Union(f, _, mode) = &data_type { let fields = f .iter() - .map(|x| new_null_array(x.data_type().clone(), length).into()) + .map(|x| new_null_array(x.data_type().clone(), length)) .collect(); let offsets = if mode.is_sparse() { @@ -155,7 +155,7 @@ impl UnionArray { if let DataType::Union(f, _, mode) = &data_type { let fields = f .iter() - .map(|x| new_empty_array(x.data_type().clone()).into()) + .map(|x| new_empty_array(x.data_type().clone())) .collect(); let offsets = if mode.is_sparse() { @@ -243,7 +243,7 @@ impl UnionArray { } /// The fields. - pub fn fields(&self) -> &Vec> { + pub fn fields(&self) -> &Vec> { &self.fields } @@ -253,11 +253,11 @@ impl UnionArray { } #[inline] - fn field(&self, type_: i8) -> &Arc { + fn field(&self, type_: i8) -> &dyn Array { self.fields_hash .as_ref() - .map(|x| &x[&type_].1) - .unwrap_or_else(|| &self.fields[type_ as usize]) + .map(|x| x[&type_].1.as_ref()) + .unwrap_or_else(|| self.fields[type_ as usize].as_ref()) } #[inline] @@ -285,7 +285,7 @@ impl UnionArray { let type_ = self.types()[index]; let field = self.field(type_); let index = self.field_slot(index); - new_scalar(field.as_ref(), index) + new_scalar(field, index) } } @@ -294,6 +294,11 @@ impl Array for UnionArray { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + fn len(&self) -> usize { self.len() } diff --git a/src/array/utf8/mod.rs b/src/array/utf8/mod.rs index 5d89fe4aabb..047e8778c0b 100644 --- a/src/array/utf8/mod.rs +++ b/src/array/utf8/mod.rs @@ -554,6 +554,11 @@ impl Array for Utf8Array { self } + #[inline] + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } + #[inline] fn len(&self) -> usize { self.len() diff --git a/src/buffer/immutable.rs b/src/buffer/immutable.rs index 726099b14d1..d914c8aa715 100644 --- a/src/buffer/immutable.rs +++ b/src/buffer/immutable.rs @@ -137,7 +137,7 @@ impl Buffer { self.offset } - /// Converts this [`Buffer`] to [`Vec`], returning itself if the conversion + /// Converts this [`Buffer`] to either a [`Buffer`] or a [`Vec`], returning itself if the conversion /// is not possible /// /// This operation returns a [`Vec`] iff this [`Buffer`]: diff --git a/src/compute/cast/dictionary_to.rs b/src/compute/cast/dictionary_to.rs index 87c3bd6a103..16b1c06f105 100644 --- a/src/compute/cast/dictionary_to.rs +++ b/src/compute/cast/dictionary_to.rs @@ -32,7 +32,7 @@ pub fn dictionary_to_dictionary_values( let keys = from.keys(); let values = from.values(); - let values = cast(values.as_ref(), values_type, CastOptions::default())?.into(); + let values = cast(values.as_ref(), values_type, CastOptions::default())?; Ok(DictionaryArray::from_data(keys.clone(), values)) } @@ -51,8 +51,7 @@ pub fn wrapping_dictionary_to_dictionary_values( wrapped: true, partial: false, }, - )? - .into(); + )?; Ok(DictionaryArray::from_data(keys.clone(), values)) } @@ -111,7 +110,7 @@ pub(super) fn dictionary_cast_dyn( match to_type { DataType::Dictionary(to_keys_type, to_values_type, _) => { - let values = cast(values.as_ref(), to_values_type, options)?.into(); + let values = cast(values.as_ref(), to_values_type, options)?; // create the appropriate array type let data_type = (*to_keys_type).into(); diff --git a/src/compute/cast/mod.rs b/src/compute/cast/mod.rs index 2cae7a21923..de3fcff43d8 100644 --- a/src/compute/cast/mod.rs +++ b/src/compute/cast/mod.rs @@ -296,8 +296,7 @@ fn cast_list( values.as_ref(), ListArray::::get_child_type(to_type), options, - )? - .into(); + )?; Ok(ListArray::::new( to_type.clone(), @@ -398,7 +397,7 @@ pub fn cast(array: &dyn Array, to_type: &DataType, options: CastOptions) -> Resu (_, List(to)) => { // cast primitive to list's primitive - let values = cast(array, &to.data_type, options)?.into(); + let values = cast(array, &to.data_type, options)?; // create offsets, where if array.len() = 2, we have [0,1,2] let offsets = (0..=array.len() as i32).collect::>(); diff --git a/src/compute/take/structure.rs b/src/compute/take/structure.rs index 2b94e8e43ab..1f7c4e4a07f 100644 --- a/src/compute/take/structure.rs +++ b/src/compute/take/structure.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - use crate::{ array::{Array, PrimitiveArray, StructArray}, bitmap::{Bitmap, MutableBitmap}, @@ -54,10 +52,10 @@ fn take_validity( } pub fn take(array: &StructArray, indices: &PrimitiveArray) -> Result { - let values: Vec> = array + let values: Vec> = array .values() .iter() - .map(|a| super::take(a.as_ref(), indices).map(|x| x.into())) + .map(|a| super::take(a.as_ref(), indices)) .collect::>()?; let validity = take_validity(array.validity(), indices)?; Ok(StructArray::new( diff --git a/src/ffi/array.rs b/src/ffi/array.rs index 5ebdf5d7fcd..08ae9b68bf2 100644 --- a/src/ffi/array.rs +++ b/src/ffi/array.rs @@ -85,7 +85,7 @@ unsafe extern "C" fn c_release_array(array: *mut ArrowArray) { #[allow(dead_code)] struct PrivateData { - array: Arc, + array: Box, buffers_ptr: Box<[*const std::os::raw::c_void]>, children_ptr: Box<[*mut ArrowArray]>, dictionary_ptr: Option<*mut ArrowArray>, @@ -96,7 +96,7 @@ impl ArrowArray { /// # 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 { + pub(crate) fn new(array: Box) -> Self { let (offset, buffers, children, dictionary) = offset_buffers_children_dictionary(array.as_ref()); diff --git a/src/ffi/bridge.rs b/src/ffi/bridge.rs index 1a51e56e780..9a098cc8b2c 100644 --- a/src/ffi/bridge.rs +++ b/src/ffi/bridge.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::array::*; macro_rules! ffi_dyn { @@ -8,12 +6,12 @@ macro_rules! ffi_dyn { if a.offset().is_some() { $array } else { - Arc::new(a.to_ffi_aligned()) + Box::new(a.to_ffi_aligned()) } }}; } -pub fn align_to_c_data_interface(array: Arc) -> Arc { +pub fn align_to_c_data_interface(array: Box) -> Box { use crate::datatypes::PhysicalType::*; match array.data_type().to_physical_type() { Null => ffi_dyn!(array, NullArray), diff --git a/src/ffi/mod.rs b/src/ffi/mod.rs index f3d689b7f8c..4a0c82a0557 100644 --- a/src/ffi/mod.rs +++ b/src/ffi/mod.rs @@ -20,10 +20,10 @@ 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. +/// Exports an [`Box`] 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 ArrowArray) { +pub unsafe fn export_array_to_c(array: Box, ptr: *mut ArrowArray) { let array = bridge::align_to_c_data_interface(array); std::ptr::write_unaligned(ptr, ArrowArray::new(array)); diff --git a/src/ffi/stream.rs b/src/ffi/stream.rs index 69e102da3ae..ab128e4626f 100644 --- a/src/ffi/stream.rs +++ b/src/ffi/stream.rs @@ -1,5 +1,4 @@ use std::ffi::{CStr, CString}; -use std::sync::Arc; use crate::{array::Array, datatypes::Field, error::Error}; @@ -125,7 +124,7 @@ impl ArrowArrayStreamReader { } struct PrivateData { - iter: Box, Error>>>, + iter: Box, Error>>>, field: Field, error: Option, } @@ -199,7 +198,7 @@ unsafe extern "C" fn release(iter: *mut ArrowArrayStream) { /// # Safety /// The pointer `consumer` must be allocated pub unsafe fn export_iterator( - iter: Box, Error>>>, + iter: Box, Error>>>, field: Field, consumer: *mut ArrowArrayStream, ) { diff --git a/src/io/avro/read/deserialize.rs b/src/io/avro/read/deserialize.rs index d77ceb96b88..1bce0fb52e0 100644 --- a/src/io/avro/read/deserialize.rs +++ b/src/io/avro/read/deserialize.rs @@ -1,5 +1,4 @@ use std::convert::TryInto; -use std::sync::Arc; use avro_schema::Record; use avro_schema::{Enum, Schema as AvroSchema}; @@ -444,7 +443,7 @@ pub fn deserialize( fields: &[Field], avro_schemas: &[AvroSchema], projection: &[bool], -) -> Result>> { +) -> Result>> { let rows = block.number_of_rows; let mut block = block.data.as_ref(); @@ -484,7 +483,7 @@ pub fn deserialize( .iter_mut() .zip(projection.iter()) .filter_map(|x| if *x.1 { Some(x.0) } else { None }) - .map(|array| array.as_arc()) + .map(|array| array.as_box()) .collect(), ) } diff --git a/src/io/avro/read/mod.rs b/src/io/avro/read/mod.rs index e437b4ef2da..f02af4052ee 100644 --- a/src/io/avro/read/mod.rs +++ b/src/io/avro/read/mod.rs @@ -1,6 +1,5 @@ //! APIs to read from Avro format to arrow. use std::io::Read; -use std::sync::Arc; use avro_schema::{Record, Schema as AvroSchema}; use fallible_streaming_iterator::FallibleStreamingIterator; @@ -75,7 +74,7 @@ impl Reader { } impl Iterator for Reader { - type Item = Result>>; + type Item = Result>>; fn next(&mut self) -> Option { let fields = &self.fields[..]; diff --git a/src/io/avro/read/nested.rs b/src/io/avro/read/nested.rs index daa07fd911c..e6138e74d33 100644 --- a/src/io/avro/read/nested.rs +++ b/src/io/avro/read/nested.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::array::*; use crate::bitmap::*; use crate::datatypes::*; @@ -83,16 +81,16 @@ impl MutableArray for DynMutableListArray { Box::new(ListArray::new( self.data_type.clone(), std::mem::take(&mut self.offsets).into(), - self.values.as_arc(), + self.values.as_box(), std::mem::take(&mut self.validity).map(|x| x.into()), )) } - fn as_arc(&mut self) -> Arc { - Arc::new(ListArray::new( + fn as_arc(&mut self) -> std::sync::Arc { + std::sync::Arc::new(ListArray::new( self.data_type.clone(), std::mem::take(&mut self.offsets).into(), - self.values.as_arc(), + self.values.as_box(), std::mem::take(&mut self.validity).map(|x| x.into()), )) } @@ -161,14 +159,14 @@ impl MutableArray for FixedItemsUtf8Dictionary { fn as_box(&mut self) -> Box { Box::new(DictionaryArray::from_data( std::mem::take(&mut self.keys).into(), - Arc::new(self.values.clone()), + Box::new(self.values.clone()), )) } - fn as_arc(&mut self) -> Arc { - Arc::new(DictionaryArray::from_data( + fn as_arc(&mut self) -> std::sync::Arc { + std::sync::Arc::new(DictionaryArray::from_data( std::mem::take(&mut self.keys).into(), - Arc::new(self.values.clone()), + Box::new(self.values.clone()), )) } @@ -245,7 +243,7 @@ impl MutableArray for DynMutableStructArray { } fn as_box(&mut self) -> Box { - let values = self.values.iter_mut().map(|x| x.as_arc()).collect(); + let values = self.values.iter_mut().map(|x| x.as_box()).collect(); Box::new(StructArray::new( self.data_type.clone(), @@ -254,10 +252,10 @@ impl MutableArray for DynMutableStructArray { )) } - fn as_arc(&mut self) -> Arc { - let values = self.values.iter_mut().map(|x| x.as_arc()).collect(); + fn as_arc(&mut self) -> std::sync::Arc { + let values = self.values.iter_mut().map(|x| x.as_box()).collect(); - Arc::new(StructArray::new( + std::sync::Arc::new(StructArray::new( self.data_type.clone(), values, std::mem::take(&mut self.validity).map(|x| x.into()), diff --git a/src/io/csv/read/deserialize.rs b/src/io/csv/read/deserialize.rs index 2a86e97c167..e819b0a7484 100644 --- a/src/io/csv/read/deserialize.rs +++ b/src/io/csv/read/deserialize.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use csv::ByteRecord; use crate::{ @@ -27,7 +25,7 @@ pub fn deserialize_column( column: usize, datatype: DataType, line_number: usize, -) -> Result> { +) -> Result> { deserialize_column_gen(rows, column, datatype, line_number) } @@ -40,9 +38,9 @@ pub fn deserialize_batch( projection: Option<&[usize]>, line_number: usize, deserialize_column: F, -) -> Result>> +) -> Result>> where - F: Fn(&[ByteRecord], usize, DataType, usize) -> Result>, + F: Fn(&[ByteRecord], usize, DataType, usize) -> Result>, { deserialize_batch_gen(rows, fields, projection, line_number, deserialize_column) } diff --git a/src/io/csv/read_async/deserialize.rs b/src/io/csv/read_async/deserialize.rs index be64b7398b3..c327c15282e 100644 --- a/src/io/csv/read_async/deserialize.rs +++ b/src/io/csv/read_async/deserialize.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use csv_async::ByteRecord; use crate::{ @@ -27,7 +25,7 @@ pub fn deserialize_column( column: usize, datatype: DataType, line_number: usize, -) -> Result> { +) -> Result> { deserialize_column_gen(rows, column, datatype, line_number) } @@ -40,9 +38,9 @@ pub fn deserialize_batch( projection: Option<&[usize]>, line_number: usize, deserialize_column: F, -) -> Result>> +) -> Result>> where - F: Fn(&[ByteRecord], usize, DataType, usize) -> Result>, + F: Fn(&[ByteRecord], usize, DataType, usize) -> Result>, { deserialize_batch_gen(rows, fields, projection, line_number, deserialize_column) } diff --git a/src/io/csv/read_utils.rs b/src/io/csv/read_utils.rs index 75ded615129..5406e94d59b 100644 --- a/src/io/csv/read_utils.rs +++ b/src/io/csv/read_utils.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use chrono::Datelike; // Ideally this trait should not be needed and both `csv` and `csv_async` crates would share @@ -31,7 +29,7 @@ fn deserialize_primitive( column: usize, datatype: DataType, op: F, -) -> Arc +) -> Box where T: NativeType + lexical_core::FromLexical, F: Fn(&[u8]) -> Option, @@ -45,7 +43,7 @@ where } None => None, }); - Arc::new(PrimitiveArray::::from_trusted_len_iter(iter).to(datatype)) + Box::new(PrimitiveArray::::from_trusted_len_iter(iter).to(datatype)) } #[inline] @@ -93,7 +91,7 @@ fn deserialize_decimal(bytes: &[u8], precision: usize, scale: usize) -> Option(rows: &[B], column: usize, op: F) -> Arc +fn deserialize_boolean(rows: &[B], column: usize, op: F) -> Box where B: ByteRecordGeneric, F: Fn(&[u8]) -> Option, @@ -107,25 +105,25 @@ where } None => None, }); - Arc::new(BooleanArray::from_trusted_len_iter(iter)) + Box::new(BooleanArray::from_trusted_len_iter(iter)) } #[inline] -fn deserialize_utf8(rows: &[B], column: usize) -> Arc { +fn deserialize_utf8(rows: &[B], column: usize) -> Box { let iter = rows.iter().map(|row| match row.get(column) { Some(bytes) => to_utf8(bytes), None => None, }); - Arc::new(Utf8Array::::from_trusted_len_iter(iter)) + Box::new(Utf8Array::::from_trusted_len_iter(iter)) } #[inline] fn deserialize_binary( rows: &[B], column: usize, -) -> Arc { +) -> Box { let iter = rows.iter().map(|row| row.get(column)); - Arc::new(BinaryArray::::from_trusted_len_iter(iter)) + Box::new(BinaryArray::::from_trusted_len_iter(iter)) } #[inline] @@ -151,7 +149,7 @@ pub(crate) fn deserialize_column( column: usize, datatype: DataType, _line_number: usize, -) -> Result> { +) -> Result> { use DataType::*; Ok(match datatype { Boolean => deserialize_boolean(rows, column, |bytes| { @@ -266,9 +264,9 @@ pub(crate) fn deserialize_batch( projection: Option<&[usize]>, line_number: usize, deserialize_column: F, -) -> Result>> +) -> Result>> where - F: Fn(&[B], usize, DataType, usize) -> Result>, + F: Fn(&[B], usize, DataType, usize) -> Result>, { let projection: Vec = match projection { Some(v) => v.to_vec(), diff --git a/src/io/flight/mod.rs b/src/io/flight/mod.rs index 4cb19390100..524b3c140e0 100644 --- a/src/io/flight/mod.rs +++ b/src/io/flight/mod.rs @@ -1,5 +1,4 @@ //! Serialization and deserialization to Arrow's flight protocol -use std::sync::Arc; use arrow_format::flight::data::{FlightData, SchemaResult}; use arrow_format::ipc; @@ -21,7 +20,7 @@ use super::ipc::{IpcField, IpcSchema}; /// Serializes [`Chunk`] to a vector of [`FlightData`] representing the serialized dictionaries /// and a [`FlightData`] representing the batch. pub fn serialize_batch( - columns: &Chunk>, + columns: &Chunk>, fields: &[IpcField], options: &WriteOptions, ) -> (Vec, FlightData) { @@ -114,7 +113,7 @@ pub fn deserialize_batch( fields: &[Field], ipc_schema: &IpcSchema, dictionaries: &read::Dictionaries, -) -> Result>> { +) -> Result>> { // check that the data_header is a record batch message let message = arrow_format::ipc::MessageRef::read_as_root(&data.data_header) .map_err(|err| Error::OutOfSpec(format!("Unable to get root as message: {:?}", err)))?; diff --git a/src/io/ipc/mod.rs b/src/io/ipc/mod.rs index 6877a5574bd..3fa3eb01a6b 100644 --- a/src/io/ipc/mod.rs +++ b/src/io/ipc/mod.rs @@ -31,7 +31,6 @@ //! ``` //! use arrow2::io::ipc::{{read::{FileReader, read_file_metadata}}, {write::{FileWriter, WriteOptions}}}; //! # use std::fs::File; -//! # use std::sync::Arc; //! # use arrow2::datatypes::{Field, Schema, DataType}; //! # use arrow2::array::{Int32Array, Array}; //! # use arrow2::chunk::Chunk; @@ -48,7 +47,7 @@ //! // Setup the data //! let x_data = Int32Array::from_slice([-1i32, 1]); //! let y_data = Int32Array::from_slice([1i32, -1]); -//! let chunk = Chunk::try_new(vec![x_data.arced(), y_data.arced()])?; +//! let chunk = Chunk::try_new(vec![x_data.boxed(), y_data.boxed()])?; //! //! // Write the messages and finalize the stream //! for _ in 0..5 { diff --git a/src/io/ipc/read/common.rs b/src/io/ipc/read/common.rs index 64517ff813b..1190b158879 100644 --- a/src/io/ipc/read/common.rs +++ b/src/io/ipc/read/common.rs @@ -1,6 +1,5 @@ use std::collections::{HashMap, HashSet, VecDeque}; use std::io::{Read, Seek}; -use std::sync::Arc; use arrow_format; @@ -13,8 +12,6 @@ use crate::io::ipc::{IpcField, IpcSchema}; use super::deserialize::{read, skip}; use super::Dictionaries; -type ArrayRef = Arc; - #[derive(Debug, Eq, PartialEq, Hash)] enum ProjectionResult { Selected(A), @@ -86,7 +83,7 @@ pub fn read_record_batch( version: arrow_format::ipc::MetadataVersion, reader: &mut R, block_offset: u64, -) -> Result>> { +) -> Result>> { assert_eq!(fields.len(), ipc_schema.fields.len()); let buffers = batch .buffers()? @@ -221,7 +218,7 @@ pub fn read_dictionary( // As the dictionary batch does not contain the type of the // values array, we need to retrieve this from the schema. // Get an array representing this dictionary's values. - let dictionary_values: ArrayRef = match &first_field.data_type { + let dictionary_values: Box = match &first_field.data_type { DataType::Dictionary(_, ref value_type, _) => { // Make a fake schema for the dictionary batch. let fields = vec![Field::new("", value_type.as_ref().clone(), false)]; @@ -301,10 +298,10 @@ pub fn prepare_projection( } pub fn apply_projection( - chunk: Chunk>, + chunk: Chunk>, projection: &[usize], map: &HashMap, -) -> Chunk> { +) -> Chunk> { // re-order according to projection let arrays = chunk.into_arrays(); let arrays = projection diff --git a/src/io/ipc/read/deserialize.rs b/src/io/ipc/read/deserialize.rs index 7873f0b9f20..f5c8b97c00f 100644 --- a/src/io/ipc/read/deserialize.rs +++ b/src/io/ipc/read/deserialize.rs @@ -1,8 +1,5 @@ use std::collections::VecDeque; -use std::{ - io::{Read, Seek}, - sync::Arc, -}; +use std::io::{Read, Seek}; use arrow_format::ipc::BodyCompressionRef; use arrow_format::ipc::MetadataVersion; @@ -27,12 +24,12 @@ pub fn read( is_little_endian: bool, compression: Option, version: MetadataVersion, -) -> Result> { +) -> Result> { use PhysicalType::*; let data_type = field.data_type.clone(); match data_type.to_physical_type() { - Null => read_null(field_nodes, data_type).map(|x| x.arced()), + Null => read_null(field_nodes, data_type).map(|x| x.boxed()), Boolean => read_boolean( field_nodes, data_type, @@ -42,7 +39,7 @@ pub fn read( is_little_endian, compression, ) - .map(|x| x.arced()), + .map(|x| x.boxed()), Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { read_primitive::<$T, _>( field_nodes, @@ -53,7 +50,7 @@ pub fn read( is_little_endian, compression, ) - .map(|x| x.arced()) + .map(|x| x.boxed()) }), Binary => { let array = read_binary::( @@ -65,7 +62,7 @@ pub fn read( is_little_endian, compression, )?; - Ok(Arc::new(array)) + Ok(Box::new(array)) } LargeBinary => { let array = read_binary::( @@ -77,7 +74,7 @@ pub fn read( is_little_endian, compression, )?; - Ok(Arc::new(array)) + Ok(Box::new(array)) } FixedSizeBinary => { let array = read_fixed_size_binary( @@ -89,7 +86,7 @@ pub fn read( is_little_endian, compression, )?; - Ok(Arc::new(array)) + Ok(Box::new(array)) } Utf8 => { let array = read_utf8::( @@ -101,7 +98,7 @@ pub fn read( is_little_endian, compression, )?; - Ok(Arc::new(array)) + Ok(Box::new(array)) } LargeUtf8 => { let array = read_utf8::( @@ -113,7 +110,7 @@ pub fn read( is_little_endian, compression, )?; - Ok(Arc::new(array)) + Ok(Box::new(array)) } List => read_list::( field_nodes, @@ -127,7 +124,7 @@ pub fn read( compression, version, ) - .map(|x| x.arced()), + .map(|x| x.boxed()), LargeList => read_list::( field_nodes, data_type, @@ -140,7 +137,7 @@ pub fn read( compression, version, ) - .map(|x| x.arced()), + .map(|x| x.boxed()), FixedSizeList => read_fixed_size_list( field_nodes, data_type, @@ -153,7 +150,7 @@ pub fn read( compression, version, ) - .map(|x| x.arced()), + .map(|x| x.boxed()), Struct => read_struct( field_nodes, data_type, @@ -166,7 +163,7 @@ pub fn read( compression, version, ) - .map(|x| x.arced()), + .map(|x| x.boxed()), Dictionary(key_type) => { match_integer_type!(key_type, |$T| { read_dictionary::<$T, _>( @@ -179,7 +176,7 @@ pub fn read( compression, is_little_endian, ) - .map(|x| x.arced()) + .map(|x| x.boxed()) }) } Union => read_union( @@ -194,7 +191,7 @@ pub fn read( compression, version, ) - .map(|x| x.arced()), + .map(|x| x.boxed()), Map => read_map( field_nodes, data_type, @@ -207,7 +204,7 @@ pub fn read( compression, version, ) - .map(|x| x.arced()), + .map(|x| x.boxed()), } } diff --git a/src/io/ipc/read/file_async.rs b/src/io/ipc/read/file_async.rs index 01e7d93ffa5..90f90cadc6b 100644 --- a/src/io/ipc/read/file_async.rs +++ b/src/io/ipc/read/file_async.rs @@ -1,7 +1,6 @@ //! Async reader for Arrow IPC files use std::collections::HashMap; use std::io::SeekFrom; -use std::sync::Arc; use arrow_format::ipc::{planus::ReadAsRoot, Block, MessageHeaderRef, MessageRef}; use futures::{ @@ -21,7 +20,7 @@ use super::FileMetadata; /// Async reader for Arrow IPC files pub struct FileStream<'a> { - stream: BoxStream<'a, Result>>>, + stream: BoxStream<'a, Result>>>, schema: Option, metadata: FileMetadata, } @@ -69,7 +68,7 @@ impl<'a> FileStream<'a> { mut dictionaries: Option, metadata: FileMetadata, projection: Option<(Vec, HashMap)>, - ) -> BoxStream<'a, Result>>> + ) -> BoxStream<'a, Result>>> where R: AsyncRead + AsyncSeek + Unpin + Send + 'a, { @@ -105,7 +104,7 @@ impl<'a> FileStream<'a> { } impl<'a> Stream for FileStream<'a> { - type Item = Result>>; + type Item = Result>>; fn poll_next( self: std::pin::Pin<&mut Self>, @@ -156,7 +155,7 @@ async fn read_batch( block: usize, meta_buffer: &mut Vec, block_buffer: &mut Vec, -) -> Result>> +) -> Result>> where R: AsyncRead + AsyncSeek + Unpin, { diff --git a/src/io/ipc/read/mod.rs b/src/io/ipc/read/mod.rs index fa220226b0c..283df26c53f 100644 --- a/src/io/ipc/read/mod.rs +++ b/src/io/ipc/read/mod.rs @@ -5,7 +5,6 @@ //! [`StreamReader`](stream::StreamReader), which only supports reading //! data in the order it was written in. use std::collections::HashMap; -use std::sync::Arc; use crate::array::Array; @@ -32,7 +31,7 @@ pub use schema::deserialize_schema; pub use stream::{read_stream_metadata, StreamMetadata, StreamReader, StreamState}; /// how dictionaries are tracked in this crate -pub type Dictionaries = HashMap>; +pub type Dictionaries = HashMap>; pub(crate) type Node<'a> = arrow_format::ipc::FieldNodeRef<'a>; pub(crate) type IpcBuffer<'a> = arrow_format::ipc::BufferRef<'a>; diff --git a/src/io/ipc/read/reader.rs b/src/io/ipc/read/reader.rs index 173e060e15a..3bd82591d7f 100644 --- a/src/io/ipc/read/reader.rs +++ b/src/io/ipc/read/reader.rs @@ -1,7 +1,6 @@ use std::collections::HashMap; use std::convert::TryInto; use std::io::{Read, Seek, SeekFrom}; -use std::sync::Arc; use crate::array::Array; use crate::chunk::Chunk; @@ -234,7 +233,7 @@ pub fn read_batch( projection: Option<&[usize]>, index: usize, stratch: &mut Vec, -) -> Result>> { +) -> Result>> { let block = metadata.blocks[index]; // read length @@ -318,7 +317,7 @@ impl FileReader { } impl Iterator for FileReader { - type Item = Result>>; + type Item = Result>>; fn next(&mut self) -> Option { // get current block diff --git a/src/io/ipc/read/stream.rs b/src/io/ipc/read/stream.rs index baa701cb8ff..cc81148ebe6 100644 --- a/src/io/ipc/read/stream.rs +++ b/src/io/ipc/read/stream.rs @@ -1,5 +1,4 @@ use std::io::Read; -use std::sync::Arc; use arrow_format; use arrow_format::ipc::planus::ReadAsRoot; @@ -61,7 +60,7 @@ pub enum StreamState { /// A live stream without data Waiting, /// Next item in the stream - Some(Chunk>), + Some(Chunk>), } impl StreamState { @@ -70,7 +69,7 @@ impl StreamState { /// # Panics /// /// If the `StreamState` was `Waiting`. - pub fn unwrap(self) -> Chunk> { + pub fn unwrap(self) -> Chunk> { if let StreamState::Some(batch) = self { batch } else { diff --git a/src/io/ipc/read/stream_async.rs b/src/io/ipc/read/stream_async.rs index 5f67ed6338c..b59bdc612eb 100644 --- a/src/io/ipc/read/stream_async.rs +++ b/src/io/ipc/read/stream_async.rs @@ -1,5 +1,4 @@ //! APIs to read Arrow streams asynchronously -use std::sync::Arc; use arrow_format::ipc::planus::ReadAsRoot; use futures::future::BoxFuture; @@ -33,7 +32,7 @@ enum StreamState { /// The stream does not contain new chunks (and it has not been closed) Waiting(ReadState), /// The stream contain a new chunk - Some((ReadState, Chunk>)), + Some((ReadState, Chunk>)), } /// Reads the [`StreamMetadata`] of the Arrow stream asynchronously @@ -140,7 +139,7 @@ async fn maybe_next( 0, )?; - // read the next message until we encounter a Chunk> message + // read the next message until we encounter a Chunk> message Ok(Some(StreamState::Waiting(state))) } t => Err(Error::OutOfSpec(format!( @@ -177,7 +176,7 @@ impl<'a, R: AsyncRead + Unpin + Send + 'a> AsyncStreamReader<'a, R> { } impl<'a, R: AsyncRead + Unpin + Send> Stream for AsyncStreamReader<'a, R> { - type Item = Result>>; + type Item = Result>>; fn poll_next( self: std::pin::Pin<&mut Self>, diff --git a/src/io/ipc/write/common.rs b/src/io/ipc/write/common.rs index b1be4f51549..02ddc1eca87 100644 --- a/src/io/ipc/write/common.rs +++ b/src/io/ipc/write/common.rs @@ -1,5 +1,4 @@ use std::borrow::{Borrow, Cow}; -use std::sync::Arc; use arrow_format::ipc::planus::Builder; @@ -173,7 +172,7 @@ fn encode_dictionary( } pub fn encode_chunk( - columns: &Chunk>, + columns: &Chunk>, fields: &[IpcField], dictionary_tracker: &mut DictionaryTracker, options: &WriteOptions, @@ -214,7 +213,7 @@ fn serialize_compression( /// Write [`Chunk`] into two sets of bytes, one for the header (ipc::Schema::Message) and the /// other for the batch's data -fn columns_to_bytes(columns: &Chunk>, options: &WriteOptions) -> EncodedData { +fn columns_to_bytes(columns: &Chunk>, options: &WriteOptions) -> EncodedData { let mut nodes: Vec = vec![]; let mut buffers: Vec = vec![]; let mut arrow_data: Vec = vec![]; @@ -378,7 +377,7 @@ pub(crate) fn pad_to_8(len: usize) -> usize { /// An array [`Chunk`] with optional accompanying IPC fields. #[derive(Debug, Clone, PartialEq)] pub struct Record<'a> { - columns: Cow<'a, Chunk>>, + columns: Cow<'a, Chunk>>, fields: Option>, } @@ -389,13 +388,13 @@ impl<'a> Record<'a> { } /// Get the Arrow columns in this record. - pub fn columns(&self) -> &Chunk> { + pub fn columns(&self) -> &Chunk> { self.columns.borrow() } } -impl From>> for Record<'static> { - fn from(columns: Chunk>) -> Self { +impl From>> for Record<'static> { + fn from(columns: Chunk>) -> Self { Self { columns: Cow::Owned(columns), fields: None, @@ -403,11 +402,11 @@ impl From>> for Record<'static> { } } -impl<'a, F> From<(Chunk>, Option)> for Record<'a> +impl<'a, F> From<(Chunk>, Option)> for Record<'a> where F: Into>, { - fn from((columns, fields): (Chunk>, Option)) -> Self { + fn from((columns, fields): (Chunk>, Option)) -> Self { Self { columns: Cow::Owned(columns), fields: fields.map(|f| f.into()), @@ -415,11 +414,11 @@ where } } -impl<'a, F> From<(&'a Chunk>, Option)> for Record<'a> +impl<'a, F> From<(&'a Chunk>, Option)> for Record<'a> where F: Into>, { - fn from((columns, fields): (&'a Chunk>, Option)) -> Self { + fn from((columns, fields): (&'a Chunk>, Option)) -> Self { Self { columns: Cow::Borrowed(columns), fields: fields.map(|f| f.into()), diff --git a/src/io/ipc/write/file_async.rs b/src/io/ipc/write/file_async.rs index 3a1e5fbd87e..dfe0c0b7fc4 100644 --- a/src/io/ipc/write/file_async.rs +++ b/src/io/ipc/write/file_async.rs @@ -23,7 +23,6 @@ type WriteOutput = (usize, Option, Vec, Option); /// # Examples /// /// ``` -/// use std::sync::Arc; /// use futures::{SinkExt, TryStreamExt, io::Cursor}; /// use arrow2::array::{Array, Int32Array}; /// use arrow2::datatypes::{DataType, Field, Schema}; @@ -46,7 +45,7 @@ type WriteOutput = (usize, Option, Vec, Option); /// // Write chunks to file /// for i in 0..3 { /// let values = Int32Array::from(&[Some(i), None]); -/// let chunk = Chunk::new(vec![values.arced()]); +/// let chunk = Chunk::new(vec![values.boxed()]); /// sink.feed(chunk.into()).await?; /// } /// sink.close().await?; diff --git a/src/io/ipc/write/stream.rs b/src/io/ipc/write/stream.rs index 81cafe06777..b6b6c7ac8e4 100644 --- a/src/io/ipc/write/stream.rs +++ b/src/io/ipc/write/stream.rs @@ -4,7 +4,6 @@ //! however the `FileWriter` expects a reader that supports `Seek`ing use std::io::Write; -use std::sync::Arc; use super::super::IpcField; use super::common::{encode_chunk, DictionaryTracker, EncodedData, WriteOptions}; @@ -70,7 +69,7 @@ impl StreamWriter { /// Writes [`Chunk`] to the stream pub fn write( &mut self, - columns: &Chunk>, + columns: &Chunk>, ipc_fields: Option<&[IpcField]>, ) -> Result<()> { if self.finished { diff --git a/src/io/ipc/write/stream_async.rs b/src/io/ipc/write/stream_async.rs index 238bc5338d0..8d156ff16ad 100644 --- a/src/io/ipc/write/stream_async.rs +++ b/src/io/ipc/write/stream_async.rs @@ -20,7 +20,6 @@ use crate::error::{Error, Result}; /// # Examples /// /// ``` -/// use std::sync::Arc; /// use futures::SinkExt; /// use arrow2::array::{Array, Int32Array}; /// use arrow2::datatypes::{DataType, Field, Schema}; @@ -41,7 +40,7 @@ use crate::error::{Error, Result}; /// /// for i in 0..3 { /// let values = Int32Array::from(&[Some(i), None]); -/// let chunk = Chunk::new(vec![values.arced()]); +/// let chunk = Chunk::new(vec![values.boxed()]); /// sink.feed(chunk.into()).await?; /// } /// sink.close().await?; diff --git a/src/io/ipc/write/writer.rs b/src/io/ipc/write/writer.rs index ab297da71b2..65354aca83e 100644 --- a/src/io/ipc/write/writer.rs +++ b/src/io/ipc/write/writer.rs @@ -1,4 +1,4 @@ -use std::{io::Write, sync::Arc}; +use std::io::Write; use arrow_format::ipc::planus::Builder; @@ -118,7 +118,7 @@ impl FileWriter { /// Writes [`Chunk`] to the file pub fn write( &mut self, - columns: &Chunk>, + columns: &Chunk>, ipc_fields: Option<&[IpcField]>, ) -> Result<()> { if self.state != State::Started { diff --git a/src/io/json/read/deserialize.rs b/src/io/json/read/deserialize.rs index 41d69f00c01..aba6cc6f01d 100644 --- a/src/io/json/read/deserialize.rs +++ b/src/io/json/read/deserialize.rs @@ -1,6 +1,6 @@ use std::borrow::Borrow; +use std::collections::hash_map::DefaultHasher; use std::hash::Hasher; -use std::{collections::hash_map::DefaultHasher, sync::Arc}; use hash_hasher::HashedMap; use indexmap::map::IndexMap as HashMap; @@ -314,17 +314,17 @@ fn deserialize_dictionary<'a, K: DictionaryKey, A: Borrow>>( pub(crate) fn _deserialize<'a, A: Borrow>>( rows: &[A], data_type: DataType, -) -> Arc { +) -> Box { match &data_type { - DataType::Null => Arc::new(NullArray::new(data_type, rows.len())), - DataType::Boolean => Arc::new(deserialize_boolean(rows)), - DataType::Int8 => Arc::new(deserialize_int::(rows, data_type)), - DataType::Int16 => Arc::new(deserialize_int::(rows, data_type)), + DataType::Null => Box::new(NullArray::new(data_type, rows.len())), + DataType::Boolean => Box::new(deserialize_boolean(rows)), + DataType::Int8 => Box::new(deserialize_int::(rows, data_type)), + DataType::Int16 => Box::new(deserialize_int::(rows, data_type)), DataType::Int32 | DataType::Date32 | DataType::Time32(_) | DataType::Interval(IntervalUnit::YearMonth) => { - Arc::new(deserialize_int::(rows, data_type)) + Box::new(deserialize_int::(rows, data_type)) } DataType::Interval(IntervalUnit::DayTime) => { unimplemented!("There is no natural representation of DayTime in JSON.") @@ -333,24 +333,24 @@ pub(crate) fn _deserialize<'a, A: Borrow>>( | DataType::Date64 | DataType::Time64(_) | DataType::Timestamp(_, _) - | DataType::Duration(_) => Arc::new(deserialize_int::(rows, data_type)), - DataType::UInt8 => Arc::new(deserialize_int::(rows, data_type)), - DataType::UInt16 => Arc::new(deserialize_int::(rows, data_type)), - DataType::UInt32 => Arc::new(deserialize_int::(rows, data_type)), - DataType::UInt64 => Arc::new(deserialize_int::(rows, data_type)), + | DataType::Duration(_) => Box::new(deserialize_int::(rows, data_type)), + DataType::UInt8 => Box::new(deserialize_int::(rows, data_type)), + DataType::UInt16 => Box::new(deserialize_int::(rows, data_type)), + DataType::UInt32 => Box::new(deserialize_int::(rows, data_type)), + DataType::UInt64 => Box::new(deserialize_int::(rows, data_type)), DataType::Float16 => unreachable!(), - DataType::Float32 => Arc::new(deserialize_float::(rows, data_type)), - DataType::Float64 => Arc::new(deserialize_float::(rows, data_type)), - DataType::Utf8 => Arc::new(deserialize_utf8::(rows)), - DataType::LargeUtf8 => Arc::new(deserialize_utf8::(rows)), - DataType::List(_) => Arc::new(deserialize_list::(rows, data_type)), - DataType::LargeList(_) => Arc::new(deserialize_list::(rows, data_type)), - DataType::Binary => Arc::new(deserialize_binary::(rows)), - DataType::LargeBinary => Arc::new(deserialize_binary::(rows)), - DataType::Struct(_) => Arc::new(deserialize_struct(rows, data_type)), + DataType::Float32 => Box::new(deserialize_float::(rows, data_type)), + DataType::Float64 => Box::new(deserialize_float::(rows, data_type)), + DataType::Utf8 => Box::new(deserialize_utf8::(rows)), + DataType::LargeUtf8 => Box::new(deserialize_utf8::(rows)), + DataType::List(_) => Box::new(deserialize_list::(rows, data_type)), + DataType::LargeList(_) => Box::new(deserialize_list::(rows, data_type)), + DataType::Binary => Box::new(deserialize_binary::(rows)), + DataType::LargeBinary => Box::new(deserialize_binary::(rows)), + DataType::Struct(_) => Box::new(deserialize_struct(rows, data_type)), DataType::Dictionary(key_type, _, _) => { match_integer_type!(key_type, |$T| { - Arc::new(deserialize_dictionary::<$T, _>(rows, data_type)) + Box::new(deserialize_dictionary::<$T, _>(rows, data_type)) }) } _ => todo!(), @@ -368,7 +368,7 @@ pub(crate) fn _deserialize<'a, A: Borrow>>( /// 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, Error> { +pub fn deserialize(json: &Value, data_type: DataType) -> Result, Error> { match json { Value::Array(rows) => match data_type { DataType::List(inner) | DataType::LargeList(inner) => { diff --git a/src/io/json_integration/read/array.rs b/src/io/json_integration/read/array.rs index 6210a02ca91..c0cc688b828 100644 --- a/src/io/json_integration/read/array.rs +++ b/src/io/json_integration/read/array.rs @@ -1,4 +1,4 @@ -use std::{collections::HashMap, sync::Arc}; +use std::collections::HashMap; use num_traits::NumCast; use serde_json::Value; @@ -162,7 +162,7 @@ fn to_primitive( PrimitiveArray::::new(data_type, values, validity) } -fn to_binary(json_col: &ArrowJsonColumn, data_type: DataType) -> Arc { +fn to_binary(json_col: &ArrowJsonColumn, data_type: DataType) -> Box { let validity = to_validity(&json_col.validity); let offsets = to_offsets::(json_col.offset.as_ref()); let values = json_col @@ -172,10 +172,10 @@ fn to_binary(json_col: &ArrowJsonColumn, data_type: DataType) -> Arc< .iter() .flat_map(|value| value.as_str().map(|x| hex::decode(x).unwrap()).unwrap()) .collect(); - Arc::new(BinaryArray::new(data_type, offsets, values, validity)) + Box::new(BinaryArray::new(data_type, offsets, values, validity)) } -fn to_utf8(json_col: &ArrowJsonColumn, data_type: DataType) -> Arc { +fn to_utf8(json_col: &ArrowJsonColumn, data_type: DataType) -> Box { let validity = to_validity(&json_col.validity); let offsets = to_offsets::(json_col.offset.as_ref()); let values = json_col @@ -185,7 +185,7 @@ fn to_utf8(json_col: &ArrowJsonColumn, data_type: DataType) -> Arc( @@ -193,7 +193,7 @@ fn to_list( data_type: DataType, field: &IpcField, dictionaries: &HashMap, -) -> Result> { +) -> Result> { let validity = to_validity(&json_col.validity); let child_field = ListArray::::get_child_field(&data_type); @@ -205,7 +205,7 @@ fn to_list( dictionaries, )?; let offsets = to_offsets::(json_col.offset.as_ref()); - Ok(Arc::new(ListArray::::new( + Ok(Box::new(ListArray::::new( data_type, offsets, values, validity, ))) } @@ -215,7 +215,7 @@ fn to_map( data_type: DataType, field: &IpcField, dictionaries: &HashMap, -) -> Result> { +) -> Result> { let validity = to_validity(&json_col.validity); let child_field = MapArray::get_field(&data_type); @@ -227,7 +227,7 @@ fn to_map( dictionaries, )?; let offsets = to_offsets::(json_col.offset.as_ref()); - Ok(Arc::new(MapArray::new(data_type, offsets, field, validity))) + Ok(Box::new(MapArray::new(data_type, offsets, field, validity))) } fn to_dictionary( @@ -235,7 +235,7 @@ fn to_dictionary( field: &IpcField, json_col: &ArrowJsonColumn, dictionaries: &HashMap, -) -> Result> { +) -> Result> { // find dictionary let dict_id = field.dictionary_id.unwrap(); let dictionary = dictionaries @@ -252,7 +252,7 @@ fn to_dictionary( dictionaries, )?; - Ok(Arc::new(DictionaryArray::::from_data(keys, values))) + Ok(Box::new(DictionaryArray::::from_data(keys, values))) } /// Construct an [`Array`] from the JSON integration format @@ -261,10 +261,10 @@ pub fn to_array( field: &IpcField, json_col: &ArrowJsonColumn, dictionaries: &HashMap, -) -> Result> { +) -> Result> { use PhysicalType::*; match data_type.to_physical_type() { - Null => Ok(Arc::new(NullArray::new(data_type, json_col.count))), + Null => Ok(Box::new(NullArray::new(data_type, json_col.count))), Boolean => { let validity = to_validity(&json_col.validity); let values = json_col @@ -274,23 +274,23 @@ pub fn to_array( .iter() .map(|value| value.as_bool().unwrap()) .collect::(); - Ok(Arc::new(BooleanArray::new(data_type, values, validity))) + Ok(Box::new(BooleanArray::new(data_type, values, validity))) } - Primitive(PrimitiveType::Int8) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::Int16) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::Int32) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::Int64) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::Int128) => Ok(Arc::new(to_decimal(json_col, data_type))), - Primitive(PrimitiveType::DaysMs) => Ok(Arc::new(to_primitive_days_ms(json_col, data_type))), + Primitive(PrimitiveType::Int8) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::Int16) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::Int32) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::Int64) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::Int128) => Ok(Box::new(to_decimal(json_col, data_type))), + Primitive(PrimitiveType::DaysMs) => Ok(Box::new(to_primitive_days_ms(json_col, data_type))), Primitive(PrimitiveType::MonthDayNano) => { - Ok(Arc::new(to_primitive_months_days_ns(json_col, data_type))) + Ok(Box::new(to_primitive_months_days_ns(json_col, data_type))) } - Primitive(PrimitiveType::UInt8) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::UInt16) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::UInt32) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::UInt64) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::Float32) => Ok(Arc::new(to_primitive::(json_col, data_type))), - Primitive(PrimitiveType::Float64) => Ok(Arc::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::UInt8) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::UInt16) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::UInt32) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::UInt64) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::Float32) => Ok(Box::new(to_primitive::(json_col, data_type))), + Primitive(PrimitiveType::Float64) => Ok(Box::new(to_primitive::(json_col, data_type))), Binary => Ok(to_binary::(json_col, data_type)), LargeBinary => Ok(to_binary::(json_col, data_type)), Utf8 => Ok(to_utf8::(json_col, data_type)), @@ -305,7 +305,7 @@ pub fn to_array( .iter() .flat_map(|value| value.as_str().map(|x| hex::decode(x).unwrap()).unwrap()) .collect(); - Ok(Arc::new(FixedSizeBinaryArray::new( + Ok(Box::new(FixedSizeBinaryArray::new( data_type, values, validity, ))) } @@ -324,7 +324,7 @@ pub fn to_array( dictionaries, )?; - Ok(Arc::new(FixedSizeListArray::new( + Ok(Box::new(FixedSizeListArray::new( data_type, values, validity, ))) } @@ -343,7 +343,7 @@ pub fn to_array( .collect::>>()?; let array = StructArray::new(data_type, values, validity); - Ok(Arc::new(array)) + Ok(Box::new(array)) } Dictionary(key_type) => { match_integer_type!(key_type, |$T| { @@ -397,7 +397,7 @@ pub fn to_array( .unwrap_or_default(); let array = UnionArray::new(data_type, types, fields, offsets); - Ok(Arc::new(array)) + Ok(Box::new(array)) } Map => to_map(json_col, data_type, field, dictionaries), } @@ -409,7 +409,7 @@ pub fn deserialize_chunk( ipc_fields: &[IpcField], json_batch: &ArrowJsonBatch, json_dictionaries: &HashMap, -) -> Result>> { +) -> Result>> { let arrays = schema .fields .iter() diff --git a/src/io/json_integration/write/array.rs b/src/io/json_integration/write/array.rs index 0d317b9a7e4..af4871138fd 100644 --- a/src/io/json_integration/write/array.rs +++ b/src/io/json_integration/write/array.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::{ array::{Array, PrimitiveArray}, chunk::Chunk, @@ -10,7 +8,7 @@ use super::super::{ArrowJsonBatch, ArrowJsonColumn}; /// Serializes a [`Chunk`] to [`ArrowJsonBatch`]. pub fn serialize_chunk( - columns: &Chunk>, + columns: &Chunk>, names: &[A], ) -> ArrowJsonBatch { let count = columns.len(); diff --git a/src/io/ndjson/read/deserialize.rs b/src/io/ndjson/read/deserialize.rs index dae87d66d74..e41405e5d43 100644 --- a/src/io/ndjson/read/deserialize.rs +++ b/src/io/ndjson/read/deserialize.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use json_deserializer::parse; use crate::array::Array; @@ -14,7 +12,7 @@ use super::super::super::json::read::_deserialize; /// This function is guaranteed to return an array of length equal to `rows.len()`. /// # Errors /// This function errors iff any of the rows is not a valid JSON (i.e. the format is not valid NDJSON). -pub fn deserialize(rows: &[String], data_type: DataType) -> Result, Error> { +pub fn deserialize(rows: &[String], data_type: DataType) -> Result, Error> { if rows.is_empty() { return Err(Error::ExternalFormat( "Cannot deserialize 0 NDJSON rows because empty string is not a valid JSON value" @@ -34,7 +32,7 @@ pub fn deserialize(rows: &[String], data_type: DataType) -> Result( rows: impl Iterator, data_type: DataType, -) -> Result, Error> { +) -> Result, Error> { // deserialize strings to `Value`s let rows = rows .map(|row| parse(row.as_bytes()).map_err(Error::from)) diff --git a/src/io/parquet/read/deserialize/binary/dictionary.rs b/src/io/parquet/read/deserialize/binary/dictionary.rs index 4f8adc70268..1c7e9f09179 100644 --- a/src/io/parquet/read/deserialize/binary/dictionary.rs +++ b/src/io/parquet/read/deserialize/binary/dictionary.rs @@ -1,4 +1,4 @@ -use std::{collections::VecDeque, sync::Arc}; +use std::collections::VecDeque; use parquet2::page::{BinaryPageDict, DictPage}; @@ -51,7 +51,7 @@ where } } -fn read_dict(data_type: DataType, dict: &dyn DictPage) -> Arc { +fn read_dict(data_type: DataType, dict: &dyn DictPage) -> Box { let dict = dict.as_any().downcast_ref::().unwrap(); let offsets = dict .offsets() @@ -61,13 +61,13 @@ fn read_dict(data_type: DataType, dict: &dyn DictPage) -> Arc Arc::new(Utf8Array::::from_data( + PhysicalType::Utf8 | PhysicalType::LargeUtf8 => Box::new(Utf8Array::::from_data( data_type, offsets.into(), values.into(), None, )) as _, - PhysicalType::Binary | PhysicalType::LargeBinary => Arc::new(BinaryArray::::from_data( + PhysicalType::Binary | PhysicalType::LargeBinary => Box::new(BinaryArray::::from_data( data_type, offsets.into(), values.into(), diff --git a/src/io/parquet/read/deserialize/binary/mod.rs b/src/io/parquet/read/deserialize/binary/mod.rs index 103c6c5fcab..fc2da78cb04 100644 --- a/src/io/parquet/read/deserialize/binary/mod.rs +++ b/src/io/parquet/read/deserialize/binary/mod.rs @@ -3,8 +3,6 @@ mod dictionary; mod nested; mod utils; -use std::sync::Arc; - use crate::{ array::{Array, Offset}, datatypes::DataType, @@ -36,7 +34,7 @@ where ArrayIterator::::new(iter, init, data_type, chunk_size).map(|x| { x.map(|(mut nested, array)| { let _ = nested.nested.pop().unwrap(); // the primitive - let values = Arc::new(array) as Arc; + let values = Box::new(array) as Box; (nested, values) }) }), diff --git a/src/io/parquet/read/deserialize/boolean/mod.rs b/src/io/parquet/read/deserialize/boolean/mod.rs index e928c86b56a..e0a2c09350d 100644 --- a/src/io/parquet/read/deserialize/boolean/mod.rs +++ b/src/io/parquet/read/deserialize/boolean/mod.rs @@ -21,7 +21,7 @@ where Box::new(ArrayIterator::new(iter, init, chunk_size).map(|x| { x.map(|(mut nested, array)| { let _ = nested.nested.pop().unwrap(); // the primitive - let values = array.arced(); + let values = array.boxed(); (nested, values) }) })) diff --git a/src/io/parquet/read/deserialize/dictionary.rs b/src/io/parquet/read/deserialize/dictionary.rs index b37cb46d7fe..30181b0a8c8 100644 --- a/src/io/parquet/read/deserialize/dictionary.rs +++ b/src/io/parquet/read/deserialize/dictionary.rs @@ -1,4 +1,4 @@ -use std::{collections::VecDeque, sync::Arc}; +use std::collections::VecDeque; use parquet2::{ deserialize::SliceFilteredIter, @@ -190,11 +190,11 @@ where #[derive(Debug)] pub enum Dict { Empty, - Complete(Arc), + Complete(Box), } impl Dict { - pub fn unwrap(&self) -> Arc { + pub fn unwrap(&self) -> Box { match self { Self::Empty => panic!(), Self::Complete(array) => array.clone(), @@ -211,7 +211,7 @@ pub(super) fn next_dict< 'a, K: DictionaryKey, I: DataPages, - F: Fn(&dyn DictPage) -> Arc, + F: Fn(&dyn DictPage) -> Box, >( iter: &'a mut I, items: &mut VecDeque<(Vec, MutableBitmap)>, diff --git a/src/io/parquet/read/deserialize/fixed_size_binary/dictionary.rs b/src/io/parquet/read/deserialize/fixed_size_binary/dictionary.rs index 4dc4fbcc81c..30948697352 100644 --- a/src/io/parquet/read/deserialize/fixed_size_binary/dictionary.rs +++ b/src/io/parquet/read/deserialize/fixed_size_binary/dictionary.rs @@ -1,4 +1,4 @@ -use std::{collections::VecDeque, sync::Arc}; +use std::collections::VecDeque; use parquet2::page::{DictPage, FixedLenByteArrayPageDict}; @@ -47,14 +47,14 @@ where } } -fn read_dict(data_type: DataType, dict: &dyn DictPage) -> Arc { +fn read_dict(data_type: DataType, dict: &dyn DictPage) -> Box { let dict = dict .as_any() .downcast_ref::() .unwrap(); let values = dict.values().to_vec(); - Arc::new(FixedSizeBinaryArray::from_data( + Box::new(FixedSizeBinaryArray::from_data( data_type, values.into(), None, diff --git a/src/io/parquet/read/deserialize/mod.rs b/src/io/parquet/read/deserialize/mod.rs index 3d157916977..9e68206c14a 100644 --- a/src/io/parquet/read/deserialize/mod.rs +++ b/src/io/parquet/read/deserialize/mod.rs @@ -40,15 +40,15 @@ pub fn get_page_iterator( fn create_list( data_type: DataType, nested: &mut NestedState, - values: Arc, -) -> Arc { + values: Box, +) -> Box { let (mut offsets, validity) = nested.nested.pop().unwrap().inner(); match data_type.to_logical_type() { DataType::List(_) => { offsets.push(values.len() as i64); let offsets = offsets.iter().map(|x| *x as i32).collect::>(); - Arc::new(ListArray::::new( + Box::new(ListArray::::new( data_type, offsets.into(), values, @@ -58,14 +58,14 @@ fn create_list( DataType::LargeList(_) => { offsets.push(values.len() as i64); - Arc::new(ListArray::::new( + Box::new(ListArray::::new( data_type, offsets.into(), values, validity.and_then(|x| x.into()), )) } - DataType::FixedSizeList(_, _) => Arc::new(FixedSizeListArray::new( + DataType::FixedSizeList(_, _) => Box::new(FixedSizeListArray::new( data_type, values, validity.and_then(|x| x.into()), @@ -317,7 +317,7 @@ where inner, None, ); - Ok((nested, array.arced())) + Ok((nested, array.boxed())) })) } other => { diff --git a/src/io/parquet/read/deserialize/nested_utils.rs b/src/io/parquet/read/deserialize/nested_utils.rs index 5fbfd51cf38..8eb2b2a6a93 100644 --- a/src/io/parquet/read/deserialize/nested_utils.rs +++ b/src/io/parquet/read/deserialize/nested_utils.rs @@ -1,4 +1,4 @@ -use std::{collections::VecDeque, sync::Arc}; +use std::collections::VecDeque; use parquet2::{ encoding::hybrid_rle::HybridRleDecoder, page::DataPage, read::levels::get_bit_width, @@ -529,4 +529,4 @@ where } pub type NestedArrayIter<'a> = - Box)>> + Send + Sync + 'a>; + Box)>> + Send + Sync + 'a>; diff --git a/src/io/parquet/read/deserialize/null.rs b/src/io/parquet/read/deserialize/null.rs index f1efb87ef65..a37d38011c6 100644 --- a/src/io/parquet/read/deserialize/null.rs +++ b/src/io/parquet/read/deserialize/null.rs @@ -20,11 +20,11 @@ where let remainder = chunk_size % len; let i_data_type = data_type.clone(); let complete = (0..complete_chunks) - .map(move |_| Ok(NullArray::new(i_data_type.clone(), chunk_size).arced())); + .map(move |_| Ok(NullArray::new(i_data_type.clone(), chunk_size).boxed())); if len % chunk_size == 0 { Box::new(complete) } else { let array = NullArray::new(data_type, remainder); - Box::new(complete.chain(std::iter::once(Ok(array.arced())))) + Box::new(complete.chain(std::iter::once(Ok(array.boxed())))) } } diff --git a/src/io/parquet/read/deserialize/primitive/dictionary.rs b/src/io/parquet/read/deserialize/primitive/dictionary.rs index b3a9a9cc521..a555aba635b 100644 --- a/src/io/parquet/read/deserialize/primitive/dictionary.rs +++ b/src/io/parquet/read/deserialize/primitive/dictionary.rs @@ -1,4 +1,4 @@ -use std::{collections::VecDeque, sync::Arc}; +use std::collections::VecDeque; use parquet2::{ page::{DictPage, PrimitivePageDict}, @@ -18,7 +18,7 @@ use super::super::utils::MaybeNext; use super::super::DataPages; #[inline] -fn read_dict(data_type: DataType, op: F, dict: &dyn DictPage) -> Arc +fn read_dict(data_type: DataType, op: F, dict: &dyn DictPage) -> Box where T: NativeType, P: ParquetNativeType, @@ -30,7 +30,7 @@ where .unwrap(); let values = dict.values().iter().map(|x| (op)(*x)).collect::>(); - Arc::new(PrimitiveArray::new(data_type, values.into(), None)) + Box::new(PrimitiveArray::new(data_type, values.into(), None)) } /// An iterator adapter over [`DataPages`] assumed to be encoded as boolean arrays diff --git a/src/io/parquet/read/deserialize/primitive/mod.rs b/src/io/parquet/read/deserialize/primitive/mod.rs index 44ab76fa4d0..c8124363e89 100644 --- a/src/io/parquet/read/deserialize/primitive/mod.rs +++ b/src/io/parquet/read/deserialize/primitive/mod.rs @@ -29,7 +29,7 @@ where ArrayIterator::::new(iter, init, data_type, chunk_size, op).map(|x| { x.map(|(mut nested, array)| { let _ = nested.nested.pop().unwrap(); // the primitive - (nested, array.arced()) + (nested, array.boxed()) }) }), ) diff --git a/src/io/parquet/read/deserialize/simple.rs b/src/io/parquet/read/deserialize/simple.rs index fe895c326f1..894f2f7cbf7 100644 --- a/src/io/parquet/read/deserialize/simple.rs +++ b/src/io/parquet/read/deserialize/simple.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use parquet2::{ schema::types::{ PhysicalType, PrimitiveLogicalType, PrimitiveType, TimeUnit as ParquetTimeUnit, @@ -25,10 +23,10 @@ use super::primitive; #[inline] fn dyn_iter<'a, A, I>(iter: I) -> ArrayIter<'a> where - A: Array + 'static, + A: Array, I: Iterator> + Send + Sync + 'a, { - Box::new(iter.map(|x| x.map(|x| Arc::new(x) as Arc))) + Box::new(iter.map(|x| x.map(|x| Box::new(x) as Box))) } /// Converts an iterator of [MutablePrimitiveArray] into an iterator of [PrimitiveArray] @@ -165,7 +163,7 @@ pub fn page_iter_to_arrays<'a, I: 'a + DataPages>( PrimitiveArray::::try_new(data_type.clone(), values.into(), validity) }); - let arrays = pages.map(|x| x.map(|x| x.arced())); + let arrays = pages.map(|x| x.map(|x| x.boxed())); Box::new(arrays) as _ } diff --git a/src/io/parquet/read/deserialize/struct_.rs b/src/io/parquet/read/deserialize/struct_.rs index aa6f1192a8f..dea07ad18dd 100644 --- a/src/io/parquet/read/deserialize/struct_.rs +++ b/src/io/parquet/read/deserialize/struct_.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::array::{Array, StructArray}; use crate::datatypes::{DataType, Field}; use crate::error::Error; @@ -19,7 +17,7 @@ impl<'a> StructIterator<'a> { } impl<'a> Iterator for StructIterator<'a> { - type Item = Result<(NestedState, Arc), Error>; + type Item = Result<(NestedState, Box), Error>; fn next(&mut self) -> Option { let values = self @@ -49,7 +47,7 @@ impl<'a> Iterator for StructIterator<'a> { Some(Ok(( nested, - Arc::new(StructArray::from_data( + Box::new(StructArray::from_data( DataType::Struct(self.fields.clone()), new_values, validity.and_then(|x| x.into()), diff --git a/src/io/parquet/read/file.rs b/src/io/parquet/read/file.rs index f854f5c90f9..9f866a8dce5 100644 --- a/src/io/parquet/read/file.rs +++ b/src/io/parquet/read/file.rs @@ -129,7 +129,7 @@ impl FileReader { } impl Iterator for FileReader { - type Item = Result>>; + type Item = Result>>; fn next(&mut self) -> Option { if self.remaining_rows == 0 { diff --git a/src/io/parquet/read/mod.rs b/src/io/parquet/read/mod.rs index dfac4c38df4..f2a7cf2a5d9 100644 --- a/src/io/parquet/read/mod.rs +++ b/src/io/parquet/read/mod.rs @@ -8,10 +8,7 @@ mod row_group; pub mod schema; pub mod statistics; -use std::{ - io::{Read, Seek}, - sync::Arc, -}; +use std::io::{Read, Seek}; use futures::{AsyncRead, AsyncSeek}; @@ -56,7 +53,7 @@ impl + Send } /// Type def for a sharable, boxed dyn [`Iterator`] of arrays -pub type ArrayIter<'a> = Box>> + Send + Sync + 'a>; +pub type ArrayIter<'a> = Box>> + Send + Sync + 'a>; /// Reads parquets' metadata syncronously. pub fn read_metadata(reader: &mut R) -> Result { diff --git a/src/io/parquet/read/row_group.rs b/src/io/parquet/read/row_group.rs index fea9638e5e2..007286a476b 100644 --- a/src/io/parquet/read/row_group.rs +++ b/src/io/parquet/read/row_group.rs @@ -1,7 +1,4 @@ -use std::{ - io::{Read, Seek}, - sync::Arc, -}; +use std::io::{Read, Seek}; use futures::{ future::{try_join_all, BoxFuture}, @@ -59,7 +56,7 @@ impl RowGroupDeserializer { } impl Iterator for RowGroupDeserializer { - type Item = Result>>; + type Item = Result>>; fn next(&mut self) -> Option { if self.remaining_rows == 0 { @@ -71,7 +68,7 @@ impl Iterator for RowGroupDeserializer { .map(|iter| { let array = iter.next().unwrap()?; Ok(if array.len() > self.remaining_rows { - array.slice(0, array.len() - self.remaining_rows).into() + array.slice(0, array.len() - self.remaining_rows) } else { array }) @@ -186,7 +183,7 @@ pub fn to_deserializer<'a>( let pages = PageReader::new( std::io::Cursor::new(chunk), column_meta, - Arc::new(|_, _| true), + std::sync::Arc::new(|_, _| true), vec![], ); ( diff --git a/src/io/parquet/read/statistics/dictionary.rs b/src/io/parquet/read/statistics/dictionary.rs index 03c7e7ce966..d29feb46802 100644 --- a/src/io/parquet/read/statistics/dictionary.rs +++ b/src/io/parquet/read/statistics/dictionary.rs @@ -37,7 +37,7 @@ impl MutableArray for DynMutableDictionary { } fn as_box(&mut self) -> Box { - let inner = self.inner.as_arc(); + let inner = self.inner.as_box(); match self.data_type.to_physical_type() { PhysicalType::Dictionary(key) => match_integer_type!(key, |$T| { let keys = PrimitiveArray::<$T>::from_iter((0..inner.len() as $T).map(Some)); diff --git a/src/io/parquet/read/statistics/list.rs b/src/io/parquet/read/statistics/list.rs index 5ce562c5bec..6b4bb2d7d3d 100644 --- a/src/io/parquet/read/statistics/list.rs +++ b/src/io/parquet/read/statistics/list.rs @@ -36,7 +36,7 @@ impl MutableArray for DynMutableListArray { } fn as_box(&mut self) -> Box { - let inner = self.inner.as_arc(); + let inner = self.inner.as_box(); match self.data_type.to_logical_type() { DataType::List(_) => { diff --git a/src/io/parquet/read/statistics/map.rs b/src/io/parquet/read/statistics/map.rs index ddc09c9c518..71ef8154dd0 100644 --- a/src/io/parquet/read/statistics/map.rs +++ b/src/io/parquet/read/statistics/map.rs @@ -41,7 +41,7 @@ impl MutableArray for DynMutableMapArray { Box::new(MapArray::new( self.data_type.clone(), vec![0, self.inner.len() as i32].into(), - self.inner.as_arc(), + self.inner.as_box(), None, )) } diff --git a/src/io/parquet/read/statistics/struct_.rs b/src/io/parquet/read/statistics/struct_.rs index 1f59188b5a4..7ae80af3e79 100644 --- a/src/io/parquet/read/statistics/struct_.rs +++ b/src/io/parquet/read/statistics/struct_.rs @@ -38,7 +38,7 @@ impl MutableArray for DynMutableStructArray { } fn as_box(&mut self) -> Box { - let inner = self.inner.iter_mut().map(|x| x.as_arc()).collect(); + let inner = self.inner.iter_mut().map(|x| x.as_box()).collect(); Box::new(StructArray::new(self.data_type.clone(), inner, None)) } diff --git a/src/io/parquet/write/pages.rs b/src/io/parquet/write/pages.rs index 1f3cac6434f..4c0c0c23420 100644 --- a/src/io/parquet/write/pages.rs +++ b/src/io/parquet/write/pages.rs @@ -234,8 +234,8 @@ mod tests { #[test] fn test_struct() { - let boolean = BooleanArray::from_slice(&[false, false, true, true]).arced(); - let int = Int32Array::from_slice(&[42, 28, 19, 31]).arced(); + let boolean = BooleanArray::from_slice(&[false, false, true, true]).boxed(); + let int = Int32Array::from_slice(&[42, 28, 19, 31]).boxed(); let fields = vec![ Field::new("b", DataType::Boolean, false), @@ -298,9 +298,8 @@ mod tests { #[test] fn test_struct_struct() { - use std::sync::Arc; - let boolean = BooleanArray::from_slice(&[false, false, true, true]).arced(); - let int = Int32Array::from_slice(&[42, 28, 19, 31]).arced(); + let boolean = BooleanArray::from_slice(&[false, false, true, true]).boxed(); + let int = Int32Array::from_slice(&[42, 28, 19, 31]).boxed(); let fields = vec![ Field::new("b", DataType::Boolean, false), @@ -320,7 +319,7 @@ mod tests { let array = StructArray::from_data( DataType::Struct(fields), - vec![Arc::new(array.clone()), Arc::new(array)], + vec![Box::new(array.clone()), Box::new(array)], None, ); @@ -402,9 +401,8 @@ mod tests { #[test] fn test_list_struct() { - use std::sync::Arc; - let boolean = BooleanArray::from_slice(&[false, false, true, true]).arced(); - let int = Int32Array::from_slice(&[42, 28, 19, 31]).arced(); + let boolean = BooleanArray::from_slice(&[false, false, true, true]).boxed(); + let int = Int32Array::from_slice(&[42, 28, 19, 31]).boxed(); let fields = vec![ Field::new("b", DataType::Boolean, false), @@ -420,7 +418,7 @@ mod tests { let array = ListArray::new( DataType::List(Box::new(Field::new("l", array.data_type().clone(), true))), vec![0i32, 2, 4].into(), - Arc::new(array), + Box::new(array), None, ); diff --git a/src/io/parquet/write/sink.rs b/src/io/parquet/write/sink.rs index b7b3e95143a..160ac81d7fe 100644 --- a/src/io/parquet/write/sink.rs +++ b/src/io/parquet/write/sink.rs @@ -1,4 +1,4 @@ -use std::{collections::HashMap, pin::Pin, sync::Arc, task::Poll}; +use std::{collections::HashMap, pin::Pin, task::Poll}; use futures::{future::BoxFuture, AsyncWrite, FutureExt, Sink, TryFutureExt}; use parquet2::metadata::KeyValue; @@ -18,7 +18,6 @@ use super::{Encoding, SchemaDescriptor, WriteOptions}; /// # Examples /// /// ``` -/// use std::sync::Arc; /// use futures::SinkExt; /// use arrow2::array::{Array, Int32Array}; /// use arrow2::datatypes::{DataType, Field, Schema}; @@ -47,7 +46,7 @@ use super::{Encoding, SchemaDescriptor, WriteOptions}; /// /// for i in 0..3 { /// let values = Int32Array::from(&[Some(i), None]); -/// let chunk = Chunk::new(vec![values.arced()]); +/// let chunk = Chunk::new(vec![values.boxed()]); /// sink.feed(chunk).await?; /// } /// sink.metadata.insert(String::from("key"), Some(String::from("value"))); @@ -147,13 +146,13 @@ where } } -impl<'a, W> Sink>> for FileSink<'a, W> +impl<'a, W> Sink>> for FileSink<'a, W> where W: AsyncWrite + Send + Unpin + 'a, { type Error = Error; - fn start_send(self: Pin<&mut Self>, item: Chunk>) -> Result<(), Self::Error> { + fn start_send(self: Pin<&mut Self>, item: Chunk>) -> Result<(), Self::Error> { if self.schema.fields.len() != item.arrays().len() { return Err(Error::InvalidArgumentError( "The number of arrays in the chunk must equal the number of fields in the schema" diff --git a/src/lib.rs b/src/lib.rs index 7abafb78288..ec01cec8755 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -4,6 +4,8 @@ #![allow(unused_unsafe)] // #![allow(clippy::len_without_is_empty)] +// Trait objects must be returned as a &Box so that they can be cloned +#![allow(clippy::borrowed_box)] #![cfg_attr(docsrs, feature(doc_cfg))] #![cfg_attr(feature = "simd", feature(portable_simd))] diff --git a/src/scalar/dictionary.rs b/src/scalar/dictionary.rs index 450930e401c..bba0225084c 100644 --- a/src/scalar/dictionary.rs +++ b/src/scalar/dictionary.rs @@ -1,5 +1,4 @@ use std::any::Any; -use std::sync::Arc; use crate::{array::*, datatypes::DataType}; @@ -8,7 +7,7 @@ use super::Scalar; /// The [`DictionaryArray`] equivalent of [`Array`] for [`Scalar`]. #[derive(Debug, Clone)] pub struct DictionaryScalar { - value: Option>, + value: Option>, phantom: std::marker::PhantomData, data_type: DataType, } @@ -26,7 +25,7 @@ impl DictionaryScalar { /// * the `data_type` is not `List` or `LargeList` (depending on this scalar's offset `O`) /// * the child of the `data_type` is not equal to the `values` #[inline] - pub fn new(data_type: DataType, value: Option>) -> Self { + pub fn new(data_type: DataType, value: Option>) -> Self { Self { value, phantom: std::marker::PhantomData, @@ -35,7 +34,7 @@ impl DictionaryScalar { } /// The values of the [`DictionaryScalar`] - pub fn value(&self) -> Option<&Arc> { + pub fn value(&self) -> Option<&Box> { self.value.as_ref() } } diff --git a/src/scalar/fixed_size_list.rs b/src/scalar/fixed_size_list.rs index fd41e2100d5..2aea2307649 100644 --- a/src/scalar/fixed_size_list.rs +++ b/src/scalar/fixed_size_list.rs @@ -1,5 +1,4 @@ use std::any::Any; -use std::sync::Arc; use crate::{array::*, datatypes::DataType}; @@ -9,7 +8,7 @@ use super::Scalar; /// [`Array`]. The only difference is that this has only one element. #[derive(Debug, Clone)] pub struct FixedSizeListScalar { - values: Option>, + values: Option>, data_type: DataType, } @@ -29,7 +28,7 @@ impl FixedSizeListScalar { /// * the child of the `data_type` is not equal to the `values` /// * the size of child array is not equal #[inline] - pub fn new(data_type: DataType, values: Option>) -> Self { + pub fn new(data_type: DataType, values: Option>) -> Self { let (field, size) = FixedSizeListArray::get_child_and_size(&data_type); let inner_data_type = field.data_type(); let values = values.map(|x| { @@ -41,7 +40,7 @@ impl FixedSizeListScalar { } /// The values of the [`FixedSizeListScalar`] - pub fn values(&self) -> Option<&Arc> { + pub fn values(&self) -> Option<&Box> { self.values.as_ref() } } diff --git a/src/scalar/list.rs b/src/scalar/list.rs index c8c505ef503..438e13f7f55 100644 --- a/src/scalar/list.rs +++ b/src/scalar/list.rs @@ -1,5 +1,4 @@ use std::any::Any; -use std::sync::Arc; use crate::{array::*, datatypes::DataType}; @@ -9,7 +8,7 @@ use super::Scalar; /// [`Array`]. The only difference is that this has only one element. #[derive(Debug, Clone)] pub struct ListScalar { - values: Arc, + values: Box, is_valid: bool, phantom: std::marker::PhantomData, data_type: DataType, @@ -30,14 +29,14 @@ impl ListScalar { /// * the `data_type` is not `List` or `LargeList` (depending on this scalar's offset `O`) /// * the child of the `data_type` is not equal to the `values` #[inline] - pub fn new(data_type: DataType, values: Option>) -> Self { + pub fn new(data_type: DataType, values: Option>) -> Self { let inner_data_type = ListArray::::get_child_type(&data_type); let (is_valid, values) = match values { Some(values) => { assert_eq!(inner_data_type, values.data_type()); (true, values) } - None => (false, new_empty_array(inner_data_type.clone()).into()), + None => (false, new_empty_array(inner_data_type.clone())), }; Self { values, @@ -48,7 +47,7 @@ impl ListScalar { } /// The values of the [`ListScalar`] - pub fn values(&self) -> &Arc { + pub fn values(&self) -> &Box { &self.values } } diff --git a/src/scalar/mod.rs b/src/scalar/mod.rs index c7394af0097..41d32aaa957 100644 --- a/src/scalar/mod.rs +++ b/src/scalar/mod.rs @@ -30,7 +30,7 @@ pub use union::UnionScalar; /// Trait object declaring an optional value with a [`DataType`]. /// This strait is often used in APIs that accept multiple scalar types. -pub trait Scalar: std::fmt::Debug + Send + Sync { +pub trait Scalar: std::fmt::Debug + Send + Sync + dyn_clone::DynClone + 'static { /// convert itself to fn as_any(&self) -> &dyn Any; @@ -41,6 +41,8 @@ pub trait Scalar: std::fmt::Debug + Send + Sync { fn data_type(&self) -> &DataType; } +dyn_clone::clone_trait_object!(Scalar); + macro_rules! dyn_new_utf8 { ($array:expr, $index:expr, $type:ty) => {{ let array = $array.as_any().downcast_ref::>().unwrap(); @@ -118,7 +120,7 @@ pub fn new_scalar(array: &dyn Array, index: usize) -> Box { let values = array .values() .iter() - .map(|x| new_scalar(x.as_ref(), index).into()) + .map(|x| new_scalar(x.as_ref(), index)) .collect(); Box::new(StructScalar::new(array.data_type().clone(), Some(values))) } else { @@ -140,7 +142,7 @@ pub fn new_scalar(array: &dyn Array, index: usize) -> Box { FixedSizeList => { let array = array.as_any().downcast_ref::().unwrap(); let value = if array.is_valid(index) { - Some(array.value(index).into()) + Some(array.value(index)) } else { None }; @@ -151,7 +153,7 @@ pub fn new_scalar(array: &dyn Array, index: usize) -> Box { Box::new(UnionScalar::new( array.data_type().clone(), array.types()[index], - array.value(index).into(), + array.value(index), )) } Map => todo!(), diff --git a/src/scalar/struct_.rs b/src/scalar/struct_.rs index b7822b3ae02..e91f7803877 100644 --- a/src/scalar/struct_.rs +++ b/src/scalar/struct_.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::datatypes::DataType; use super::Scalar; @@ -7,7 +5,7 @@ use super::Scalar; /// A single entry of a [`crate::array::StructArray`]. #[derive(Debug, Clone)] pub struct StructScalar { - values: Vec>, + values: Vec>, is_valid: bool, data_type: DataType, } @@ -23,7 +21,7 @@ impl PartialEq for StructScalar { impl StructScalar { /// Returns a new [`StructScalar`] #[inline] - pub fn new(data_type: DataType, values: Option>>) -> Self { + pub fn new(data_type: DataType, values: Option>>) -> Self { let is_valid = values.is_some(); Self { values: values.unwrap_or_default(), @@ -34,7 +32,7 @@ impl StructScalar { /// Returns the values irrespectively of the validity. #[inline] - pub fn values(&self) -> &[Arc] { + pub fn values(&self) -> &[Box] { &self.values } } diff --git a/src/scalar/union.rs b/src/scalar/union.rs index fc273457b32..998b0607d6a 100644 --- a/src/scalar/union.rs +++ b/src/scalar/union.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use crate::datatypes::DataType; use super::Scalar; @@ -7,7 +5,7 @@ use super::Scalar; /// A single entry of a [`crate::array::UnionArray`]. #[derive(Debug, Clone, PartialEq)] pub struct UnionScalar { - value: Arc, + value: Box, type_: i8, data_type: DataType, } @@ -15,7 +13,7 @@ pub struct UnionScalar { impl UnionScalar { /// Returns a new [`UnionScalar`] #[inline] - pub fn new(data_type: DataType, type_: i8, value: Arc) -> Self { + pub fn new(data_type: DataType, type_: i8, value: Box) -> Self { Self { value, type_, @@ -25,7 +23,7 @@ impl UnionScalar { /// Returns the inner value #[inline] - pub fn value(&self) -> &Arc { + pub fn value(&self) -> &Box { &self.value } diff --git a/tests/it/array/equal/dictionary.rs b/tests/it/array/equal/dictionary.rs index 8ae82e7d691..1bbe9627d6f 100644 --- a/tests/it/array/equal/dictionary.rs +++ b/tests/it/array/equal/dictionary.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::array::*; use super::test_equal; @@ -8,7 +6,7 @@ fn create_dictionary_array(values: &[Option<&str>], keys: &[Option]) -> Dic let keys = Int16Array::from(keys); let values = Utf8Array::::from(values); - DictionaryArray::from_data(keys, Arc::new(values)) + DictionaryArray::from_data(keys, Box::new(values)) } #[test] diff --git a/tests/it/array/equal/list.rs b/tests/it/array/equal/list.rs index c57a1888fcb..343b7514e77 100644 --- a/tests/it/array/equal/list.rs +++ b/tests/it/array/equal/list.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::array::{Int32Array, ListArray, MutableListArray, MutablePrimitiveArray, TryExtend}; use arrow2::bitmap::Bitmap; use arrow2::buffer::Buffer; @@ -71,7 +69,7 @@ fn test_list_offsets() { fn test_bla() { let offsets = Buffer::from(vec![0, 3, 3, 6]); let data_type = ListArray::::default_datatype(DataType::Int32); - let values = Arc::new(Int32Array::from([ + let values = Box::new(Int32Array::from([ Some(1), Some(2), Some(3), @@ -85,7 +83,7 @@ fn test_bla() { let offsets = Buffer::from(vec![0, 0, 3]); let data_type = ListArray::::default_datatype(DataType::Int32); - let values = Arc::new(Int32Array::from([Some(4), None, Some(6)])); + let values = Box::new(Int32Array::from([Some(4), None, Some(6)])); let validity = Bitmap::from([false, true]); let rhs = ListArray::::from_data(data_type, offsets, values, Some(validity)); diff --git a/tests/it/array/growable/dictionary.rs b/tests/it/array/growable/dictionary.rs index 8c27fc50070..a45c0b298be 100644 --- a/tests/it/array/growable/dictionary.rs +++ b/tests/it/array/growable/dictionary.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::array::growable::{Growable, GrowableDictionary}; use arrow2::array::*; use arrow2::datatypes::DataType; @@ -17,7 +15,7 @@ fn test_single() -> Result<()> { // same values, less keys let expected = DictionaryArray::::from_data( PrimitiveArray::from(vec![Some(1), Some(0)]), - Arc::new(Utf8Array::::from(&original_data)), + Box::new(Utf8Array::::from(&original_data)), ); let mut growable = GrowableDictionary::new(&[&array], false, 0); @@ -41,7 +39,7 @@ fn test_negative_keys() { Some(vec![true, true, true, false].into()), ); - let arr = DictionaryArray::from_data(keys, Arc::new(Utf8Array::::from(vals))); + let arr = DictionaryArray::from_data(keys, Box::new(Utf8Array::::from(vals))); // check that we don't panic with negative keys to usize conversion let mut growable = GrowableDictionary::new(&[&arr], false, 0); growable.extend(0, 0, 4); @@ -69,7 +67,7 @@ fn test_multi() -> Result<()> { original_data1.extend(original_data2.iter().cloned()); let expected = DictionaryArray::::from_data( PrimitiveArray::from(vec![Some(1), None, Some(3), None]), - Arc::new(Utf8Array::::from_slice(&["a", "b", "c", "b", "a"])), + Box::new(Utf8Array::::from_slice(&["a", "b", "c", "b", "a"])), ); let mut growable = GrowableDictionary::new(&[&array1, &array2], false, 0); diff --git a/tests/it/array/growable/mod.rs b/tests/it/array/growable/mod.rs index e3de20a424e..acc5da6ddc5 100644 --- a/tests/it/array/growable/mod.rs +++ b/tests/it/array/growable/mod.rs @@ -40,7 +40,7 @@ fn test_make_growable() { let array = DictionaryArray::::from_data( Int32Array::from_slice([1, 2]), - std::sync::Arc::new(Int32Array::from_slice([1, 2])), + Box::new(Int32Array::from_slice([1, 2])), ); make_growable(&[&array], false, 2); } diff --git a/tests/it/array/growable/struct_.rs b/tests/it/array/growable/struct_.rs index 0dd85e0dcd1..b10ad5a4717 100644 --- a/tests/it/array/growable/struct_.rs +++ b/tests/it/array/growable/struct_.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::array::{ growable::{Growable, GrowableStruct}, Array, PrimitiveArray, StructArray, Utf8Array, @@ -7,15 +5,15 @@ use arrow2::array::{ use arrow2::bitmap::Bitmap; use arrow2::datatypes::{DataType, Field}; -fn some_values() -> (DataType, Vec>) { - let strings: Arc = Arc::new(Utf8Array::::from(&[ +fn some_values() -> (DataType, Vec>) { + let strings: Box = Box::new(Utf8Array::::from(&[ Some("a"), Some("aa"), None, Some("mark"), Some("doe"), ])); - let ints: Arc = Arc::new(PrimitiveArray::::from(&[ + let ints: Box = Box::new(PrimitiveArray::::from(&[ Some(1), Some(2), Some(3), @@ -42,7 +40,7 @@ fn basic() { let expected = StructArray::from_data( fields, - vec![values[0].slice(1, 2).into(), values[1].slice(1, 2).into()], + vec![values[0].slice(1, 2), values[1].slice(1, 2)], None, ); assert_eq!(result, expected) @@ -61,7 +59,7 @@ fn offset() { let expected = StructArray::from_data( fields, - vec![values[0].slice(2, 2).into(), values[1].slice(2, 2).into()], + vec![values[0].slice(2, 2), values[1].slice(2, 2)], None, ); @@ -85,7 +83,7 @@ fn nulls() { let expected = StructArray::from_data( fields, - vec![values[0].slice(1, 2).into(), values[1].slice(1, 2).into()], + vec![values[0].slice(1, 2), values[1].slice(1, 2)], Some(Bitmap::from_u8_slice(&[0b00000010], 5).slice(1, 2)), ); @@ -105,14 +103,14 @@ fn many() { mutable.extend_validity(1); let result = mutable.as_box(); - let expected_string: Arc = Arc::new(Utf8Array::::from(&[ + let expected_string: Box = Box::new(Utf8Array::::from(&[ Some("aa"), None, Some("a"), Some("aa"), None, ])); - let expected_int: Arc = Arc::new(PrimitiveArray::::from(vec![ + let expected_int: Box = Box::new(PrimitiveArray::::from(vec![ Some(2), Some(3), Some(1), diff --git a/tests/it/array/growable/union.rs b/tests/it/array/growable/union.rs index eb513c2c217..c8f99a0f82d 100644 --- a/tests/it/array/growable/union.rs +++ b/tests/it/array/growable/union.rs @@ -16,8 +16,8 @@ fn sparse() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Sparse); let types = vec![0, 0, 1].into(); let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields, None); @@ -46,8 +46,8 @@ fn dense() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Dense); let types = vec![0, 0, 1].into(); let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("c")]).boxed(), ]; let offsets = Some(vec![0, 1, 0].into()); diff --git a/tests/it/array/list/mod.rs b/tests/it/array/list/mod.rs index c0e8ff19b19..ef07860edb2 100644 --- a/tests/it/array/list/mod.rs +++ b/tests/it/array/list/mod.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::array::*; use arrow2::buffer::Buffer; use arrow2::datatypes::DataType; @@ -15,7 +13,7 @@ fn debug() { let array = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 2, 3, 5]), - Arc::new(values), + Box::new(values), None, ); @@ -32,7 +30,7 @@ fn test_nested_panic() { let array = ListArray::::from_data( data_type.clone(), Buffer::from(vec![0, 2, 2, 3, 5]), - Arc::new(values), + Box::new(values), None, ); @@ -41,7 +39,7 @@ fn test_nested_panic() { let _ = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 4]), - Arc::new(array), + Box::new(array), None, ); } @@ -55,7 +53,7 @@ fn test_nested_display() { let array = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 4, 7, 7, 8, 10]), - Arc::new(values), + Box::new(values), None, ); @@ -63,7 +61,7 @@ fn test_nested_display() { let nested = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 5, 6]), - Arc::new(array), + Box::new(array), None, ); diff --git a/tests/it/array/list/mutable.rs b/tests/it/array/list/mutable.rs index f7d21bdc519..64b593bc2e0 100644 --- a/tests/it/array/list/mutable.rs +++ b/tests/it/array/list/mutable.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::{array::*, bitmap::Bitmap, buffer::Buffer, datatypes::DataType}; #[test] @@ -24,7 +22,7 @@ fn basics() { let expected = ListArray::::from_data( data_type, Buffer::from(vec![0, 3, 3, 6]), - Arc::new(values), + Box::new(values), Some(Bitmap::from([true, false, true])), ); assert_eq!(expected, array); diff --git a/tests/it/array/map/mod.rs b/tests/it/array/map/mod.rs index ccd8fb75b9d..38fde84367e 100644 --- a/tests/it/array/map/mod.rs +++ b/tests/it/array/map/mod.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::{ array::*, datatypes::{DataType, Field}, @@ -16,21 +14,21 @@ fn basics() { let field = StructArray::new( dt.clone(), vec![ - Arc::new(Utf8Array::::from_slice(["a", "aa", "aaa"])) as _, - Arc::new(Utf8Array::::from_slice(["b", "bb", "bbb"])), + Box::new(Utf8Array::::from_slice(["a", "aa", "aaa"])) as _, + Box::new(Utf8Array::::from_slice(["b", "bb", "bbb"])), ], None, ); - let array = MapArray::new(data_type, vec![0, 1, 2].into(), Arc::new(field), None); + let array = MapArray::new(data_type, vec![0, 1, 2].into(), Box::new(field), None); assert_eq!( array.value(0), Box::new(StructArray::new( dt.clone(), vec![ - Arc::new(Utf8Array::::from_slice(["a"])) as _, - Arc::new(Utf8Array::::from_slice(["b"])), + Box::new(Utf8Array::::from_slice(["a"])) as _, + Box::new(Utf8Array::::from_slice(["b"])), ], None, )) as Box @@ -42,8 +40,8 @@ fn basics() { Box::new(StructArray::new( dt, vec![ - Arc::new(Utf8Array::::from_slice(["aa"])) as _, - Arc::new(Utf8Array::::from_slice(["bb"])), + Box::new(Utf8Array::::from_slice(["aa"])) as _, + Box::new(Utf8Array::::from_slice(["bb"])), ], None, )) as Box diff --git a/tests/it/array/mod.rs b/tests/it/array/mod.rs index dfa9329fffb..ef14b993d25 100644 --- a/tests/it/array/mod.rs +++ b/tests/it/array/mod.rs @@ -102,5 +102,5 @@ fn test_with_validity() { // check that `PartialEq` can be derived #[derive(PartialEq)] struct A { - array: std::sync::Arc, + array: Box, } diff --git a/tests/it/array/struct_/iterator.rs b/tests/it/array/struct_/iterator.rs index a73fc500654..03c1368a199 100644 --- a/tests/it/array/struct_/iterator.rs +++ b/tests/it/array/struct_/iterator.rs @@ -4,8 +4,8 @@ use arrow2::scalar::new_scalar; #[test] fn test_simple_iter() { - let boolean = BooleanArray::from_slice(&[false, false, true, true]).arced(); - let int = Int32Array::from_slice(&[42, 28, 19, 31]).arced(); + let boolean = BooleanArray::from_slice(&[false, false, true, true]).boxed(); + let int = Int32Array::from_slice(&[42, 28, 19, 31]).boxed(); let fields = vec![ Field::new("b", DataType::Boolean, false), diff --git a/tests/it/array/struct_/mod.rs b/tests/it/array/struct_/mod.rs index 23862c37471..9ec34eca0ef 100644 --- a/tests/it/array/struct_/mod.rs +++ b/tests/it/array/struct_/mod.rs @@ -6,8 +6,8 @@ use arrow2::datatypes::*; #[test] fn debug() { - let boolean = BooleanArray::from_slice(&[false, false, true, true]).arced(); - let int = Int32Array::from_slice(&[42, 28, 19, 31]).arced(); + let boolean = BooleanArray::from_slice(&[false, false, true, true]).boxed(); + let int = Int32Array::from_slice(&[42, 28, 19, 31]).boxed(); let fields = vec![ Field::new("b", DataType::Boolean, false), diff --git a/tests/it/array/union.rs b/tests/it/array/union.rs index 3f6213f7aa3..c5b572c60b1 100644 --- a/tests/it/array/union.rs +++ b/tests/it/array/union.rs @@ -28,8 +28,8 @@ fn sparse_debug() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Sparse); let types = vec![0, 0, 1].into(); let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields, None); @@ -48,8 +48,8 @@ fn dense_debug() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Dense); let types = vec![0, 0, 1].into(); let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("c")]).boxed(), ]; let offsets = Some(vec![0, 1, 0].into()); @@ -69,8 +69,8 @@ fn slice() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Sparse); let types = Buffer::from(vec![0, 0, 1]); let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type.clone(), types, fields.clone(), None); @@ -79,8 +79,8 @@ fn slice() -> Result<()> { let sliced_types = Buffer::from(vec![0, 1]); let sliced_fields = vec![ - Int32Array::from(&[None, Some(2)]).arced(), - Utf8Array::::from(&[Some("b"), Some("c")]).arced(), + Int32Array::from(&[None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("b"), Some("c")]).boxed(), ]; let expected = UnionArray::from_data(data_type, sliced_types, sliced_fields, None); @@ -97,8 +97,8 @@ fn iter_sparse() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Sparse); let types = Buffer::from(vec![0, 0, 1]); let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields.clone(), None); @@ -131,8 +131,8 @@ fn iter_dense() -> Result<()> { let types = Buffer::from(vec![0, 0, 1]); let offsets = Buffer::::from(vec![0, 1, 0]); let fields = vec![ - Int32Array::from(&[Some(1), None]).arced(), - Utf8Array::::from(&[Some("c")]).arced(), + Int32Array::from(&[Some(1), None]).boxed(), + Utf8Array::::from(&[Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields.clone(), Some(offsets)); @@ -164,8 +164,8 @@ fn iter_sparse_slice() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Sparse); let types = Buffer::from(vec![0, 0, 1]); let fields = vec![ - Int32Array::from(&[Some(1), Some(3), Some(2)]).arced(), - Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).arced(), + Int32Array::from(&[Some(1), Some(3), Some(2)]).boxed(), + Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields.clone(), None); @@ -191,8 +191,8 @@ fn iter_dense_slice() -> Result<()> { let types = Buffer::from(vec![0, 0, 1]); let offsets = Buffer::::from(vec![0, 1, 0]); let fields = vec![ - Int32Array::from(&[Some(1), Some(3)]).arced(), - Utf8Array::::from(&[Some("c")]).arced(), + Int32Array::from(&[Some(1), Some(3)]).boxed(), + Utf8Array::::from(&[Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields.clone(), Some(offsets)); @@ -218,8 +218,8 @@ fn scalar() -> Result<()> { let types = Buffer::from(vec![0, 0, 1]); let offsets = Buffer::::from(vec![0, 1, 0]); let fields = vec![ - Int32Array::from(&[Some(1), None]).arced(), - Utf8Array::::from(&[Some("c")]).arced(), + Int32Array::from(&[Some(1), None]).boxed(), + Utf8Array::::from(&[Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields.clone(), Some(offsets)); diff --git a/tests/it/compute/arithmetics/mod.rs b/tests/it/compute/arithmetics/mod.rs index ed84daa3998..f697cd8041a 100644 --- a/tests/it/compute/arithmetics/mod.rs +++ b/tests/it/compute/arithmetics/mod.rs @@ -97,12 +97,12 @@ fn test_neg() { fn test_neg_dict() { let a = DictionaryArray::::from_data( UInt8Array::from_slice(&[0, 0, 1]), - std::sync::Arc::new(Int8Array::from_slice(&[1, 2])), + Box::new(Int8Array::from_slice(&[1, 2])), ); let result = neg(&a); let expected = DictionaryArray::::from_data( UInt8Array::from_slice(&[0, 0, 1]), - std::sync::Arc::new(Int8Array::from_slice(&[-1, -2])), + Box::new(Int8Array::from_slice(&[-1, -2])), ); assert_eq!(expected, result.as_ref()); } diff --git a/tests/it/compute/take.rs b/tests/it/compute/take.rs index b707c22d339..ce17581a0db 100644 --- a/tests/it/compute/take.rs +++ b/tests/it/compute/take.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::compute::take::{can_take, take}; use arrow2::datatypes::{DataType, Field, IntervalUnit}; use arrow2::error::Result; @@ -75,7 +73,7 @@ fn create_test_struct() -> StructArray { ]; StructArray::from_data( DataType::Struct(fields), - vec![boolean.arced(), int.arced()], + vec![boolean.boxed(), int.boxed()], validity, ) } @@ -96,7 +94,7 @@ fn test_struct_with_nulls() { .into(); let expected = StructArray::from_data( array.data_type().clone(), - vec![boolean.arced(), int.arced()], + vec![boolean.boxed(), int.boxed()], validity, ); assert_eq!(expected, output.as_ref()); @@ -179,7 +177,7 @@ fn list_with_no_none() { let array = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 2, 6, 9, 10]), - Arc::new(values), + Box::new(values), None, ); @@ -192,7 +190,7 @@ fn list_with_no_none() { let expected = ListArray::::from_data( expected_type, Buffer::from(vec![0, 1, 1, 4]), - Arc::new(expected_values), + Box::new(expected_values), None, ); @@ -211,7 +209,7 @@ fn list_with_none() { let array = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 2, 6, 9, 10]), - Arc::new(values), + Box::new(values), Some(validity), ); @@ -270,7 +268,7 @@ fn test_nested() { let array = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 4, 7, 7, 8, 10]), - Arc::new(values), + Box::new(values), None, ); @@ -278,7 +276,7 @@ fn test_nested() { let nested = ListArray::::from_data( data_type, Buffer::from(vec![0, 2, 5, 6]), - Arc::new(array), + Box::new(array), None, ); @@ -293,7 +291,7 @@ fn test_nested() { let expected_array = ListArray::::from_data( expected_data_type, Buffer::from(vec![0, 2, 4, 7, 7, 8]), - Arc::new(expected_values), + Box::new(expected_values), None, ); @@ -301,7 +299,7 @@ fn test_nested() { let expected = ListArray::::from_data( expected_data_type, Buffer::from(vec![0, 2, 5]), - Arc::new(expected_array), + Box::new(expected_array), None, ); diff --git a/tests/it/ffi/data.rs b/tests/it/ffi/data.rs index f4880161ab8..29a0122384d 100644 --- a/tests/it/ffi/data.rs +++ b/tests/it/ffi/data.rs @@ -3,9 +3,8 @@ use arrow2::bitmap::Bitmap; use arrow2::datatypes::{DataType, Field, TimeUnit}; use arrow2::{error::Result, ffi}; use std::collections::BTreeMap; -use std::sync::Arc; -fn _test_round_trip(array: Arc, expected: Box) -> Result<()> { +fn _test_round_trip(array: Box, expected: Box) -> Result<()> { let field = Field::new("a", array.data_type().clone(), true); let array_ptr = Box::new(ffi::ArrowArray::empty()); @@ -33,12 +32,12 @@ fn _test_round_trip(array: Arc, expected: Box) -> Result<( } fn test_round_trip(expected: impl Array + Clone + 'static) -> Result<()> { - let array: Arc = Arc::new(expected.clone()); + let array: Box = Box::new(expected.clone()); let expected = Box::new(expected) as Box; _test_round_trip(array.clone(), clone(expected.as_ref()))?; // sliced - _test_round_trip(array.slice(1, 2).into(), expected.slice(1, 2)) + _test_round_trip(array.slice(1, 2), expected.slice(1, 2)) } fn test_round_trip_schema(field: Field) -> Result<()> { @@ -232,7 +231,7 @@ fn list_sliced() -> Result<()> { let array = ListArray::::try_new( DataType::List(Box::new(Field::new("a", DataType::Int32, true))), vec![0, 1, 1, 2].into(), - Arc::new(PrimitiveArray::::from_vec(vec![1, 2])), + Box::new(PrimitiveArray::::from_vec(vec![1, 2])), Some(bitmap), )?; @@ -277,7 +276,7 @@ fn fixed_size_list_sliced() -> Result<()> { let array = FixedSizeListArray::try_new( DataType::FixedSizeList(Box::new(Field::new("a", DataType::Int32, true)), 2), - Arc::new(PrimitiveArray::::from_vec(vec![1, 2, 3, 4, 5, 6])), + Box::new(PrimitiveArray::::from_vec(vec![1, 2, 3, 4, 5, 6])), Some(bitmap), )?; @@ -308,7 +307,7 @@ fn list_list() -> Result<()> { #[test] fn struct_() -> Result<()> { let data_type = DataType::Struct(vec![Field::new("a", DataType::Int32, true)]); - let values = vec![Int32Array::from([Some(1), None, Some(3)]).arced()]; + let values = vec![Int32Array::from([Some(1), None, Some(3)]).boxed()]; let validity = Bitmap::from([true, false, true]); let array = StructArray::from_data(data_type, values, validity.into()); diff --git a/tests/it/ffi/stream.rs b/tests/it/ffi/stream.rs index 9d99c77e08b..944da6e9b49 100644 --- a/tests/it/ffi/stream.rs +++ b/tests/it/ffi/stream.rs @@ -1,10 +1,8 @@ -use std::sync::Arc; - use arrow2::array::*; use arrow2::datatypes::Field; use arrow2::{error::Result, ffi}; -fn _test_round_trip(arrays: Vec>) -> Result<()> { +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 _; @@ -14,9 +12,9 @@ fn _test_round_trip(arrays: Vec>) -> Result<()> { let mut stream = unsafe { ffi::ArrowArrayStreamReader::try_new(stream)? }; - let mut produced_arrays: Vec> = vec![]; + let mut produced_arrays: Vec> = vec![]; while let Some(array) = unsafe { stream.next() } { - produced_arrays.push(array?.into()); + produced_arrays.push(array?); } assert_eq!(produced_arrays, arrays); @@ -27,7 +25,7 @@ fn _test_round_trip(arrays: Vec>) -> Result<()> { #[test] fn round_trip() -> Result<()> { let array = Int32Array::from(&[Some(2), None, Some(1), None]); - let array: Arc = Arc::new(array); + let array: Box = Box::new(array); _test_round_trip(vec![array.clone(), array.clone(), array]) } diff --git a/tests/it/io/avro/read.rs b/tests/it/io/avro/read.rs index 57e40663513..939c179732f 100644 --- a/tests/it/io/avro/read.rs +++ b/tests/it/io/avro/read.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::chunk::Chunk; use avro_rs::types::{Record, Value}; use avro_rs::{Codec, Writer}; @@ -83,7 +81,7 @@ pub(super) fn schema() -> (AvroSchema, Schema) { (AvroSchema::parse_str(raw_schema).unwrap(), schema) } -pub(super) fn data() -> Chunk> { +pub(super) fn data() -> Chunk> { let data = vec![ Some(vec![Some(1i32), None, Some(3)]), Some(vec![Some(1i32), None, Some(3)]), @@ -93,28 +91,29 @@ pub(super) fn data() -> Chunk> { array.try_extend(data).unwrap(); let columns = vec![ - Int64Array::from_slice([27, 47]).arced(), - Arc::new(Utf8Array::::from_slice(["foo", "bar"])), - Arc::new(Int32Array::from_slice([1, 1])), - Arc::new(Int32Array::from_slice([1, 2]).to(DataType::Date32)), - Arc::new(BinaryArray::::from_slice([b"foo", b"bar"])), - Arc::new(PrimitiveArray::::from_slice([1.0, 2.0])), - Arc::new(BooleanArray::from_slice([true, false])), - Arc::new(Utf8Array::::from([Some("foo"), None])), - array.into_arc(), - Arc::new(StructArray::from_data( + Int64Array::from_slice([27, 47]).boxed(), + Utf8Array::::from_slice(["foo", "bar"]).boxed(), + Int32Array::from_slice([1, 1]).boxed(), + Int32Array::from_slice([1, 2]).to(DataType::Date32).boxed(), + BinaryArray::::from_slice([b"foo", b"bar"]).boxed(), + PrimitiveArray::::from_slice([1.0, 2.0]).boxed(), + BooleanArray::from_slice([true, false]).boxed(), + Utf8Array::::from([Some("foo"), None]).boxed(), + array.into_box(), + StructArray::from_data( DataType::Struct(vec![Field::new("e", DataType::Float64, false)]), - vec![Arc::new(PrimitiveArray::::from_slice([1.0, 2.0]))], + vec![Box::new(PrimitiveArray::::from_slice([1.0, 2.0]))], None, - )), - Arc::new(DictionaryArray::::from_data( + ) + .boxed(), + DictionaryArray::::from_data( Int32Array::from_slice([1, 0]), - Arc::new(Utf8Array::::from_slice(["SPADES", "HEARTS"])), - )), - Arc::new( - PrimitiveArray::::from_slice([12345678i128, -12345678i128]) - .to(DataType::Decimal(18, 5)), - ), + Box::new(Utf8Array::::from_slice(["SPADES", "HEARTS"])), + ) + .boxed(), + PrimitiveArray::::from_slice([12345678i128, -12345678i128]) + .to(DataType::Decimal(18, 5)) + .boxed(), ]; Chunk::try_new(columns).unwrap() @@ -193,7 +192,7 @@ pub(super) fn write_avro(codec: Codec) -> std::result::Result, avro_rs:: pub(super) fn read_avro( mut avro: &[u8], projection: Option>, -) -> Result<(Chunk>, Schema)> { +) -> Result<(Chunk>, Schema)> { let file = &mut avro; let (avro_schema, schema, codec, file_marker) = read::read_metadata(file)?; diff --git a/tests/it/io/avro/write.rs b/tests/it/io/avro/write.rs index 8bf3c2f60b6..06e8d72f112 100644 --- a/tests/it/io/avro/write.rs +++ b/tests/it/io/avro/write.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::array::*; use arrow2::chunk::Chunk; use arrow2::datatypes::*; @@ -82,7 +80,7 @@ pub(super) fn data() -> Chunk> { Box::new(ListArray::::new( list_dt, vec![0, 2, 5].into(), - Arc::new(PrimitiveArray::::from([ + Box::new(PrimitiveArray::::from([ None, Some(1), None, @@ -94,7 +92,7 @@ pub(super) fn data() -> Chunk> { Box::new(ListArray::::new( list_dt1, vec![0, 2, 2].into(), - Arc::new(PrimitiveArray::::from([None, Some(1)])), + Box::new(PrimitiveArray::::from([None, Some(1)])), Some([true, false].into()), )), ]; @@ -262,16 +260,16 @@ fn struct_data() -> Chunk> { Box::new(StructArray::new( struct_dt.clone(), vec![ - Arc::new(PrimitiveArray::::from_slice([1, 2])), - Arc::new(PrimitiveArray::::from([None, Some(1)])), + Box::new(PrimitiveArray::::from_slice([1, 2])), + Box::new(PrimitiveArray::::from([None, Some(1)])), ], None, )), Box::new(StructArray::new( struct_dt, vec![ - Arc::new(PrimitiveArray::::from_slice([1, 2])), - Arc::new(PrimitiveArray::::from([None, Some(1)])), + Box::new(PrimitiveArray::::from_slice([1, 2])), + Box::new(PrimitiveArray::::from([None, Some(1)])), ], Some([true, false].into()), )), diff --git a/tests/it/io/csv/read.rs b/tests/it/io/csv/read.rs index 5bf193c1496..b77326122b7 100644 --- a/tests/it/io/csv/read.rs +++ b/tests/it/io/csv/read.rs @@ -1,7 +1,6 @@ use proptest::prelude::*; use std::io::Cursor; -use std::sync::Arc; use arrow2::array::*; use arrow2::datatypes::*; @@ -89,7 +88,7 @@ fn infer_ints() -> Result<()> { Ok(()) } -fn test_deserialize(input: &str, data_type: DataType) -> Result> { +fn test_deserialize(input: &str, data_type: DataType) -> Result> { let reader = std::io::Cursor::new(input); let mut reader = ReaderBuilder::new().has_headers(false).from_reader(reader); diff --git a/tests/it/io/csv/write.rs b/tests/it/io/csv/write.rs index 817db76bd1b..e17672d6287 100644 --- a/tests/it/io/csv/write.rs +++ b/tests/it/io/csv/write.rs @@ -1,5 +1,4 @@ use std::io::Cursor; -use std::sync::Arc; use arrow2::array::*; use arrow2::chunk::Chunk; @@ -17,7 +16,7 @@ fn data() -> Chunk> { let c6 = PrimitiveArray::::from_vec(vec![1234, 24680, 85563]) .to(DataType::Time32(TimeUnit::Second)); let keys = UInt32Array::from_slice(&[2, 0, 1]); - let c7 = DictionaryArray::from_data(keys, Arc::new(c1.clone())); + let c7 = DictionaryArray::from_data(keys, Box::new(c1.clone())); Chunk::new(vec![ Box::new(c1) as Box, @@ -85,57 +84,57 @@ d|-556132.25|1||2019-04-18 02:45:55.555|11:46:03 PM|c Ok(()) } -fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { +fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { let (array, expected) = match column { 0 => ( - Utf8Array::::from_slice(["a b", "c", "d"]).arced(), + Utf8Array::::from_slice(["a b", "c", "d"]).boxed(), vec!["a b", "c", "d"], ), 1 => ( - BinaryArray::::from_slice(["a b", "c", "d"]).arced(), + BinaryArray::::from_slice(["a b", "c", "d"]).boxed(), vec!["a b", "c", "d"], ), 2 => ( - BinaryArray::::from_slice(["a b", "c", "d"]).arced(), + BinaryArray::::from_slice(["a b", "c", "d"]).boxed(), vec!["a b", "c", "d"], ), 3 => ( - Int8Array::from_slice(&[3, 2, 1]).arced(), + Int8Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 4 => ( - Int16Array::from_slice(&[3, 2, 1]).arced(), + Int16Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 5 => ( - Int32Array::from_slice(&[3, 2, 1]).arced(), + Int32Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 6 => ( - Int64Array::from_slice(&[3, 2, 1]).arced(), + Int64Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 7 => ( - UInt8Array::from_slice(&[3, 2, 1]).arced(), + UInt8Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 8 => ( - UInt16Array::from_slice(&[3, 2, 1]).arced(), + UInt16Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 9 => ( - UInt32Array::from_slice(&[3, 2, 1]).arced(), + UInt32Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 10 => ( - UInt64Array::from_slice(&[3, 2, 1]).arced(), + UInt64Array::from_slice(&[3, 2, 1]).boxed(), vec!["3", "2", "1"], ), 11 => { let array = PrimitiveArray::::from_vec(vec![1_234_001, 24_680_001, 85_563_001]) .to(DataType::Time32(TimeUnit::Millisecond)); ( - array.arced(), + array.boxed(), vec!["00:20:34.001", "06:51:20.001", "23:46:03.001"], ) } @@ -147,7 +146,7 @@ fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { ]) .to(DataType::Time64(TimeUnit::Microsecond)); ( - array.arced(), + array.boxed(), vec!["00:20:34.000001", "06:51:20.000001", "23:46:03.000001"], ) } @@ -159,7 +158,7 @@ fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { ]) .to(DataType::Time64(TimeUnit::Nanosecond)); ( - array.arced(), + array.boxed(), vec![ "00:20:34.000000001", "06:51:20.000000001", @@ -174,7 +173,7 @@ fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { ]) .to(DataType::Timestamp(TimeUnit::Nanosecond, None)); ( - array.arced(), + array.boxed(), vec![ "2019-04-18 10:54:47.378000001", "2019-04-18 02:45:55.555000001", @@ -191,7 +190,7 @@ fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { Some("+01:00".to_string()), )); ( - array.arced(), + array.boxed(), vec![ "2019-04-18 11:54:47.378000001 +01:00", "2019-04-18 03:45:55.555000001 +01:00", @@ -200,9 +199,9 @@ fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { } 16 => { let keys = UInt32Array::from_slice(&[2, 1, 0]); - let values = Utf8Array::::from_slice(["a b", "c", "d"]).arced(); + let values = Utf8Array::::from_slice(["a b", "c", "d"]).boxed(); let array = DictionaryArray::from_data(keys, values); - (array.arced(), vec!["d", "c", "a b"]) + (array.boxed(), vec!["d", "c", "a b"]) } 17 => { let array = PrimitiveArray::::from_slice([ @@ -214,7 +213,7 @@ fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { Some("Europe/Lisbon".to_string()), )); ( - array.arced(), + array.boxed(), vec![ "2019-04-18 11:54:47.378000001 WEST", "2019-04-18 03:45:55.555000001 WEST", @@ -228,7 +227,7 @@ fn data_array(column: usize) -> (Chunk>, Vec<&'static str>) { } fn test_array( - columns: Chunk>, + columns: Chunk>, data: Vec<&'static str>, options: SerializeOptions, ) -> Result<()> { @@ -276,7 +275,7 @@ fn write_tz_timezone_formatted_offset() -> Result<()> { Some("+01:00".to_string()), )); - let columns = Chunk::new(vec![array.arced()]); + let columns = Chunk::new(vec![array.boxed()]); let expected = vec![ "2019-04-18T11:54:47.378000001+01:00", "2019-04-18T03:45:55.555000001+01:00", @@ -301,7 +300,7 @@ fn write_tz_timezone_formatted_tz() -> Result<()> { Some("Europe/Lisbon".to_string()), )); - let columns = Chunk::new(vec![array.arced()]); + let columns = Chunk::new(vec![array.boxed()]); let expected = vec![ "2019-04-18T11:54:47.378000001+01:00", "2019-04-18T03:45:55.555000001+01:00", @@ -320,7 +319,7 @@ fn write_tz_timezone_formatted_tz() -> Result<()> { fn write_empty_and_missing() { let a = Utf8Array::::from(&[Some(""), None]); let b = Utf8Array::::from(&[None, Some("")]); - let columns = Chunk::new(vec![a.arced(), b.arced()]); + let columns = Chunk::new(vec![a.boxed(), b.boxed()]); let mut writer = vec![]; let options = SerializeOptions::default(); @@ -333,7 +332,7 @@ fn write_empty_and_missing() { #[test] fn write_escaping() { let a = Utf8Array::::from_slice(&["Acme co., Ltd."]); - let columns = Chunk::new(vec![a.arced()]); + let columns = Chunk::new(vec![a.boxed()]); let mut writer = vec![]; let options = SerializeOptions::default(); @@ -353,7 +352,7 @@ fn write_escaping_resize_local_buf() { let a = Utf8Array::::from_slice(&[ payload ]); - let columns = Chunk::new(vec![a.arced()]); + let columns = Chunk::new(vec![a.boxed()]); let mut writer = vec![]; let options = SerializeOptions::default(); diff --git a/tests/it/io/ipc/common.rs b/tests/it/io/ipc/common.rs index 0f5270d76e1..45df69f46af 100644 --- a/tests/it/io/ipc/common.rs +++ b/tests/it/io/ipc/common.rs @@ -1,4 +1,4 @@ -use std::{collections::HashMap, fs::File, io::Read, sync::Arc}; +use std::{collections::HashMap, fs::File, io::Read}; use arrow2::{ array::Array, chunk::Chunk, datatypes::Schema, error::Result, @@ -8,7 +8,7 @@ use arrow2::{ use flate2::read::GzDecoder; -type IpcRead = (Schema, Vec, Vec>>); +type IpcRead = (Schema, Vec, Vec>>); /// Read gzipped JSON file pub fn read_gzip_json(version: &str, file_name: &str) -> Result { diff --git a/tests/it/io/ipc/write/file.rs b/tests/it/io/ipc/write/file.rs index 42aea0d0b3d..4b261d97731 100644 --- a/tests/it/io/ipc/write/file.rs +++ b/tests/it/io/ipc/write/file.rs @@ -1,5 +1,4 @@ use std::io::Cursor; -use std::sync::Arc; use arrow2::array::*; use arrow2::chunk::Chunk; @@ -7,11 +6,12 @@ use arrow2::datatypes::{Field, Schema}; use arrow2::error::Result; use arrow2::io::ipc::read::{read_file_metadata, FileReader}; use arrow2::io::ipc::{write::*, IpcField}; +use arrow2::types::months_days_ns; use crate::io::ipc::common::read_gzip_json; pub(crate) fn write( - batches: &[Chunk>], + batches: &[Chunk>], schema: &Schema, ipc_fields: Option>, compression: Option, @@ -27,7 +27,7 @@ pub(crate) fn write( } fn round_trip( - columns: Chunk>, + columns: Chunk>, schema: Schema, ipc_fields: Option>, compression: Option, @@ -327,7 +327,7 @@ fn write_generated_017_union() -> Result<()> { #[test] #[cfg_attr(miri, ignore)] // compression uses FFI, which miri does not support fn write_boolean() -> Result<()> { - let array = BooleanArray::from([Some(true), Some(false), None, Some(true)]).arced(); + let array = BooleanArray::from([Some(true), Some(false), None, Some(true)]).boxed(); let schema = Schema::from(vec![Field::new("a", array.data_type().clone(), true)]); let columns = Chunk::try_new(vec![array])?; round_trip(columns, schema, None, Some(Compression::ZSTD)) @@ -338,7 +338,7 @@ fn write_boolean() -> Result<()> { fn write_sliced_utf8() -> Result<()> { let array = Utf8Array::::from_slice(["aa", "bb"]) .slice(1, 1) - .arced(); + .boxed(); let schema = Schema::from(vec![Field::new("a", array.data_type().clone(), true)]); let columns = Chunk::try_new(vec![array])?; round_trip(columns, schema, None, Some(Compression::ZSTD)) @@ -354,9 +354,22 @@ fn write_sliced_list() -> Result<()> { let mut array = MutableListArray::>::new(); array.try_extend(data).unwrap(); - let array: Arc = array.into_arc().slice(1, 2).into(); + let array: Box = array.into_box().slice(1, 2); let schema = Schema::from(vec![Field::new("a", array.data_type().clone(), true)]); let columns = Chunk::try_new(vec![array])?; round_trip(columns, schema, None, None) } + +#[test] +fn write_months_days_ns() -> Result<()> { + let array = Box::new(MonthsDaysNsArray::from([ + Some(months_days_ns::new(1, 1, 0)), + Some(months_days_ns::new(1, 1, 1)), + None, + Some(months_days_ns::new(1, 1, 2)), + ])) as Box; + let schema = Schema::from(vec![Field::new("a", array.data_type().clone(), true)]); + let columns = Chunk::try_new(vec![array])?; + round_trip(columns, schema, None, None) +} diff --git a/tests/it/io/ipc/write/file_append.rs b/tests/it/io/ipc/write/file_append.rs index 337360a8717..6abec8eabc9 100644 --- a/tests/it/io/ipc/write/file_append.rs +++ b/tests/it/io/ipc/write/file_append.rs @@ -10,7 +10,7 @@ use super::file::write; #[test] fn basic() -> Result<()> { // prepare some data - let array = BooleanArray::from([Some(true), Some(false), None, Some(true)]).arced(); + let array = BooleanArray::from([Some(true), Some(false), None, Some(true)]).boxed(); let schema = Schema::from(vec![Field::new("a", array.data_type().clone(), true)]); let columns = Chunk::try_new(vec![array])?; diff --git a/tests/it/io/ipc/write/stream.rs b/tests/it/io/ipc/write/stream.rs index b28767be073..46f62aa11b4 100644 --- a/tests/it/io/ipc/write/stream.rs +++ b/tests/it/io/ipc/write/stream.rs @@ -1,5 +1,4 @@ use std::io::Cursor; -use std::sync::Arc; use arrow2::array::Array; use arrow2::chunk::Chunk; @@ -16,7 +15,7 @@ use crate::io::ipc::common::read_gzip_json; fn write_( schema: &Schema, ipc_fields: Option>, - batches: &[Chunk>], + batches: &[Chunk>], ) -> Vec { let mut result = vec![]; diff --git a/tests/it/io/ipc/write_file_async.rs b/tests/it/io/ipc/write_file_async.rs index ffa6530feed..7bd76b86be0 100644 --- a/tests/it/io/ipc/write_file_async.rs +++ b/tests/it/io/ipc/write_file_async.rs @@ -1,5 +1,4 @@ use std::io::Cursor; -use std::sync::Arc; use arrow2::array::Array; use arrow2::chunk::Chunk; @@ -18,7 +17,7 @@ use crate::io::ipc::common::read_gzip_json; async fn write_( schema: &Schema, ipc_fields: &[IpcField], - batches: &[Chunk>], + batches: &[Chunk>], ) -> Result> { let mut result = AsyncCursor::new(vec![]); diff --git a/tests/it/io/ipc/write_stream_async.rs b/tests/it/io/ipc/write_stream_async.rs index 2840692a75f..e033d5e9435 100644 --- a/tests/it/io/ipc/write_stream_async.rs +++ b/tests/it/io/ipc/write_stream_async.rs @@ -1,5 +1,4 @@ use std::io::Cursor; -use std::sync::Arc; use arrow2::array::Array; use arrow2::chunk::Chunk; @@ -18,7 +17,7 @@ use crate::io::ipc::common::read_gzip_json; async fn write_( schema: &Schema, ipc_fields: &[IpcField], - batches: &[Chunk>], + batches: &[Chunk>], ) -> Result> { let mut result = AsyncCursor::new(vec![]); diff --git a/tests/it/io/json/mod.rs b/tests/it/io/json/mod.rs index 785789ae0e3..e84971431ea 100644 --- a/tests/it/io/json/mod.rs +++ b/tests/it/io/json/mod.rs @@ -1,8 +1,6 @@ mod read; mod write; -use std::sync::Arc; - use arrow2::array::*; use arrow2::error::Result; use arrow2::io::json::write as json_write; diff --git a/tests/it/io/json/read.rs b/tests/it/io/json/read.rs index e584324931b..11bd93cd264 100644 --- a/tests/it/io/json/read.rs +++ b/tests/it/io/json/read.rs @@ -3,8 +3,6 @@ use arrow2::datatypes::*; use arrow2::error::Result; use arrow2::io::json::read; -use super::*; - #[test] fn read_json() -> Result<()> { let data = br#"[ @@ -27,7 +25,7 @@ fn read_json() -> Result<()> { let expected = StructArray::from_data( DataType::Struct(vec![Field::new("a", DataType::Int64, true)]), - vec![Arc::new(Int64Array::from_slice([1, 2, 3])) as _], + vec![Box::new(Int64Array::from_slice([1, 2, 3])) as _], None, ); diff --git a/tests/it/io/json/write.rs b/tests/it/io/json/write.rs index dc7cd987465..f98ef1fc71f 100644 --- a/tests/it/io/json/write.rs +++ b/tests/it/io/json/write.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::{ array::*, bitmap::Bitmap, @@ -64,7 +62,7 @@ fn struct_() -> Result<()> { 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 array = StructArray::from_data(data_type, vec![Box::new(c1) as _, Box::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}]"#; @@ -85,12 +83,12 @@ fn nested_struct_with_validity() -> Result<()> { let c1 = StructArray::from_data( DataType::Struct(fields), vec![ - Arc::new(Int32Array::from(&[Some(1), None, Some(5)])), - Arc::new(StructArray::from_data( + Box::new(Int32Array::from(&[Some(1), None, Some(5)])), + Box::new(StructArray::from_data( DataType::Struct(inner), vec![ - Arc::new(Utf8Array::::from(&vec![None, Some("f"), Some("g")])), - Arc::new(Int32Array::from(&[Some(20), None, Some(43)])), + Box::new(Utf8Array::::from(&vec![None, Some("f"), Some("g")])), + Box::new(Int32Array::from(&[Some(20), None, Some(43)])), ], Some(Bitmap::from([false, true, true])), )), @@ -103,7 +101,7 @@ fn nested_struct_with_validity() -> Result<()> { 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 array = StructArray::from_data(data_type, vec![Box::new(c1) as _, Box::new(c2)], None); let expected = r#"[{"c1":{"c11":1,"c12":null},"c2":"a"},{"c1":{"c11":null,"c12":{"c121":"f","c122":null}},"c2":"b"},{"c1":null,"c2":"c"}]"#; @@ -121,10 +119,10 @@ fn nested_struct() -> Result<()> { let c1 = StructArray::from_data( DataType::Struct(fields), vec![ - Arc::new(Int32Array::from(&[Some(1), None, Some(5)])), - Arc::new(StructArray::from_data( + Box::new(Int32Array::from(&[Some(1), None, Some(5)])), + Box::new(StructArray::from_data( DataType::Struct(vec![c121]), - vec![Arc::new(Utf8Array::::from(&vec![ + vec![Box::new(Utf8Array::::from(&vec![ Some("e"), Some("f"), Some("g"), @@ -141,7 +139,7 @@ fn nested_struct() -> Result<()> { 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 array = StructArray::from_data(data_type, vec![Box::new(c1) as _, Box::new(c2)], None); 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"}]"#; @@ -170,7 +168,7 @@ fn struct_with_list_field() -> Result<()> { 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 array = StructArray::from_data(data_type, vec![Box::new(c1) as _, Box::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}]"#; @@ -205,7 +203,7 @@ fn nested_list() -> Result<()> { 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 array = StructArray::from_data(data_type, vec![Box::new(c1) as _, Box::new(c2)], None); let expected = r#"[{"c1":[[1,2],[3]],"c2":"foo"},{"c1":[],"c2":"bar"},{"c1":[[4,5,6]],"c2":null}]"#; @@ -229,10 +227,10 @@ fn list_of_struct() -> Result<()> { let s = StructArray::from_data( DataType::Struct(fields), vec![ - Arc::new(Int32Array::from(&[Some(1), None, Some(5)])), - Arc::new(StructArray::from_data( + Box::new(Int32Array::from(&[Some(1), None, Some(5)])), + Box::new(StructArray::from_data( DataType::Struct(inner), - vec![Arc::new(Utf8Array::::from(&vec![ + vec![Box::new(Utf8Array::::from(&vec![ Some("e"), Some("f"), Some("g"), @@ -250,7 +248,7 @@ fn list_of_struct() -> Result<()> { let c1 = ListArray::::from_data( c1_datatype, Buffer::from(vec![0, 2, 2, 3]), - Arc::new(s), + Box::new(s), Some(Bitmap::from_u8_slice([0b00000101], 3)), ); @@ -260,7 +258,7 @@ fn list_of_struct() -> Result<()> { 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 array = StructArray::from_data(data_type, vec![Box::new(c1) as _, Box::new(c2)], None); let expected = r#"[{"c1":[{"c11":1,"c12":null},{"c11":null,"c12":{"c121":"f"}}],"c2":1},{"c1":null,"c2":2},{"c1":[null],"c2":3}]"#; diff --git a/tests/it/io/ndjson/mod.rs b/tests/it/io/ndjson/mod.rs index 0640e81af2b..14e8d6378dc 100644 --- a/tests/it/io/ndjson/mod.rs +++ b/tests/it/io/ndjson/mod.rs @@ -1,7 +1,5 @@ mod read; -use std::sync::Arc; - use arrow2::array::*; use arrow2::bitmap::Bitmap; use arrow2::buffer::Buffer; @@ -42,7 +40,7 @@ fn round_trip_list() -> Result<()> { round_trip(data) } -fn case_list() -> (String, Arc) { +fn case_list() -> (String, Box) { 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"} @@ -87,14 +85,14 @@ fn case_list() -> (String, Arc) { let array = StructArray::from_data( data_type, - vec![a.arced(), b.arced(), c.arced(), d.arced()], + vec![a.boxed(), b.boxed(), c.boxed(), d.boxed()], None, ); - (data, array.arced()) + (data, array.boxed()) } -fn case_dict() -> (String, Arc) { +fn case_dict() -> (String, Box) { 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"]} @@ -133,15 +131,15 @@ fn case_dict() -> (String, Arc) { ( data, - Arc::new(StructArray::from_data( + Box::new(StructArray::from_data( DataType::Struct(fields), - vec![array.arced()], + vec![array.boxed()], None, )), ) } -fn case_basics() -> (String, Arc) { +fn case_basics() -> (String, Box) { 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"}"# @@ -155,17 +153,17 @@ fn case_basics() -> (String, Arc) { let array = StructArray::from_data( data_type, vec![ - Int64Array::from_slice(&[1, -10, 100000000]).arced(), - 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")])), + Int64Array::from_slice(&[1, -10, 100000000]).boxed(), + Float64Array::from_slice(&[2.0, -3.5, 0.6]).boxed(), + BooleanArray::from(&[Some(false), Some(true), None]).boxed(), + Utf8Array::::from(&[Some("4"), None, Some("text")]).boxed(), ], None, ); - (data, Arc::new(array)) + (data, Box::new(array)) } -fn case_projection() -> (String, Arc) { +fn case_projection() -> (String, Box) { 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"}"# @@ -180,21 +178,17 @@ fn case_projection() -> (String, Arc) { let array = StructArray::from_data( data_type, vec![ - UInt32Array::from_slice(&[1, 10, 100000000]).arced(), - 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, - ])), + UInt32Array::from_slice(&[1, 10, 100000000]).boxed(), + Float32Array::from_slice(&[2.0, -3.5, 0.6]).boxed(), + BooleanArray::from(&[Some(false), Some(true), None]).boxed(), + BinaryArray::::from(&[Some(b"4".as_ref()), Some(b"text".as_ref()), None]).boxed(), ], None, ); - (data, Arc::new(array)) + (data, Box::new(array)) } -fn case_struct() -> (String, Arc) { +fn case_struct() -> (String, Box) { let data = r#"{"a": {"b": true, "c": {"d": "text"}}} {"a": {"b": false, "c": null}} {"a": {"b": true, "c": {"d": "text"}}} @@ -217,7 +211,7 @@ fn case_struct() -> (String, Arc) { 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)], + vec![Box::new(d)], Some(Bitmap::from_u8_slice([0b11111101], 4)), ); @@ -225,7 +219,7 @@ fn case_struct() -> (String, Arc) { 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)], + vec![Box::new(b), Box::new(c)], Some(Bitmap::from_u8_slice([0b11110111], 4)), ); @@ -233,15 +227,15 @@ fn case_struct() -> (String, Arc) { ( data, - Arc::new(StructArray::from_data( + Box::new(StructArray::from_data( data_type, - vec![expected.arced()], + vec![expected.boxed()], None, )), ) } -fn case_nested_list() -> (String, Arc) { +fn case_nested_list() -> (String, Box) { 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); @@ -273,7 +267,7 @@ fn case_nested_list() -> (String, Arc) { let c = StructArray::from_data( DataType::Struct(vec![d_field]), - vec![Arc::new(d)], + vec![d.boxed()], Some(Bitmap::from_u8_slice([0b11111011], 6)), ); @@ -287,26 +281,26 @@ fn case_nested_list() -> (String, Arc) { ]); let a_struct = StructArray::from_data( DataType::Struct(vec![b_field, c_field]), - vec![b.arced(), c.arced()], + vec![b.boxed(), c.boxed()], None, ); let expected = ListArray::from_data( a_list_data_type, Buffer::from(vec![0i32, 2, 3, 6, 6, 6]), - a_struct.arced(), + a_struct.boxed(), Some(Bitmap::from_u8_slice([0b00010111], 5)), ); - let array = Arc::new(StructArray::from_data( + let array = Box::new(StructArray::from_data( DataType::Struct(vec![a_field]), - vec![Arc::new(expected)], + vec![Box::new(expected)], None, )); (data, array) } -fn case(case: &str) -> (String, Arc) { +fn case(case: &str) -> (String, Box) { match case { "basics" => case_basics(), "projection" => case_projection(), diff --git a/tests/it/io/ndjson/read.rs b/tests/it/io/ndjson/read.rs index ef3e6882e56..8837402d018 100644 --- a/tests/it/io/ndjson/read.rs +++ b/tests/it/io/ndjson/read.rs @@ -1,5 +1,4 @@ use std::io::Cursor; -use std::sync::Arc; use arrow2::array::*; use arrow2::datatypes::{DataType, Field}; @@ -29,7 +28,7 @@ pub fn read_and_deserialize( ndjson: &str, data_type: &DataType, batch_size: usize, -) -> Result>> { +) -> Result>> { let reader = Cursor::new(ndjson); let mut reader = ndjson_read::FileReader::new(reader, vec!["".to_string(); batch_size], None); @@ -83,7 +82,7 @@ fn read_empty_reader() -> Result<()> { Ok(()) } -fn case_nested_struct() -> (String, Arc) { +fn case_nested_struct() -> (String, Box) { let ndjson = r#"{"a": {"a": 2.0, "b": 2}} {"a": {"b": 2}} {"a": {"a": 2.0, "b": 2, "c": true}} @@ -99,14 +98,14 @@ fn case_nested_struct() -> (String, Arc) { let data_type = DataType::Struct(vec![Field::new("a", inner.clone(), true)]); let values = vec![ - Float64Array::from([Some(2.0), None, Some(2.0), Some(2.0)]).arced(), - Arc::new(Int64Array::from([Some(2), Some(2), Some(2), Some(2)])), - Arc::new(BooleanArray::from([None, None, Some(true), None])), + Float64Array::from([Some(2.0), None, Some(2.0), Some(2.0)]).boxed(), + Int64Array::from([Some(2), Some(2), Some(2), Some(2)]).boxed(), + BooleanArray::from([None, None, Some(true), None]).boxed(), ]; - let values = vec![StructArray::from_data(inner, values, None).arced()]; + let values = vec![StructArray::from_data(inner, values, None).boxed()]; - let array = Arc::new(StructArray::from_data(data_type, values, None)); + let array = Box::new(StructArray::from_data(data_type, values, None)); (ndjson.to_string(), array) } @@ -139,8 +138,8 @@ fn read_nested_struct_batched() -> Result<()> { 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()) + let expected: Vec> = (0..(expected.len() + batch_size - 1) / batch_size) + .map(|offset| expected.slice(offset * batch_size, batch_size)) .collect(); let data_type = infer(&ndjson)?; diff --git a/tests/it/io/parquet/mod.rs b/tests/it/io/parquet/mod.rs index 700f4557586..955df23e663 100644 --- a/tests/it/io/parquet/mod.rs +++ b/tests/it/io/parquet/mod.rs @@ -1,5 +1,4 @@ use std::io::{Cursor, Read, Seek}; -use std::sync::Arc; use arrow2::{ array::*, bitmap::Bitmap, buffer::Buffer, chunk::Chunk, datatypes::*, error::Result, @@ -13,7 +12,7 @@ mod read_indexes; mod write; mod write_async; -type ArrayStats = (Arc, Statistics); +type ArrayStats = (Box, Statistics); pub fn read_column(mut reader: R, column: &str) -> Result { let metadata = read_metadata(&mut reader)?; @@ -87,7 +86,7 @@ pub fn pyarrow_nested_nullable(column: &str) -> Box { Some(9), Some(10), ]) - .arced() + .boxed() } "list_int64_required" | "list_int64_optional_required" | "list_int64_required_required" => { // [[0, 1], None, [2, 0, 3], [4, 5, 6], [], [7, 8, 9], None, [10]] @@ -105,7 +104,7 @@ pub fn pyarrow_nested_nullable(column: &str) -> Box { Some(9), Some(10), ]) - .arced() + .boxed() } "list_int16" => PrimitiveArray::::from(&[ Some(0), @@ -121,7 +120,7 @@ pub fn pyarrow_nested_nullable(column: &str) -> Box { Some(9), Some(10), ]) - .arced(), + .boxed(), "list_bool" => BooleanArray::from(&[ Some(false), Some(true), @@ -136,7 +135,7 @@ pub fn pyarrow_nested_nullable(column: &str) -> Box { Some(false), Some(true), ]) - .arced(), + .boxed(), /* string = [ ["Hello", "bbb"], @@ -149,7 +148,7 @@ pub fn pyarrow_nested_nullable(column: &str) -> Box { [""], ] */ - "list_utf8" => Arc::new(Utf8Array::::from(&[ + "list_utf8" => Box::new(Utf8Array::::from(&[ Some("Hello".to_string()), Some("bbb".to_string()), Some("aa".to_string()), @@ -163,7 +162,7 @@ pub fn pyarrow_nested_nullable(column: &str) -> Box { Some("bbb".to_string()), Some("".to_string()), ])), - "list_large_binary" => Arc::new(BinaryArray::::from(&[ + "list_large_binary" => Box::new(BinaryArray::::from(&[ Some(b"Hello".to_vec()), Some(b"bbb".to_vec()), Some(b"aa".to_vec()), @@ -180,7 +179,7 @@ pub fn pyarrow_nested_nullable(column: &str) -> Box { "list_nested_i64" | "list_nested_inner_required_i64" | "list_nested_inner_required_required_i64" => { - Arc::new(NullArray::from_data(DataType::Null, 1)) + Box::new(NullArray::from_data(DataType::Null, 1)) } other => unreachable!("{}", other), }; @@ -351,7 +350,7 @@ pub fn pyarrow_nullable(column: &str) -> Box { "uint32" => Box::new(PrimitiveArray::::from(u32_values)), "int32_dict" => { let keys = PrimitiveArray::::from([Some(0), Some(1), None, Some(1)]); - let values = Arc::new(PrimitiveArray::::from_slice([10, 200])); + let values = Box::new(PrimitiveArray::::from_slice([10, 200])); Box::new(DictionaryArray::::from_data(keys, values)) } "decimal_9" => { @@ -432,7 +431,7 @@ pub fn pyarrow_nullable_statistics(column: &str) -> Statistics { max_value: Box::new(UInt32Array::from_slice([9])), }, "int32_dict" => { - let new_dict = |array: Arc| -> Box { + let new_dict = |array: Box| -> Box { Box::new(DictionaryArray::::from_data( vec![Some(0)].into(), array, @@ -442,8 +441,8 @@ pub fn pyarrow_nullable_statistics(column: &str) -> Statistics { Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single(UInt64Array::from([Some(0)])), - min_value: new_dict(Arc::new(Int32Array::from_slice([10]))), - max_value: new_dict(Arc::new(Int32Array::from_slice([200]))), + min_value: new_dict(Box::new(Int32Array::from_slice([10]))), + max_value: new_dict(Box::new(Int32Array::from_slice([200]))), } } "decimal_9" => Statistics { @@ -555,7 +554,7 @@ pub fn pyarrow_required_statistics(column: &str) -> Statistics { } pub fn pyarrow_nested_nullable_statistics(column: &str) -> Statistics { - let new_list = |array: Arc, nullable: bool| { + let new_list = |array: Box, nullable: bool| { Box::new(ListArray::::new( DataType::List(Box::new(Field::new( "item", @@ -572,87 +571,69 @@ pub fn pyarrow_nested_nullable_statistics(column: &str) -> Statistics { "list_int16" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single(UInt64Array::from([Some(1)])), - min_value: new_list(Arc::new(Int16Array::from_slice([0])), true), - max_value: new_list(Arc::new(Int16Array::from_slice([10])), true), + min_value: new_list(Box::new(Int16Array::from_slice([0])), true), + max_value: new_list(Box::new(Int16Array::from_slice([10])), true), }, "list_bool" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single(UInt64Array::from([Some(1)])), - min_value: new_list(Arc::new(BooleanArray::from_slice([false])), true), - max_value: new_list(Arc::new(BooleanArray::from_slice([true])), true), + min_value: new_list(Box::new(BooleanArray::from_slice([false])), true), + max_value: new_list(Box::new(BooleanArray::from_slice([true])), true), }, "list_utf8" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(1)].into()), - min_value: new_list(Arc::new(Utf8Array::::from_slice([""])), true), - max_value: new_list(Arc::new(Utf8Array::::from_slice(["ccc"])), true), + min_value: new_list(Box::new(Utf8Array::::from_slice([""])), true), + max_value: new_list(Box::new(Utf8Array::::from_slice(["ccc"])), true), }, "list_large_binary" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(1)].into()), - min_value: new_list(Arc::new(BinaryArray::::from_slice([b""])), true), - max_value: new_list(Arc::new(BinaryArray::::from_slice([b"ccc"])), true), + min_value: new_list(Box::new(BinaryArray::::from_slice([b""])), true), + max_value: new_list(Box::new(BinaryArray::::from_slice([b"ccc"])), true), }, "list_int64" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(1)].into()), - min_value: new_list(Arc::new(Int64Array::from_slice([0])), true), - max_value: new_list(Arc::new(Int64Array::from_slice([10])), true), + min_value: new_list(Box::new(Int64Array::from_slice([0])), true), + max_value: new_list(Box::new(Int64Array::from_slice([10])), true), }, "list_int64_required" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(1)].into()), - min_value: new_list(Arc::new(Int64Array::from_slice([0])), false), - max_value: new_list(Arc::new(Int64Array::from_slice([10])), false), + min_value: new_list(Box::new(Int64Array::from_slice([0])), false), + max_value: new_list(Box::new(Int64Array::from_slice([10])), false), }, "list_int64_required_required" | "list_int64_optional_required" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(0)].into()), - min_value: new_list(Arc::new(Int64Array::from_slice([0])), false), - max_value: new_list(Arc::new(Int64Array::from_slice([10])), false), + min_value: new_list(Box::new(Int64Array::from_slice([0])), false), + max_value: new_list(Box::new(Int64Array::from_slice([10])), false), }, "list_nested_i64" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(2)].into()), - min_value: new_list( - new_list(Arc::new(Int64Array::from_slice([0])), true).into(), - true, - ), - max_value: new_list( - new_list(Arc::new(Int64Array::from_slice([10])), true).into(), - true, - ), + min_value: new_list(new_list(Box::new(Int64Array::from_slice([0])), true), true), + max_value: new_list(new_list(Box::new(Int64Array::from_slice([10])), true), true), }, "list_nested_inner_required_required_i64" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(0)].into()), - min_value: new_list( - new_list(Arc::new(Int64Array::from_slice([0])), true).into(), - true, - ), - max_value: new_list( - new_list(Arc::new(Int64Array::from_slice([10])), true).into(), - true, - ), + min_value: new_list(new_list(Box::new(Int64Array::from_slice([0])), true), true), + max_value: new_list(new_list(Box::new(Int64Array::from_slice([10])), true), true), }, "list_nested_inner_required_i64" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single([Some(0)].into()), - min_value: new_list( - new_list(Arc::new(Int64Array::from_slice([0])), true).into(), - true, - ), - max_value: new_list( - new_list(Arc::new(Int64Array::from_slice([10])), true).into(), - true, - ), + min_value: new_list(new_list(Box::new(Int64Array::from_slice([0])), true), true), + max_value: new_list(new_list(Box::new(Int64Array::from_slice([10])), true), true), }, other => todo!("{}", other), } } pub fn pyarrow_nested_edge_statistics(column: &str) -> Statistics { - let new_list = |array: Arc| { + let new_list = |array: Box| { Box::new(ListArray::::new( DataType::List(Box::new(Field::new( "item", @@ -669,14 +650,14 @@ pub fn pyarrow_nested_edge_statistics(column: &str) -> Statistics { "simple" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single(UInt64Array::from([Some(0)])), - min_value: new_list(Arc::new(Int64Array::from([Some(0)]))), - max_value: new_list(Arc::new(Int64Array::from([Some(1)]))), + min_value: new_list(Box::new(Int64Array::from([Some(0)]))), + max_value: new_list(Box::new(Int64Array::from([Some(1)]))), }, "null" => Statistics { distinct_count: Count::Single(UInt64Array::from([None])), null_count: Count::Single(UInt64Array::from([Some(1)])), - min_value: new_list(Arc::new(Int64Array::from([None]))), - max_value: new_list(Arc::new(Int64Array::from([None]))), + min_value: new_list(Box::new(Int64Array::from([None]))), + max_value: new_list(Box::new(Int64Array::from([None]))), }, _ => unreachable!(), } @@ -695,7 +676,7 @@ pub fn pyarrow_struct(column: &str) -> Box { Some(true), Some(true), ]; - let boolean = BooleanArray::from(boolean).arced(); + let boolean = BooleanArray::from(boolean).boxed(); let fields = vec![ Field::new("f1", DataType::Utf8, true), Field::new("f2", DataType::Boolean, true), @@ -714,15 +695,11 @@ pub fn pyarrow_struct(column: &str) -> Box { Some("def"), Some("aaa"), ]; - let values = vec![Utf8Array::::from(string).arced(), boolean]; - Box::new(StructArray::from_data( - DataType::Struct(fields), - values, - None, - )) + let values = vec![Utf8Array::::from(string).boxed(), boolean]; + StructArray::from_data(DataType::Struct(fields), values, None).boxed() } "struct_struct" => { - let struct_ = pyarrow_struct("struct").into(); + let struct_ = pyarrow_struct("struct"); let values = vec![struct_, boolean]; Box::new(StructArray::from_data( DataType::Struct(vec![ @@ -738,7 +715,7 @@ pub fn pyarrow_struct(column: &str) -> Box { } pub fn pyarrow_struct_statistics(column: &str) -> Statistics { - let new_struct = |arrays: Vec>, names: Vec| { + let new_struct = |arrays: Vec>, names: Vec| { let fields = names .into_iter() .zip(arrays.iter()) @@ -753,29 +730,29 @@ pub fn pyarrow_struct_statistics(column: &str) -> Statistics { "struct" => Statistics { distinct_count: Count::Struct(new_struct( vec![ - Arc::new(UInt64Array::from([None])), - Arc::new(UInt64Array::from([None])), + Box::new(UInt64Array::from([None])), + Box::new(UInt64Array::from([None])), ], names.clone(), )), null_count: Count::Struct(new_struct( vec![ - Arc::new(UInt64Array::from([Some(4)])), - Arc::new(UInt64Array::from([Some(4)])), + Box::new(UInt64Array::from([Some(4)])), + Box::new(UInt64Array::from([Some(4)])), ], names.clone(), )), min_value: Box::new(new_struct( vec![ - Arc::new(Utf8Array::::from_slice([""])), - Arc::new(BooleanArray::from_slice([false])), + Box::new(Utf8Array::::from_slice([""])), + Box::new(BooleanArray::from_slice([false])), ], names.clone(), )), max_value: Box::new(new_struct( vec![ - Arc::new(Utf8Array::::from_slice(["def"])), - Arc::new(BooleanArray::from_slice([true])), + Box::new(Utf8Array::::from_slice(["def"])), + Box::new(BooleanArray::from_slice([true])), ], names, )), @@ -805,13 +782,13 @@ pub fn pyarrow_map(column: &str) -> Box { StructArray::try_new( dt, vec![ - Utf8Array::::from(s1).arced(), - Utf8Array::::from(s2).arced(), + Utf8Array::::from(s1).boxed(), + Utf8Array::::from(s2).boxed(), ], None, ) .unwrap() - .arced(), + .boxed(), None, ) .unwrap() @@ -830,13 +807,13 @@ pub fn pyarrow_map(column: &str) -> Box { StructArray::try_new( dt, vec![ - Utf8Array::::from(s1).arced(), - Utf8Array::::from(s2).arced(), + Utf8Array::::from(s1).boxed(), + Utf8Array::::from(s2).boxed(), ], None, ) .unwrap() - .arced(), + .boxed(), None, ) .unwrap() @@ -847,7 +824,7 @@ pub fn pyarrow_map(column: &str) -> Box { } pub fn pyarrow_map_statistics(column: &str) -> Statistics { - let new_map = |arrays: Vec>, names: Vec| { + let new_map = |arrays: Vec>, names: Vec| { let fields = names .into_iter() .zip(arrays.iter()) @@ -859,7 +836,7 @@ pub fn pyarrow_map_statistics(column: &str) -> Statistics { false, ), vec![0, arrays[0].len() as i32].into(), - StructArray::new(DataType::Struct(fields), arrays, None).arced(), + StructArray::new(DataType::Struct(fields), arrays, None).boxed(), None, ) }; @@ -870,29 +847,29 @@ pub fn pyarrow_map_statistics(column: &str) -> Statistics { "map" => Statistics { distinct_count: Count::Map(new_map( vec![ - Arc::new(UInt64Array::from([None])), - Arc::new(UInt64Array::from([None])), + UInt64Array::from([None]).boxed(), + UInt64Array::from([None]).boxed(), ], names.clone(), )), null_count: Count::Map(new_map( vec![ - Arc::new(UInt64Array::from([Some(0)])), - Arc::new(UInt64Array::from([Some(0)])), + UInt64Array::from([Some(0)]).boxed(), + UInt64Array::from([Some(0)]).boxed(), ], names.clone(), )), min_value: Box::new(new_map( vec![ - Arc::new(Utf8Array::::from_slice(["a1"])), - Arc::new(Utf8Array::::from_slice(["b1"])), + Utf8Array::::from_slice(["a1"]).boxed(), + Utf8Array::::from_slice(["b1"]).boxed(), ], names.clone(), )), max_value: Box::new(new_map( vec![ - Arc::new(Utf8Array::::from_slice(["a2"])), - Arc::new(Utf8Array::::from_slice(["b2"])), + Utf8Array::::from_slice(["a2"]).boxed(), + Utf8Array::::from_slice(["b2"]).boxed(), ], names, )), @@ -900,29 +877,29 @@ pub fn pyarrow_map_statistics(column: &str) -> Statistics { "map_nullable" => Statistics { distinct_count: Count::Map(new_map( vec![ - Arc::new(UInt64Array::from([None])), - Arc::new(UInt64Array::from([None])), + UInt64Array::from([None]).boxed(), + UInt64Array::from([None]).boxed(), ], names.clone(), )), null_count: Count::Map(new_map( vec![ - Arc::new(UInt64Array::from([Some(0)])), - Arc::new(UInt64Array::from([Some(1)])), + UInt64Array::from([Some(0)]).boxed(), + UInt64Array::from([Some(1)]).boxed(), ], names.clone(), )), min_value: Box::new(new_map( vec![ - Arc::new(Utf8Array::::from_slice(["a1"])), - Arc::new(Utf8Array::::from_slice(["b1"])), + Utf8Array::::from_slice(["a1"]).boxed(), + Utf8Array::::from_slice(["b1"]).boxed(), ], names.clone(), )), max_value: Box::new(new_map( vec![ - Arc::new(Utf8Array::::from_slice(["a2"])), - Arc::new(Utf8Array::::from_slice(["b1"])), + Utf8Array::::from_slice(["a2"]).boxed(), + Utf8Array::::from_slice(["b1"]).boxed(), ], names, )), @@ -931,7 +908,7 @@ pub fn pyarrow_map_statistics(column: &str) -> Statistics { } } -fn integration_write(schema: &Schema, batches: &[Chunk>]) -> Result> { +fn integration_write(schema: &Schema, batches: &[Chunk>]) -> Result> { let options = WriteOptions { write_statistics: true, compression: CompressionOptions::Uncompressed, @@ -965,7 +942,7 @@ fn integration_write(schema: &Schema, batches: &[Chunk>]) -> Resu Ok(writer.into_inner().into_inner()) } -type IntegrationRead = (Schema, Vec>>); +type IntegrationRead = (Schema, Vec>>); fn integration_read(data: &[u8]) -> Result { let reader = Cursor::new(data); @@ -991,17 +968,17 @@ fn arrow_type() -> Result<()> { let indices = PrimitiveArray::from_values((0..3u64).map(|x| x % 2)); let values = PrimitiveArray::from_slice([1.0f32, 3.0]); - let array3 = DictionaryArray::from_data(indices.clone(), std::sync::Arc::new(values)); + let array3 = DictionaryArray::from_data(indices.clone(), Box::new(values)); let values = BinaryArray::::from_slice([b"ab", b"ac"]); - let array4 = DictionaryArray::from_data(indices.clone(), std::sync::Arc::new(values)); + let array4 = DictionaryArray::from_data(indices.clone(), Box::new(values)); let values = FixedSizeBinaryArray::from_data( DataType::FixedSizeBinary(2), vec![b'a', b'b', b'a', b'c'].into(), None, ); - let array5 = DictionaryArray::from_data(indices, std::sync::Arc::new(values)); + let array5 = DictionaryArray::from_data(indices, Box::new(values)); let schema = Schema::from(vec![ Field::new("a1", dt1, true), @@ -1012,12 +989,12 @@ fn arrow_type() -> Result<()> { Field::new("a6", array5.data_type().clone(), false), ]); let batch = Chunk::try_new(vec![ - array.arced(), - Arc::new(array2), - Arc::new(array3), - Arc::new(array4), - Arc::new(array5.clone()), - Arc::new(array5), + array.boxed(), + array2.boxed(), + array3.boxed(), + array4.boxed(), + array5.clone().boxed(), + array5.boxed(), ])?; let r = integration_write(&schema, &[batch.clone()])?; @@ -1057,7 +1034,7 @@ fn list_array_generic(inner_is_nullable: bool, is_nullable: bool) -> Result<()> array.data_type().clone(), is_nullable, )]); - let batch = Chunk::try_new(vec![array.arced()])?; + let batch = Chunk::try_new(vec![array.boxed()])?; let r = integration_write(&schema, &[batch.clone()])?; diff --git a/tests/it/io/parquet/read_indexes.rs b/tests/it/io/parquet/read_indexes.rs index 3f794a1cc56..52770452c5b 100644 --- a/tests/it/io/parquet/read_indexes.rs +++ b/tests/it/io/parquet/read_indexes.rs @@ -1,5 +1,4 @@ use std::io::Cursor; -use std::sync::Arc; use arrow2::error::Error; use arrow2::{array::*, datatypes::*, error::Result, io::parquet::read::*, io::parquet::write::*}; @@ -74,7 +73,7 @@ fn pages( /// Tests reading pages while skipping indexes fn read_with_indexes( (pages1, pages2, schema): (Vec, Vec, Schema), - expected: Arc, + expected: Box, ) -> Result<()> { let options = WriteOptions { write_statistics: true, @@ -138,7 +137,7 @@ fn read_with_indexes( fn indexed_required_utf8() -> Result<()> { let array21 = Utf8Array::::from_slice(["a", "b", "c"]); let array22 = Utf8Array::::from_slice(["d", "e", "f"]); - let expected = Utf8Array::::from_slice(["e"]).arced(); + let expected = Utf8Array::::from_slice(["e"]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -147,7 +146,7 @@ fn indexed_required_utf8() -> Result<()> { fn indexed_required_i32() -> Result<()> { let array21 = Int32Array::from_slice([1, 2, 3]); let array22 = Int32Array::from_slice([4, 5, 6]); - let expected = Int32Array::from_slice([5]).arced(); + let expected = Int32Array::from_slice([5]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -156,7 +155,7 @@ fn indexed_required_i32() -> Result<()> { fn indexed_optional_i32() -> Result<()> { let array21 = Int32Array::from([Some(1), Some(2), None]); let array22 = Int32Array::from([None, Some(5), Some(6)]); - let expected = Int32Array::from_slice([5]).arced(); + let expected = Int32Array::from_slice([5]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -165,7 +164,7 @@ fn indexed_optional_i32() -> Result<()> { fn indexed_optional_utf8() -> Result<()> { let array21 = Utf8Array::::from([Some("a"), Some("b"), None]); let array22 = Utf8Array::::from([None, Some("e"), Some("f")]); - let expected = Utf8Array::::from_slice(["e"]).arced(); + let expected = Utf8Array::::from_slice(["e"]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -174,7 +173,7 @@ fn indexed_optional_utf8() -> Result<()> { fn indexed_required_fixed_len() -> Result<()> { let array21 = FixedSizeBinaryArray::from_slice([[127], [128], [129]]); let array22 = FixedSizeBinaryArray::from_slice([[130], [131], [132]]); - let expected = FixedSizeBinaryArray::from_slice([[131]]).arced(); + let expected = FixedSizeBinaryArray::from_slice([[131]]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -183,7 +182,7 @@ fn indexed_required_fixed_len() -> Result<()> { fn indexed_optional_fixed_len() -> Result<()> { let array21 = FixedSizeBinaryArray::from([Some([127]), Some([128]), None]); let array22 = FixedSizeBinaryArray::from([None, Some([131]), Some([132])]); - let expected = FixedSizeBinaryArray::from_slice([[131]]).arced(); + let expected = FixedSizeBinaryArray::from_slice([[131]]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -192,7 +191,7 @@ fn indexed_optional_fixed_len() -> Result<()> { fn indexed_required_boolean() -> Result<()> { let array21 = BooleanArray::from_slice([true, false, true]); let array22 = BooleanArray::from_slice([false, false, true]); - let expected = BooleanArray::from_slice([false]).arced(); + let expected = BooleanArray::from_slice([false]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -201,7 +200,7 @@ fn indexed_required_boolean() -> Result<()> { fn indexed_optional_boolean() -> Result<()> { let array21 = BooleanArray::from([Some(true), Some(false), None]); let array22 = BooleanArray::from([None, Some(false), Some(true)]); - let expected = BooleanArray::from_slice([false]).arced(); + let expected = BooleanArray::from_slice([false]).boxed(); read_with_indexes(pages(&[&array21, &array22], Encoding::Plain)?, expected) } @@ -210,13 +209,13 @@ fn indexed_optional_boolean() -> Result<()> { fn indexed_dict() -> Result<()> { let indices = PrimitiveArray::from_values((0..6u64).map(|x| x % 2)); let values = PrimitiveArray::from_slice([4i32, 6i32]); - let array = DictionaryArray::from_data(indices, std::sync::Arc::new(values)); + let array = DictionaryArray::from_data(indices, Box::new(values)); let indices = PrimitiveArray::from_slice(&[0u64]); let values = PrimitiveArray::from_slice([4i32, 6i32]); - let expected = DictionaryArray::from_data(indices, std::sync::Arc::new(values)); + let expected = DictionaryArray::from_data(indices, Box::new(values)); - let expected = expected.arced(); + let expected = expected.boxed(); read_with_indexes(pages(&[&array], Encoding::RleDictionary)?, expected) } diff --git a/tests/it/io/parquet/write.rs b/tests/it/io/parquet/write.rs index 471b9d72afd..343bd419f3b 100644 --- a/tests/it/io/parquet/write.rs +++ b/tests/it/io/parquet/write.rs @@ -28,7 +28,6 @@ fn round_trip( "struct" => (pyarrow_struct(column), pyarrow_struct_statistics(column)), _ => unreachable!(), }; - let array: Arc = array.into(); let field = Field::new("a1", array.data_type().clone(), true); let schema = Schema::from(vec![field]); diff --git a/tests/it/io/parquet/write_async.rs b/tests/it/io/parquet/write_async.rs index c322e7c5238..89d76efea92 100644 --- a/tests/it/io/parquet/write_async.rs +++ b/tests/it/io/parquet/write_async.rs @@ -20,7 +20,7 @@ async fn test_parquet_async_roundtrip() { for i in 0..5 { let a1 = Int32Array::from(&[Some(i), None, Some(i + 1)]); let a2 = Float32Array::from(&[None, Some(i as f32), None]); - let chunk = Chunk::new(vec![a1.arced(), a2.arced()]); + let chunk = Chunk::new(vec![a1.boxed(), a2.boxed()]); data.push(chunk); } let schema = Schema::from(vec![ diff --git a/tests/it/io/print.rs b/tests/it/io/print.rs index 70f9bbf258e..a2f4eb5c9b7 100644 --- a/tests/it/io/print.rs +++ b/tests/it/io/print.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::{ array::*, bitmap::Bitmap, @@ -99,7 +97,7 @@ fn write_dictionary() -> Result<()> { fn dictionary_validities() -> Result<()> { let keys = PrimitiveArray::::from([Some(1), None, Some(0)]); let values = PrimitiveArray::::from([None, Some(10)]); - let array = DictionaryArray::::from_data(keys, Arc::new(values)); + let array = DictionaryArray::::from_data(keys, Box::new(values)); let columns = Chunk::new(vec![&array as &dyn Array]); @@ -323,8 +321,8 @@ fn write_struct() -> Result<()> { Field::new("b", DataType::Utf8, true), ]; let values = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).boxed(), ]; let validity = Some(Bitmap::from(&[true, false, true])); @@ -361,8 +359,8 @@ fn write_union() -> Result<()> { let data_type = DataType::Union(fields, None, UnionMode::Sparse); let types = Buffer::from(vec![0, 0, 1]); let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).arced(), - Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).arced(), + Int32Array::from(&[Some(1), None, Some(2)]).boxed(), + Utf8Array::::from(&[Some("a"), Some("b"), Some("c")]).boxed(), ]; let array = UnionArray::from_data(data_type, types, fields, None); diff --git a/tests/it/scalar/fixed_size_list.rs b/tests/it/scalar/fixed_size_list.rs index a377d6afa0e..89809d343a2 100644 --- a/tests/it/scalar/fixed_size_list.rs +++ b/tests/it/scalar/fixed_size_list.rs @@ -10,7 +10,7 @@ fn equal() { let dt = DataType::FixedSizeList(Box::new(Field::new("a", DataType::Boolean, true)), 2); let a = FixedSizeListScalar::new( dt.clone(), - Some(BooleanArray::from_slice([true, false]).arced()), + Some(BooleanArray::from_slice([true, false]).boxed()), ); let b = FixedSizeListScalar::new(dt.clone(), None); @@ -19,7 +19,7 @@ fn equal() { assert_eq!(b, b); assert!(a != b); - let b = FixedSizeListScalar::new(dt, Some(BooleanArray::from_slice([true, true]).arced())); + let b = FixedSizeListScalar::new(dt, Some(BooleanArray::from_slice([true, true]).boxed())); assert!(a != b); assert_eq!(b, b); } @@ -29,7 +29,7 @@ fn basics() { let dt = DataType::FixedSizeList(Box::new(Field::new("a", DataType::Boolean, true)), 2); let a = FixedSizeListScalar::new( dt.clone(), - Some(BooleanArray::from_slice([true, false]).arced()), + Some(BooleanArray::from_slice([true, false]).boxed()), ); assert_eq!( diff --git a/tests/it/scalar/list.rs b/tests/it/scalar/list.rs index c0e4243e2f2..d8954e6bba0 100644 --- a/tests/it/scalar/list.rs +++ b/tests/it/scalar/list.rs @@ -10,13 +10,13 @@ fn equal() { let dt = DataType::List(Box::new(Field::new("a", DataType::Boolean, true))); let a = ListScalar::::new( dt.clone(), - Some(BooleanArray::from_slice([true, false]).arced()), + Some(BooleanArray::from_slice([true, false]).boxed()), ); let b = ListScalar::::new(dt.clone(), None); assert_eq!(a, a); assert_eq!(b, b); assert!(a != b); - let b = ListScalar::::new(dt, Some(BooleanArray::from_slice([true, true]).arced())); + let b = ListScalar::::new(dt, Some(BooleanArray::from_slice([true, true]).boxed())); assert!(a != b); assert_eq!(b, b); } @@ -26,7 +26,7 @@ fn basics() { let dt = DataType::List(Box::new(Field::new("a", DataType::Boolean, true))); let a = ListScalar::::new( dt.clone(), - Some(BooleanArray::from_slice([true, false]).arced()), + Some(BooleanArray::from_slice([true, false]).boxed()), ); assert_eq!(BooleanArray::from_slice([true, false]), a.values().as_ref()); diff --git a/tests/it/scalar/mod.rs b/tests/it/scalar/mod.rs index 2e7d105d7fd..9f89d41f863 100644 --- a/tests/it/scalar/mod.rs +++ b/tests/it/scalar/mod.rs @@ -11,5 +11,5 @@ mod utf8; // check that `PartialEq` can be derived #[derive(PartialEq)] struct A { - array: std::sync::Arc, + array: Box, } diff --git a/tests/it/scalar/struct_.rs b/tests/it/scalar/struct_.rs index 9209afee8c9..2785ecb7b41 100644 --- a/tests/it/scalar/struct_.rs +++ b/tests/it/scalar/struct_.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use arrow2::{ datatypes::{DataType, Field}, scalar::{BooleanScalar, Scalar, StructScalar}, @@ -12,7 +10,7 @@ fn equal() { let a = StructScalar::new( dt.clone(), Some(vec![ - Arc::new(BooleanScalar::from(Some(true))) as Arc + Box::new(BooleanScalar::from(Some(true))) as Box ]), ); let b = StructScalar::new(dt.clone(), None); @@ -22,7 +20,7 @@ fn equal() { let b = StructScalar::new( dt, Some(vec![ - Arc::new(BooleanScalar::from(Some(false))) as Arc + Box::new(BooleanScalar::from(Some(false))) as Box ]), ); assert!(a != b); @@ -33,7 +31,7 @@ fn equal() { fn basics() { let dt = DataType::Struct(vec![Field::new("a", DataType::Boolean, true)]); - let values = vec![Arc::new(BooleanScalar::from(Some(true))) as Arc]; + let values = vec![Box::new(BooleanScalar::from(Some(true))) as Box]; let a = StructScalar::new(dt.clone(), Some(values.clone()));