diff --git a/.github/workflows/tpcds-reusable.yml b/.github/workflows/tpcds-reusable.yml index fc039fdaa..91f12fc05 100644 --- a/.github/workflows/tpcds-reusable.yml +++ b/.github/workflows/tpcds-reusable.yml @@ -151,6 +151,12 @@ jobs: components: cargo rustfmt + clippy + + - name: Cargo clippy + run: | +# First eliminate unwrap; then enable -D warnings to enforce all default lints. + cargo clippy --all-targets --workspace -- -A warnings -A clippy::all -D clippy::unwrap_used - name: Cargo test run: | diff --git a/Cargo.lock b/Cargo.lock index b915a5f51..75b25c4e7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -360,6 +360,7 @@ dependencies = [ "log", "once_cell", "panic-message", + "parking_lot", "poem", "pprof", "prost 0.14.1", diff --git a/Cargo.toml b/Cargo.toml index baf0915b3..1f2567dd9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -27,6 +27,10 @@ members = [ "native-engine/auron-memmgr", ] +[workspace.lints.clippy] +unwrap_used = "deny" +panic = "deny" + [profile.release] opt-level = 3 lto = true diff --git a/dev/mvn-build-helper/build-native.sh b/dev/mvn-build-helper/build-native.sh index f97eddb17..c378f775a 100755 --- a/dev/mvn-build-helper/build-native.sh +++ b/dev/mvn-build-helper/build-native.sh @@ -84,6 +84,10 @@ if [ ! -f "$cache_libpath" ] || [ "$new_checksum" != "$old_checksum" ]; then echo "Running cargo fmt..." cargo fmt --all -q -- 2>&1 + echo "Running cargo clippy..." + # First eliminate unwrap; then enable -D warnings to enforce all default lints. + cargo clippy --all-targets --workspace -- -A warnings -A clippy::all -D clippy::unwrap_used 2>&1 + echo "Building native with [$profile] profile..." cargo build --profile="$profile" $features_arg --verbose --locked --frozen 2>&1 diff --git a/native-engine/auron-jni-bridge/src/jni_bridge.rs b/native-engine/auron-jni-bridge/src/jni_bridge.rs index 90614cc23..4305d33c1 100644 --- a/native-engine/auron-jni-bridge/src/jni_bridge.rs +++ b/native-engine/auron-jni-bridge/src/jni_bridge.rs @@ -87,9 +87,13 @@ macro_rules! jni_map_error_with_env { match $result { Ok(result) => $crate::jni_bridge::datafusion::error::Result::Ok(result), Err($crate::jni_bridge::jni::errors::Error::JavaException) => { - let ex = $env.exception_occurred().unwrap(); - $env.exception_describe().unwrap(); - $env.exception_clear().unwrap(); + let ex = $env + .exception_occurred() + .expect("failed to obtain pending Java execption object"); + $env.exception_describe() + .expect("failed to print Java exception to stderr"); + $env.exception_clear() + .expect("failed to clear pending Java exception"); let message_obj = $env .call_method_unchecked( ex, @@ -102,13 +106,13 @@ macro_rules! jni_map_error_with_env { .clone(), &[], ) - .unwrap() + .expect("call Java Throwable.toString() failed") .l() - .unwrap(); + .expect("expected object return from Throwable.toString()"); let message = $env .get_string(message_obj.into()) .map(|s| String::from(s)) - .unwrap(); + .expect("failed to read Throwable.toString() result as Java string"); Err( $crate::jni_bridge::datafusion::error::DataFusionError::External( diff --git a/native-engine/auron-jni-bridge/src/lib.rs b/native-engine/auron-jni-bridge/src/lib.rs index bee861f65..7bdb77a0e 100644 --- a/native-engine/auron-jni-bridge/src/lib.rs +++ b/native-engine/auron-jni-bridge/src/lib.rs @@ -34,7 +34,9 @@ pub fn ensure_jni_bridge_inited() -> Result<()> { pub fn is_task_running() -> bool { fn is_task_running_impl() -> Result { - if !jni_call_static!(JniBridge.isTaskRunning() -> bool).unwrap() { + if !jni_call_static!(JniBridge.isTaskRunning() -> bool) + .expect("calling JniBridge.isTaskRunning() error") + { jni_exception_clear!()?; return Ok(false); } diff --git a/native-engine/auron-serde/src/from_proto.rs b/native-engine/auron-serde/src/from_proto.rs index 31082e8ba..7b4ebc43e 100644 --- a/native-engine/auron-serde/src/from_proto.rs +++ b/native-engine/auron-serde/src/from_proto.rs @@ -151,7 +151,11 @@ impl TryInto> for &protobuf::PhysicalPlanNode { Ok(Arc::new(FilterExec::try_new(predicates, input)?)) } PhysicalPlanType::ParquetScan(scan) => { - let conf: FileScanConfig = scan.base_conf.as_ref().unwrap().try_into()?; + let conf: FileScanConfig = scan + .base_conf + .as_ref() + .expect("base_conf must be set for ParquetScan") + .try_into()?; let predicate = scan .pruning_predicates .iter() @@ -168,7 +172,11 @@ impl TryInto> for &protobuf::PhysicalPlanNode { ))) } PhysicalPlanType::OrcScan(scan) => { - let conf: FileScanConfig = scan.base_conf.as_ref().unwrap().try_into()?; + let conf: FileScanConfig = scan + .base_conf + .as_ref() + .expect("base_conf must be set for OrcScan") + .try_into()?; let predicate = scan .pruning_predicates .iter() @@ -192,10 +200,18 @@ impl TryInto> for &protobuf::PhysicalPlanNode { .on .iter() .map(|col| { - let left_key = - try_parse_physical_expr(&col.left.as_ref().unwrap(), &left.schema())?; - let right_key = - try_parse_physical_expr(&col.right.as_ref().unwrap(), &right.schema())?; + let left_key = try_parse_physical_expr( + &col.left + .as_ref() + .expect("hash join: left join key must be present"), + &left.schema(), + )?; + let right_key = try_parse_physical_expr( + &col.right + .as_ref() + .expect("hash join: right join key must be present"), + &right.schema(), + )?; Ok((left_key, right_key)) }) .collect::>()?; @@ -229,10 +245,18 @@ impl TryInto> for &protobuf::PhysicalPlanNode { .on .iter() .map(|col| { - let left_key = - try_parse_physical_expr(&col.left.as_ref().unwrap(), &left.schema())?; - let right_key = - try_parse_physical_expr(&col.right.as_ref().unwrap(), &right.schema())?; + let left_key = try_parse_physical_expr( + &col.left + .as_ref() + .expect("sort-merge join: left join key must be present"), + &left.schema(), + )?; + let right_key = try_parse_physical_expr( + &col.right + .as_ref() + .expect("sort-merge join: right join key must be present"), + &right.schema(), + )?; Ok((left_key, right_key)) }) .collect::>()?; @@ -270,7 +294,7 @@ impl TryInto> for &protobuf::PhysicalPlanNode { Ok(Arc::new(ShuffleWriterExec::try_new( input, - output_partitioning.unwrap(), + output_partitioning.expect("shuffle writer: output_partitioning must be set"), shuffle_writer.output_data_file.clone(), shuffle_writer.output_index_file.clone(), )?)) @@ -285,7 +309,8 @@ impl TryInto> for &protobuf::PhysicalPlanNode { )?; Ok(Arc::new(RssShuffleWriterExec::try_new( input, - output_partitioning.unwrap(), + output_partitioning + .expect("rss shuffle writer: output_partitioning must be set"), rss_shuffle_writer.rss_partition_writer_resource_id.clone(), )?)) } @@ -339,10 +364,18 @@ impl TryInto> for &protobuf::PhysicalPlanNode { .on .iter() .map(|col| { - let left_key = - try_parse_physical_expr(&col.left.as_ref().unwrap(), &left.schema())?; - let right_key = - try_parse_physical_expr(&col.right.as_ref().unwrap(), &right.schema())?; + let left_key = try_parse_physical_expr( + &col.left + .as_ref() + .expect("broadcast join: left join key must be present"), + &left.schema(), + )?; + let right_key = try_parse_physical_expr( + &col.right + .as_ref() + .expect("broadcast join: right join key must be present"), + &right.schema(), + )?; Ok((left_key, right_key)) }) .collect::>()?; @@ -471,7 +504,7 @@ impl TryInto> for &protobuf::PhysicalPlanNode { let agg = match AggFunction::from(agg_function) { AggFunction::Udaf => { - let udaf = agg_node.udaf.as_ref().unwrap(); + let udaf = agg_node.udaf.as_ref().expect("udaf missing"); let serialized = udaf.serialized.clone(); create_udaf_agg(serialized, return_type, agg_children_exprs)? } @@ -696,7 +729,7 @@ impl TryInto> for &protobuf::PhysicalPlanNode { children, )?, GenerateFunction::Udtf => { - let udtf = pb_generator.udtf.as_ref().unwrap(); + let udtf = pb_generator.udtf.as_ref().expect("udtf missing"); let serialized = udtf.serialized.clone(); let return_schema = Arc::new(convert_required!(udtf.return_schema)?); create_udtf_generator(serialized, return_schema, children)? @@ -1136,13 +1169,19 @@ pub fn parse_protobuf_partitioning( .collect::, _>>()?; Ok(Some(Partitioning::HashPartitioning( expr, - hash_part.partition_count.try_into().unwrap(), + hash_part + .partition_count + .try_into() + .expect("hash repartition: invalid partition_count"), ))) } RepartitionType::RoundRobinRepartition(round_robin_part) => { Ok(Some(Partitioning::RoundRobinPartitioning( - round_robin_part.partition_count.try_into().unwrap(), + round_robin_part + .partition_count + .try_into() + .expect("round-robin repartition: invalid partition_count"), ))) } @@ -1150,7 +1189,7 @@ pub fn parse_protobuf_partitioning( if range_part.partition_count == 1 { Ok(Some(Partitioning::SinglePartitioning())) } else { - let sort = range_part.sort_expr.clone().unwrap(); + let sort = range_part.sort_expr.clone().expect("sort_expr missing"); let exprs = try_parse_physical_sort_expr(&input, &sort).unwrap_or_else(|e| { panic!("Failed to parse physical sort expressions: {}", e); }); @@ -1187,7 +1226,10 @@ pub fn parse_protobuf_partitioning( let bound_rows = sort_row_converter.lock().convert_columns(&bound_cols)?; Ok(Some(Partitioning::RangePartitioning( exprs, - range_part.partition_count.try_into().unwrap(), + range_part + .partition_count + .try_into() + .expect("range partition: invalid partition_count"), Arc::new(bound_rows), ))) } @@ -1250,12 +1292,12 @@ impl From<&protobuf::ColumnStats> for ColumnStatistics { max_value: cs .max_value .as_ref() - .map(|m| Precision::Exact(m.try_into().unwrap())) + .map(|m| Precision::Exact(m.try_into().expect("invalid max_value"))) .unwrap_or(Precision::Absent), min_value: cs .min_value .as_ref() - .map(|m| Precision::Exact(m.try_into().unwrap())) + .map(|m| Precision::Exact(m.try_into().expect("invalid min_value"))) .unwrap_or(Precision::Absent), sum_value: Precision::Absent, distinct_count: Precision::Exact(cs.distinct_count as usize), diff --git a/native-engine/auron/Cargo.toml b/native-engine/auron/Cargo.toml index 9e3f1353c..a8d338210 100644 --- a/native-engine/auron/Cargo.toml +++ b/native-engine/auron/Cargo.toml @@ -47,6 +47,7 @@ panic-message = { workspace = true } prost = { workspace = true } tokio = { workspace = true } chrono = { workspace = true } +parking_lot = { workspace = true } [dependencies.tikv-jemalloc-ctl] version = "0.6.1" diff --git a/native-engine/auron/src/alloc.rs b/native-engine/auron/src/alloc.rs index a65e7431e..9d9767212 100644 --- a/native-engine/auron/src/alloc.rs +++ b/native-engine/auron/src/alloc.rs @@ -20,12 +20,11 @@ use std::{ alloc::{GlobalAlloc, Layout}, - sync::{ - Mutex, - atomic::{AtomicUsize, Ordering::SeqCst}, - }, + sync::atomic::{AtomicUsize, Ordering::SeqCst}, }; +use parking_lot::Mutex; + #[cfg(any(feature = "jemalloc", feature = "jemalloc-pprof"))] #[cfg(not(windows))] #[cfg_attr(not(windows), global_allocator)] @@ -57,7 +56,7 @@ impl DebugAlloc { } fn update(&self) { - let _lock = self.mutex.lock().unwrap(); + let _lock = self.mutex.lock(); let current = self.current.load(SeqCst); let last_updated = self.last_updated.load(SeqCst); let delta = (current as isize - last_updated as isize).abs(); diff --git a/native-engine/auron/src/exec.rs b/native-engine/auron/src/exec.rs index de8d372a3..af0db9cf0 100644 --- a/native-engine/auron/src/exec.rs +++ b/native-engine/auron/src/exec.rs @@ -62,7 +62,10 @@ pub extern "system" fn Java_org_apache_auron_jni_JniBridge_callNative( INIT.get_or_try_init(|| { // logging is not initialized at this moment eprintln!("------ initializing auron native environment ------"); - let log_level = env.get_string(log_level).map(|s| String::from(s)).unwrap(); + let log_level = env + .get_string(log_level) + .map(|s| String::from(s)) + .expect("init: failed to read log_level from env"); eprintln!("initializing logging with level: {}", log_level); init_logging(log_level.as_str()); @@ -103,7 +106,10 @@ pub extern "system" fn Java_org_apache_auron_jni_JniBridge_callNative( // create execution runtime let runtime = Box::new(NativeExecutionRuntime::start( native_wrapper, - SESSION.get().unwrap().task_ctx(), + SESSION + .get() + .expect("session must be initialized") + .task_ctx(), )?); // returns runtime raw pointer diff --git a/native-engine/auron/src/http/mod.rs b/native-engine/auron/src/http/mod.rs index 86ef14322..1c51d02b1 100644 --- a/native-engine/auron/src/http/mod.rs +++ b/native-engine/auron/src/http/mod.rs @@ -18,9 +18,8 @@ mod memory_profiling; #[cfg(feature = "jemalloc-pprof")] mod pprof; -use std::sync::Mutex; - use once_cell::sync::OnceCell; +use parking_lot::Mutex; use poem::{Route, RouteMethod, Server, listener::TcpListener}; pub static HTTP_SERVICE: OnceCell = OnceCell::new(); @@ -50,7 +49,7 @@ impl DefaultHTTPServer { .worker_threads(1) .enable_io() .build() - .unwrap(), + .expect("fast fail: error initializing tokio runtime"), handlers: Mutex::new(vec![]), } } @@ -66,7 +65,7 @@ impl HTTPServer for DefaultHTTPServer { fn start(&self) { if let Some(port) = find_available_port() { let mut app = Route::new(); - let handlers = self.handlers.lock().unwrap(); + let handlers = self.handlers.lock(); for handler in handlers.iter() { app = app.at(handler.get_route_path(), handler.get_route_method()); } @@ -83,7 +82,7 @@ impl HTTPServer for DefaultHTTPServer { } fn register_handler(&self, handler: Box) { - let mut handlers = self.handlers.lock().unwrap(); + let mut handlers = self.handlers.lock(); handlers.push(handler); } } diff --git a/native-engine/auron/src/http/pprof.rs b/native-engine/auron/src/http/pprof.rs index 81cd9c11d..a7a0b3b0a 100644 --- a/native-engine/auron/src/http/pprof.rs +++ b/native-engine/auron/src/http/pprof.rs @@ -34,7 +34,7 @@ impl Default for PProfRequest { fn default() -> Self { PProfRequest { seconds: 5, - frequency: NonZeroI32::new(100).unwrap(), + frequency: NonZeroI32::new(100).expect("non-zero frequency"), } } } diff --git a/native-engine/auron/src/lib.rs b/native-engine/auron/src/lib.rs index 57492593e..2dfb336e0 100644 --- a/native-engine/auron/src/lib.rs +++ b/native-engine/auron/src/lib.rs @@ -55,7 +55,7 @@ fn handle_unwinded(err: Box) { } fn handle_unwinded_scope(scope: impl FnOnce() -> Result) -> T { - match std::panic::catch_unwind(AssertUnwindSafe(|| scope().unwrap())) { + match std::panic::catch_unwind(AssertUnwindSafe(|| scope().expect("scope failed"))) { Ok(v) => v, Err(err) => { handle_unwinded(err); diff --git a/native-engine/datafusion-ext-commons/src/arrow/cast.rs b/native-engine/datafusion-ext-commons/src/arrow/cast.rs index afa759fe0..8d1fd75bb 100644 --- a/native-engine/datafusion-ext-commons/src/arrow/cast.rs +++ b/native-engine/datafusion-ext-commons/src/arrow/cast.rs @@ -166,7 +166,11 @@ pub fn cast_impl( col = struct_.column_by_name(adjust.as_str()); } if col.is_some() { - cast_impl(col.unwrap(), field.data_type(), match_struct_fields) + cast_impl( + col.expect("column missing"), + field.data_type(), + match_struct_fields, + ) } else { null_column_name.push(field.name().clone()); Ok(new_null_array(field.data_type(), struct_.len())) @@ -227,7 +231,10 @@ pub fn cast_impl( } fn to_plain_string_array(array: &dyn Array) -> ArrayRef { - let array = array.as_any().downcast_ref::().unwrap(); + let array = array + .as_any() + .downcast_ref::() + .expect("Expected a StringArray"); let mut converted_values: Vec> = Vec::with_capacity(array.len()); for v in array.iter() { match v { @@ -252,10 +259,13 @@ fn try_cast_string_array_to_integer(array: &dyn Array, cast_type: &DataType) -> macro_rules! cast { ($target_type:ident) => {{ type B = paste::paste! {[<$target_type Builder>]}; - let array = array.as_any().downcast_ref::().unwrap(); + let string_array = array + .as_any() + .downcast_ref::() + .expect("Expected a StringArray"); let mut builder = B::new(); - for v in array.iter() { + for v in string_array.iter() { match v { Some(s) => builder.append_option(to_integer(s)), None => builder.append_null(), @@ -305,7 +315,7 @@ fn to_integer(input: &str) } let separator = b'.'; - let radix = T::from_usize(10).unwrap(); + let radix = T::from_usize(10).expect("from_usize(10) failed"); let stop_value = T::min_value() / radix; let mut result = T::zero(); @@ -333,7 +343,7 @@ fn to_integer(input: &str) return None; } - result = result * radix - T::from_u8(digit).unwrap(); + result = result * radix - T::from_u8(digit).expect("digit must be in 0..=9"); // Since the previous result is less than or equal to stopValue(Long.MIN_VALUE / // radix), we can just use `result > 0` to check overflow. If result // overflows, we should stop. @@ -425,23 +435,27 @@ fn to_date(s: &str) -> Option { #[cfg(test)] mod test { - use datafusion::common::cast::{as_decimal128_array, as_float64_array, as_int32_array}; + use datafusion::common::{ + Result, + cast::{as_decimal128_array, as_float64_array, as_int32_array}, + }; use super::*; #[test] - fn test_boolean_to_string() { + fn test_boolean_to_string() -> Result<()> { let bool_array: ArrayRef = Arc::new(BooleanArray::from_iter(vec![None, Some(true), Some(false)])); - let casted = cast(&bool_array, &DataType::Utf8).unwrap(); + let casted = cast(&bool_array, &DataType::Utf8)?; assert_eq!( as_string_array(&casted), &StringArray::from_iter(vec![None, Some("true"), Some("false")]) ); + Ok(()) } #[test] - fn test_float_to_int() { + fn test_float_to_int() -> Result<()> { let f64_array: ArrayRef = Arc::new(Float64Array::from_iter(vec![ None, Some(123.456), @@ -452,9 +466,9 @@ mod test { Some(f64::NEG_INFINITY), Some(f64::NAN), ])); - let casted = cast(&f64_array, &DataType::Int32).unwrap(); + let casted = cast(&f64_array, &DataType::Int32)?; assert_eq!( - as_int32_array(&casted).unwrap(), + as_int32_array(&casted)?, &Int32Array::from_iter(vec![ None, Some(123), @@ -466,10 +480,11 @@ mod test { Some(0), ]) ); + Ok(()) } #[test] - fn test_int_to_float() { + fn test_int_to_float() -> Result<()> { let i32_array: ArrayRef = Arc::new(Int32Array::from_iter(vec![ None, Some(123), @@ -477,9 +492,9 @@ mod test { Some(i32::MAX), Some(i32::MIN), ])); - let casted = cast(&i32_array, &DataType::Float64).unwrap(); + let casted = cast(&i32_array, &DataType::Float64)?; assert_eq!( - as_float64_array(&casted).unwrap(), + as_float64_array(&casted)?, &Float64Array::from_iter(vec![ None, Some(123.0), @@ -488,10 +503,11 @@ mod test { Some(i32::MIN as f64), ]) ); + Ok(()) } #[test] - fn test_int_to_decimal() { + fn test_int_to_decimal() -> Result<()> { let i32_array: ArrayRef = Arc::new(Int32Array::from_iter(vec![ None, Some(123), @@ -499,9 +515,9 @@ mod test { Some(i32::MAX), Some(i32::MIN), ])); - let casted = cast(&i32_array, &DataType::Decimal128(38, 18)).unwrap(); + let casted = cast(&i32_array, &DataType::Decimal128(38, 18))?; assert_eq!( - as_decimal128_array(&casted).unwrap(), + as_decimal128_array(&casted)?, &Decimal128Array::from_iter(vec![ None, Some(123000000000000000000), @@ -509,13 +525,13 @@ mod test { Some(i32::MAX as i128 * 1000000000000000000), Some(i32::MIN as i128 * 1000000000000000000), ]) - .with_precision_and_scale(38, 18) - .unwrap() + .with_precision_and_scale(38, 18)? ); + Ok(()) } #[test] - fn test_string_to_decimal() { + fn test_string_to_decimal() -> Result<()> { let string_array: ArrayRef = Arc::new(StringArray::from_iter(vec![ None, Some("1e-8"), @@ -527,9 +543,9 @@ mod test { Some("123456789012345.678901234567890"), Some("-123456789012345.678901234567890"), ])); - let casted = cast(&string_array, &DataType::Decimal128(38, 18)).unwrap(); + let casted = cast(&string_array, &DataType::Decimal128(38, 18))?; assert_eq!( - as_decimal128_array(&casted).unwrap(), + as_decimal128_array(&casted)?, &Decimal128Array::from_iter(vec![ None, Some(10000000000), @@ -541,13 +557,13 @@ mod test { Some(123456789012345678901234567890000i128), Some(-123456789012345678901234567890000i128), ]) - .with_precision_and_scale(38, 18) - .unwrap() + .with_precision_and_scale(38, 18)? ); + Ok(()) } #[test] - fn test_decimal_to_string() { + fn test_decimal_to_string() -> Result<()> { let decimal_array: ArrayRef = Arc::new( Decimal128Array::from_iter(vec![ None, @@ -557,12 +573,14 @@ mod test { Some(i32::MAX as i128 * 1000000000000000000), Some(i32::MIN as i128 * 1000000000000000000), ]) - .with_precision_and_scale(38, 18) - .unwrap(), + .with_precision_and_scale(38, 18)?, ); - let casted = cast(&decimal_array, &DataType::Utf8).unwrap(); + let casted = cast(&decimal_array, &DataType::Utf8)?; assert_eq!( - casted.as_any().downcast_ref::().unwrap(), + casted + .as_any() + .downcast_ref::() + .expect("Expected a StringArray"), &StringArray::from_iter(vec![ None, Some("123.000000000000000000"), @@ -572,10 +590,11 @@ mod test { Some("-2147483648.000000000000000000"), ]) ); + Ok(()) } #[test] - fn test_string_to_bigint() { + fn test_string_to_bigint() -> Result<()> { let string_array: ArrayRef = Arc::new(StringArray::from_iter(vec![ None, Some("123"), @@ -585,9 +604,12 @@ mod test { Some("-123456789012345"), Some("999999999999999999999999999999999"), ])); - let casted = cast(&string_array, &DataType::Int64).unwrap(); + let casted = cast(&string_array, &DataType::Int64)?; assert_eq!( - casted.as_any().downcast_ref::().unwrap(), + casted + .as_any() + .downcast_ref::() + .expect("Expected a Int64Array"), &Int64Array::from_iter(vec![ None, Some(123), @@ -598,10 +620,11 @@ mod test { None, ]) ); + Ok(()) } #[test] - fn test_string_to_date() { + fn test_string_to_date() -> Result<()> { let string_array: ArrayRef = Arc::new(StringArray::from_iter(vec![ None, Some("2001-02-03"), @@ -614,11 +637,9 @@ mod test { Some("9999-99"), Some("99999-01"), ])); - let casted = cast(&string_array, &DataType::Date32).unwrap(); + let casted = cast(&string_array, &DataType::Date32)?; assert_eq!( - arrow::compute::cast(&casted, &DataType::Utf8) - .unwrap() - .as_string(), + arrow::compute::cast(&casted, &DataType::Utf8)?.as_string(), &StringArray::from_iter(vec![ None, Some("2001-02-03"), @@ -632,5 +653,6 @@ mod test { None, ]) ); + Ok(()) } } diff --git a/native-engine/datafusion-ext-commons/src/arrow/eq_comparator.rs b/native-engine/datafusion-ext-commons/src/arrow/eq_comparator.rs index 51c05b7f5..1cccb9e7c 100644 --- a/native-engine/datafusion-ext-commons/src/arrow/eq_comparator.rs +++ b/native-engine/datafusion-ext-commons/src/arrow/eq_comparator.rs @@ -275,8 +275,14 @@ fn eq_fixed_list( ignores_null, )?; - let l_size = left.value_length().to_usize().unwrap(); - let r_size = right.value_length().to_usize().unwrap(); + let l_size = left + .value_length() + .to_usize() + .expect("left length to_usize failed"); + let r_size = right + .value_length() + .to_usize() + .expect("right length to_usize failed"); let size_eq = l_size == r_size; let f = eq_impl(left, right, ignores_null, move |i, j| { @@ -390,78 +396,85 @@ pub mod tests { use super::*; #[test] - fn test_fixed_size_binary() { + fn test_fixed_size_binary() -> Result<()> { let items = vec![vec![1u8], vec![2u8]]; - let array = FixedSizeBinaryArray::try_from_iter(items.into_iter()).unwrap(); + let array = FixedSizeBinaryArray::try_from_iter(items.into_iter())?; - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(0, 1)); + Ok(()) } #[test] - fn test_fixed_size_binary_fixed_size_binary() { + fn test_fixed_size_binary_fixed_size_binary() -> Result<()> { let items = vec![vec![1u8]]; - let array1 = FixedSizeBinaryArray::try_from_iter(items.into_iter()).unwrap(); + let array1 = FixedSizeBinaryArray::try_from_iter(items.into_iter())?; let items = vec![vec![2u8]]; - let array2 = FixedSizeBinaryArray::try_from_iter(items.into_iter()).unwrap(); + let array2 = FixedSizeBinaryArray::try_from_iter(items.into_iter())?; - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); + Ok(()) } #[test] - fn test_i32() { + fn test_i32() -> Result<()> { let array = Int32Array::from(vec![1, 2]); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, (eq)(0, 1)); + Ok(()) } #[test] - fn test_i32_i32() { + fn test_i32_i32() -> Result<()> { let array1 = Int32Array::from(vec![1]); let array2 = Int32Array::from(vec![2]); - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); + Ok(()) } #[test] - fn test_f64() { + fn test_f64() -> Result<()> { let array = Float64Array::from(vec![1.0, 2.0]); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(0, 1)); + Ok(()) } #[test] - fn test_f64_nan() { + fn test_f64_nan() -> Result<()> { let array = Float64Array::from(vec![1.0, f64::NAN]); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(true, eq(0, 0)); assert_eq!(false, eq(0, 1)); assert_eq!(false, eq(1, 1)); // NaN != NaN + Ok(()) } #[test] - fn test_f64_zeros() { + fn test_f64_zeros() -> Result<()> { let array = Float64Array::from(vec![-0.0, 0.0]); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(true, eq(0, 1)); // -0.0 == 0.0 assert_eq!(true, eq(1, 0)); + Ok(()) } #[test] - fn test_interval_day_time() { + fn test_interval_day_time() -> Result<()> { let array = IntervalDayTimeArray::from(vec![ // 0 days, 1 second IntervalDayTimeType::make_value(0, 1000), @@ -471,7 +484,7 @@ pub mod tests { IntervalDayTimeType::make_value(0, 90_000_000), ]); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(0, 1)); assert_eq!(false, eq(1, 0)); @@ -481,10 +494,11 @@ pub mod tests { // values not field by field assert_eq!(false, eq(1, 2)); assert_eq!(false, eq(2, 1)); + Ok(()) } #[test] - fn test_interval_year_month() { + fn test_interval_year_month() -> Result<()> { let array = IntervalYearMonthArray::from(vec![ // 1 year, 0 months IntervalYearMonthType::make_value(1, 0), @@ -494,7 +508,7 @@ pub mod tests { IntervalYearMonthType::make_value(1, 1), ]); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(0, 1)); assert_eq!(false, eq(1, 0)); @@ -502,10 +516,11 @@ pub mod tests { // the underlying representation is months, so both quantities are the same assert_eq!(true, eq(1, 2)); assert_eq!(true, eq(2, 1)); + Ok(()) } #[test] - fn test_interval_month_day_nano() { + fn test_interval_month_day_nano() -> Result<()> { let array = IntervalMonthDayNanoArray::from(vec![ // 100 days IntervalMonthDayNanoType::make_value(0, 100, 0), @@ -515,7 +530,7 @@ pub mod tests { IntervalMonthDayNanoType::make_value(0, 100, 2), ]); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(0, 1)); assert_eq!(false, eq(1, 0)); @@ -525,23 +540,24 @@ pub mod tests { // values not field by field assert_eq!(false, eq(1, 2)); assert_eq!(false, eq(2, 1)); + Ok(()) } #[test] - fn test_decimal() { + fn test_decimal() -> Result<()> { let array = vec![Some(5_i128), Some(2_i128), Some(3_i128)] .into_iter() .collect::() - .with_precision_and_scale(23, 6) - .unwrap(); + .with_precision_and_scale(23, 6)?; - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(1, 0)); assert_eq!(false, eq(0, 2)); + Ok(()) } #[test] - fn test_decimali256() { + fn test_decimali256() -> Result<()> { let array = vec![ Some(i256::from_i128(5_i128)), Some(i256::from_i128(2_i128)), @@ -549,42 +565,44 @@ pub mod tests { ] .into_iter() .collect::() - .with_precision_and_scale(53, 6) - .unwrap(); + .with_precision_and_scale(53, 6)?; - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(1, 0)); assert_eq!(false, eq(0, 2)); + Ok(()) } #[test] - fn test_dict() { + fn test_dict() -> Result<()> { let data = vec!["a", "b", "c", "a", "a", "c", "c"]; let array = data.into_iter().collect::>(); - let eq = make_eq_comparator(&array, &array, false).unwrap(); + let eq = make_eq_comparator(&array, &array, false)?; assert_eq!(false, eq(0, 1)); assert_eq!(true, eq(3, 4)); assert_eq!(false, eq(2, 3)); + Ok(()) } #[test] - fn test_multiple_dict() { + fn test_multiple_dict() -> Result<()> { let d1 = vec!["a", "b", "c", "d"]; let a1 = d1.into_iter().collect::>(); let d2 = vec!["e", "f", "g", "a"]; let a2 = d2.into_iter().collect::>(); - let eq = make_eq_comparator(&a1, &a2, false).unwrap(); + let eq = make_eq_comparator(&a1, &a2, false)?; assert_eq!(false, eq(0, 0)); assert_eq!(true, eq(0, 3)); assert_eq!(false, eq(1, 3)); + Ok(()) } #[test] - fn test_primitive_dict() { + fn test_primitive_dict() -> Result<()> { let values = Int32Array::from(vec![1_i32, 0, 2, 5]); let keys = Int8Array::from_iter_values([0, 0, 1, 3]); let array1 = DictionaryArray::new(keys, Arc::new(values)); @@ -593,36 +611,38 @@ pub mod tests { let keys = Int8Array::from_iter_values([0, 1, 1, 3]); let array2 = DictionaryArray::new(keys, Arc::new(values)); - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); assert_eq!(false, eq(0, 3)); assert_eq!(true, eq(3, 3)); assert_eq!(false, eq(3, 1)); assert_eq!(false, eq(3, 2)); + Ok(()) } #[test] - fn test_float_dict() { + fn test_float_dict() -> Result<()> { let values = Float32Array::from(vec![1.0, 0.5, 2.1, 5.5]); let keys = Int8Array::from_iter_values([0, 0, 1, 3]); - let array1 = DictionaryArray::try_new(keys, Arc::new(values)).unwrap(); + let array1 = DictionaryArray::try_new(keys, Arc::new(values))?; let values = Float32Array::from(vec![1.2, 3.2, 4.0, 5.5]); let keys = Int8Array::from_iter_values([0, 1, 1, 3]); let array2 = DictionaryArray::new(keys, Arc::new(values)); - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); assert_eq!(false, eq(0, 3)); assert_eq!(true, eq(3, 3)); assert_eq!(false, eq(3, 1)); assert_eq!(false, eq(3, 2)); + Ok(()) } #[test] - fn test_timestamp_dict() { + fn test_timestamp_dict() -> Result<()> { let values = TimestampSecondArray::from(vec![1, 0, 2, 5]); let keys = Int8Array::from_iter_values([0, 0, 1, 3]); let array1 = DictionaryArray::new(keys, Arc::new(values)); @@ -631,17 +651,18 @@ pub mod tests { let keys = Int8Array::from_iter_values([0, 1, 1, 3]); let array2 = DictionaryArray::new(keys, Arc::new(values)); - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); assert_eq!(false, eq(0, 3)); assert_eq!(true, eq(3, 3)); assert_eq!(false, eq(3, 1)); assert_eq!(false, eq(3, 2)); + Ok(()) } #[test] - fn test_duration_dict() { + fn test_duration_dict() -> Result<()> { let values = DurationSecondArray::from(vec![1, 0, 2, 5]); let keys = Int8Array::from_iter_values([0, 0, 1, 3]); let array1 = DictionaryArray::new(keys, Arc::new(values)); @@ -650,17 +671,18 @@ pub mod tests { let keys = Int8Array::from_iter_values([0, 1, 1, 3]); let array2 = DictionaryArray::new(keys, Arc::new(values)); - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); assert_eq!(false, eq(0, 3)); assert_eq!(true, eq(3, 3)); assert_eq!(false, eq(3, 1)); assert_eq!(false, eq(3, 2)); + Ok(()) } #[test] - fn test_decimal_dict() { + fn test_decimal_dict() -> Result<()> { let values = Decimal128Array::from(vec![1, 0, 2, 5]); let keys = Int8Array::from_iter_values([0, 0, 1, 3]); let array1 = DictionaryArray::new(keys, Arc::new(values)); @@ -669,17 +691,18 @@ pub mod tests { let keys = Int8Array::from_iter_values([0, 1, 1, 3]); let array2 = DictionaryArray::new(keys, Arc::new(values)); - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); assert_eq!(false, eq(0, 3)); assert_eq!(true, eq(3, 3)); assert_eq!(false, eq(3, 1)); assert_eq!(false, eq(3, 2)); + Ok(()) } #[test] - fn test_decimal256_dict() { + fn test_decimal256_dict() -> Result<()> { let values = Decimal256Array::from(vec![ i256::from_i128(1), i256::from_i128(0), @@ -698,23 +721,25 @@ pub mod tests { let keys = Int8Array::from_iter_values([0, 1, 1, 3]); let array2 = DictionaryArray::new(keys, Arc::new(values)); - let eq = make_eq_comparator(&array1, &array2, false).unwrap(); + let eq = make_eq_comparator(&array1, &array2, false)?; assert_eq!(false, eq(0, 0)); assert_eq!(false, eq(0, 3)); assert_eq!(true, eq(3, 3)); assert_eq!(false, eq(3, 1)); assert_eq!(false, eq(3, 2)); + Ok(()) } - fn test_bytes_impl() { + fn test_bytes_impl() -> Result<()> { let offsets = OffsetBuffer::from_lengths([3, 3, 1]); let a = GenericByteArray::::new(offsets, b"abcdefa".into(), None); - let eq = make_eq_comparator(&a, &a, false).unwrap(); + let eq = make_eq_comparator(&a, &a, false)?; assert_eq!(false, eq(0, 1)); assert_eq!(false, eq(0, 2)); assert_eq!(true, eq(1, 1)); + Ok(()) } #[test] @@ -726,7 +751,7 @@ pub mod tests { } #[test] - fn test_lists() { + fn test_lists() -> Result<()> { let mut a = ListBuilder::new(ListBuilder::new(Int32Builder::new())); a.extend([ Some(vec![Some(vec![Some(1), Some(2), None]), Some(vec![None])]), @@ -755,7 +780,7 @@ pub mod tests { ]); let b = b.finish(); - let eq = make_eq_comparator(&a, &b, false).unwrap(); + let eq = make_eq_comparator(&a, &b, false)?; assert_eq!(eq(0, 0), false); // lists contains null never equal assert_eq!(eq(0, 1), false); assert_eq!(eq(0, 2), false); @@ -763,10 +788,11 @@ pub mod tests { assert_eq!(eq(1, 3), false); assert_eq!(eq(2, 0), false); assert_eq!(eq(4, 4), true); + Ok(()) } #[test] - fn test_struct() { + fn test_struct() -> Result<()> { let fields = Fields::from(vec![ Field::new("a", DataType::Int32, true), Field::new_list("b", Field::new("item", DataType::Int32, true), true), @@ -789,7 +815,7 @@ pub mod tests { let values = vec![Arc::new(a) as _, Arc::new(b) as _]; let s2 = StructArray::new(fields.clone(), values, None); - let eq = make_eq_comparator(&s1, &s2, false).unwrap(); + let eq = make_eq_comparator(&s1, &s2, false)?; assert_eq!(eq(0, 1), false); // (1, [1, 2]) eq (2, None) assert_eq!(eq(0, 0), false); // (1, [1, 2]) eq (None, None) assert_eq!(eq(1, 1), false); // (2, [None]) eq (2, None) @@ -797,5 +823,6 @@ pub mod tests { assert_eq!(eq(3, 0), false); // None eq (None, []) assert_eq!(eq(2, 0), false); // (None, None) eq (None, None) assert_eq!(eq(3, 0), false); // None eq (None, None) + Ok(()) } } diff --git a/native-engine/datafusion-ext-commons/src/arrow/selection.rs b/native-engine/datafusion-ext-commons/src/arrow/selection.rs index bf67bc28d..0669323f9 100644 --- a/native-engine/datafusion-ext-commons/src/arrow/selection.rs +++ b/native-engine/datafusion-ext-commons/src/arrow/selection.rs @@ -248,7 +248,11 @@ pub fn create_array_interleaver( let interleaver = Interleave::new( values .iter() - .map(|v| downcast_any!(v, PrimitiveArray<$t>).unwrap().clone()) + .map(|v| { + downcast_any!(v, PrimitiveArray<$t>) + .expect("Excepted a PrimitiveArray") + .clone() + }) .collect::>(), ); let dt = $dt.clone(); @@ -266,7 +270,7 @@ pub fn create_array_interleaver( DataType::Utf8 => { let interleaver = Interleave::new(values .iter() - .map(|v| downcast_any!(v, StringArray).unwrap().clone()) + .map(|v| downcast_any!(v, StringArray).expect("Excepted a StringArray").clone()) .collect::>(), ); return Ok(Box::new(move |indices| if with_prefetching { @@ -278,7 +282,7 @@ pub fn create_array_interleaver( DataType::Binary => { let interleaver = Interleave::new(values .iter() - .map(|v| downcast_any!(v, BinaryArray).unwrap().clone()) + .map(|v| downcast_any!(v, BinaryArray).expect("Excepted a BinaryArray").clone()) .collect::>(), ); return Ok(Box::new(move |indices| if with_prefetching { diff --git a/native-engine/datafusion-ext-commons/src/io/batch_serde.rs b/native-engine/datafusion-ext-commons/src/io/batch_serde.rs index bf740b86a..bbed9b365 100644 --- a/native-engine/datafusion-ext-commons/src/io/batch_serde.rs +++ b/native-engine/datafusion-ext-commons/src/io/batch_serde.rs @@ -48,7 +48,9 @@ impl TransposeOpt { DataType::Null => 0, DataType::Boolean => 0, dt if dt.primitive_width() == Some(1) => 0, - dt if dt.primitive_width() >= Some(2) => dt.primitive_width().unwrap(), + dt if dt.primitive_width() >= Some(2) => { + dt.primitive_width().expect("primitive_width must be Some") + } DataType::Utf8 | DataType::Binary => 4, DataType::List(f) | DataType::Map(f, _) => { Self::data_type_bytes_width(f.data_type()).max(4) @@ -390,7 +392,7 @@ fn read_list_array( }; let offsets = read_offsets(input, num_rows, transpose_opt)?; - let values_len = offsets.last().cloned().unwrap() as usize; + let values_len = offsets.last().cloned().expect("offsets must be non-empty") as usize; let offsets_buffer: Buffer = Buffer::from_vec(offsets); let values = read_array( input, @@ -431,8 +433,15 @@ fn write_map_array( let value_offsets = array.value_offsets(); write_offsets(output, value_offsets, transpose_opt)?; - let first_offset = value_offsets.first().cloned().unwrap() as usize; - let entries_len = value_offsets.last().cloned().unwrap() as usize - first_offset; + let first_offset = value_offsets + .first() + .cloned() + .expect("value_offsets must be non-empty") as usize; + let entries_len = value_offsets + .last() + .cloned() + .expect("value_offsets must be non-empty") as usize + - first_offset; let keys = array.keys().slice(first_offset, entries_len); let values = array.values().slice(first_offset, entries_len); @@ -458,7 +467,7 @@ fn read_map_array( }; let offsets = read_offsets(input, num_rows, transpose_opt)?; - let entries_len = offsets.last().cloned().unwrap() as usize; + let entries_len = offsets.last().cloned().expect("offsets must be non-empty") as usize; let offsets_buffer = Buffer::from_vec(offsets); // build inner struct @@ -612,8 +621,14 @@ fn write_bytes_array, W: Write>( let value_offsets = array.value_offsets(); write_offsets(output, value_offsets, transpose_opt)?; - let first_offset = value_offsets.first().cloned().unwrap() as usize; - let last_offset = value_offsets.last().cloned().unwrap() as usize; + let first_offset = value_offsets + .first() + .cloned() + .expect("value_offsets must be non-empty") as usize; + let last_offset = value_offsets + .last() + .cloned() + .expect("value_offsets must be non-empty") as usize; output.write_all(&array.value_data()[first_offset..last_offset])?; Ok(()) } @@ -632,7 +647,7 @@ fn read_bytes_array( }; let offsets = read_offsets(input, num_rows, transpose_opt)?; - let values_len = offsets.last().cloned().unwrap() as usize; + let values_len = offsets.last().cloned().expect("offsets must be non-empty") as usize; let offsets_buffer = Buffer::from_vec(offsets); let data_buffer = Buffer::from_vec(read_bytes_slice(input, values_len)?.into()); @@ -652,7 +667,7 @@ mod test { use std::{io::Cursor, sync::Arc}; use arrow::{array::*, datatypes::*, record_batch::RecordBatch}; - use datafusion::assert_batches_eq; + use datafusion::{assert_batches_eq, common::Result}; use crate::io::{ batch_serde::{read_batch, write_batch}, @@ -660,7 +675,7 @@ mod test { }; #[test] - fn test_write_and_read_batch() { + fn test_write_and_read_batch() -> Result<()> { let array1: ArrayRef = Arc::new(StringArray::from_iter([ Some("20220101".to_owned()), Some("20220102你好🍹".to_owned()), @@ -683,35 +698,35 @@ mod test { ("str", array1, true), ("u64", array2, true), ("bool", array3, true), - ]) - .unwrap(); + ])?; // test read after write let mut buf = vec![]; - write_batch(batch.num_rows(), batch.columns(), &mut buf).unwrap(); + write_batch(batch.num_rows(), batch.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_eq!( - recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema()).unwrap(), + recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema())?, batch ); // test read after write sliced let sliced = batch.slice(1, 2); let mut buf = vec![]; - write_batch(sliced.num_rows(), sliced.columns(), &mut buf).unwrap(); + write_batch(sliced.num_rows(), sliced.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_eq!( - recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema()).unwrap(), + recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema())?, sliced ); + Ok(()) } #[test] - fn test_write_and_read_batch_for_list() { + fn test_write_and_read_batch_for_list() -> Result<()> { let data = vec![ Some(vec![Some(0), Some(1), Some(2)]), None, @@ -723,8 +738,7 @@ mod test { let batch = RecordBatch::try_from_iter_with_nullable(vec![ ("list1", list_array.clone(), true), ("list2", list_array.clone(), true), - ]) - .unwrap(); + ])?; assert_batches_eq!( vec![ @@ -742,10 +756,10 @@ mod test { // test read after write let mut buf = vec![]; - write_batch(batch.num_rows(), batch.columns(), &mut buf).unwrap(); + write_batch(batch.num_rows(), batch.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_batches_eq!( vec![ "+-----------+-----------+", @@ -757,16 +771,20 @@ mod test { "| [6, 7] | [6, 7] |", "+-----------+-----------+", ], - &[recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema()).unwrap()] + &[recover_named_batch( + decoded_num_rows, + &decoded_cols, + batch.schema() + )?] ); // test read after write sliced let sliced = batch.slice(1, 2); let mut buf = vec![]; - write_batch(sliced.num_rows(), sliced.columns(), &mut buf).unwrap(); + write_batch(sliced.num_rows(), sliced.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_batches_eq!( vec![ "+----------+----------+", @@ -776,97 +794,102 @@ mod test { "| [3, , 5] | [3, , 5] |", "+----------+----------+", ], - &[recover_named_batch(decoded_num_rows, &decoded_cols, sliced.schema()).unwrap()] + &[recover_named_batch( + decoded_num_rows, + &decoded_cols, + sliced.schema() + )?] ); + Ok(()) } #[test] - fn test_write_and_read_batch_for_map() { - let map_array: ArrayRef = Arc::new( - MapArray::new_from_strings( - ["00", "11", "22", "33", "44", "55", "66", "77"].into_iter(), - &StringArray::from(vec![ - Some("aa"), - None, - Some("cc"), - Some("dd"), - Some("ee"), - Some("ff"), - Some("gg"), - Some("hh"), - ]), - &[0, 3, 6, 8], // [00,11,22], [33,44,55], [66,77] - ) - .unwrap(), - ); + fn test_write_and_read_batch_for_map() -> Result<()> { + let map_array: ArrayRef = Arc::new(MapArray::new_from_strings( + ["00", "11", "22", "33", "44", "55", "66", "77"].into_iter(), + &StringArray::from(vec![ + Some("aa"), + None, + Some("cc"), + Some("dd"), + Some("ee"), + Some("ff"), + Some("gg"), + Some("hh"), + ]), + &[0, 3, 6, 8], // [00,11,22], [33,44,55], [66,77] + )?); let batch = RecordBatch::try_from_iter_with_nullable(vec![ ("map1", map_array.clone(), true), ("map2", map_array.clone(), true), - ]) - .unwrap(); + ])?; // test read after write let mut buf = vec![]; - write_batch(batch.num_rows(), batch.columns(), &mut buf).unwrap(); + write_batch(batch.num_rows(), batch.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_eq!( - recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema()).unwrap(), + recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema())?, batch ); // test read after write sliced let sliced = batch.slice(1, 2); let mut buf = vec![]; - write_batch(sliced.num_rows(), sliced.columns(), &mut buf).unwrap(); + write_batch(sliced.num_rows(), sliced.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_eq!( - recover_named_batch(decoded_num_rows, &decoded_cols, sliced.schema()).unwrap(), + recover_named_batch(decoded_num_rows, &decoded_cols, sliced.schema())?, sliced ); + Ok(()) } #[test] - fn test_write_and_read_batch_for_struct() { + fn test_write_and_read_batch_for_struct() -> Result<()> { let c1: ArrayRef = Arc::new(BooleanArray::from(vec![false, false, true, true])); let c2: ArrayRef = Arc::new(Int32Array::from(vec![42, 28, 19, 31])); let c3: ArrayRef = Arc::new(BooleanArray::from(vec![None, None, None, Some(true)])); let c4: ArrayRef = Arc::new(Int32Array::from(vec![None, None, None, Some(31)])); - let struct_array: ArrayRef = Arc::new( - StructArray::try_from(vec![("c1", c1), ("c2", c2), ("c3", c3), ("c4", c4)]).unwrap(), - ); + let struct_array: ArrayRef = Arc::new(StructArray::try_from(vec![ + ("c1", c1), + ("c2", c2), + ("c3", c3), + ("c4", c4), + ])?); let batch = RecordBatch::try_from_iter_with_nullable(vec![ ("struct1", struct_array.clone(), true), ("struct2", struct_array.clone(), true), - ]) - .unwrap(); + ])?; // test read after write let mut buf = vec![]; - write_batch(batch.num_rows(), batch.columns(), &mut buf).unwrap(); + write_batch(batch.num_rows(), batch.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_eq!( - recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema()).unwrap(), + recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema())?, batch ); // test read after write sliced let sliced = batch.slice(1, 2); let mut buf = vec![]; - write_batch(sliced.num_rows(), sliced.columns(), &mut buf).unwrap(); + write_batch(sliced.num_rows(), sliced.columns(), &mut buf)?; let mut cursor = Cursor::new(buf); let (decoded_num_rows, decoded_cols) = - read_batch(&mut cursor, &batch.schema()).unwrap().unwrap(); + read_batch(&mut cursor, &batch.schema())?.expect("non-empty batch"); assert_eq!( - recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema()).unwrap(), + recover_named_batch(decoded_num_rows, &decoded_cols, batch.schema())?, sliced ); + Ok(()) } } diff --git a/native-engine/datafusion-ext-commons/src/io/ipc_compression.rs b/native-engine/datafusion-ext-commons/src/io/ipc_compression.rs index d3ea0b293..451a4cd8f 100644 --- a/native-engine/datafusion-ext-commons/src/io/ipc_compression.rs +++ b/native-engine/datafusion-ext-commons/src/io/ipc_compression.rs @@ -341,10 +341,10 @@ mod tests { writer.finish_current_buf()?; let mut reader = IpcCompressionReader::new(Cursor::new(buf)); - let (num_rows1, arrays1) = reader.read_batch(&schema)?.unwrap(); + let (num_rows1, arrays1) = reader.read_batch(&schema)?.expect("non-empty batch"); assert_eq!(num_rows1, 2); assert_eq!(arrays1, &[test_array1]); - let (num_rows2, arrays2) = reader.read_batch(&schema)?.unwrap(); + let (num_rows2, arrays2) = reader.read_batch(&schema)?.expect("non-empty batch"); assert_eq!(num_rows2, 2); assert_eq!(arrays2, &[test_array2]); assert!(reader.read_batch(&schema)?.is_none()); diff --git a/native-engine/datafusion-ext-commons/src/io/scalar_serde.rs b/native-engine/datafusion-ext-commons/src/io/scalar_serde.rs index bbbd2a1e1..90d6ec98b 100644 --- a/native-engine/datafusion-ext-commons/src/io/scalar_serde.rs +++ b/native-engine/datafusion-ext-commons/src/io/scalar_serde.rs @@ -42,7 +42,7 @@ pub fn write_scalar(value: &ScalarValue, nullable: bool, output: &mut write_u8(0, output)?; } } else { - output.write_all(&$v.unwrap().to_ne_bytes())?; + output.write_all(&$v.expect("non-null value required").to_ne_bytes())?; } }}; } diff --git a/native-engine/datafusion-ext-commons/src/spark_hash.rs b/native-engine/datafusion-ext-commons/src/spark_hash.rs index 323f2ac86..66be98486 100644 --- a/native-engine/datafusion-ext-commons/src/spark_hash.rs +++ b/native-engine/datafusion-ext-commons/src/spark_hash.rs @@ -66,7 +66,10 @@ fn hash_array( macro_rules! hash_array { ($array_type:ident, $column:ident, $hashes:ident, $h:expr) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let array = $column + .as_any() + .downcast_ref::<$array_type>() + .expect("downcast to expected array type failed"); if array.null_count() == 0 { for (i, hash) in $hashes.iter_mut().enumerate() { *hash = $h(&array.value(i).as_ref(), *hash); @@ -83,7 +86,10 @@ fn hash_array( macro_rules! hash_array_primitive { ($array_type:ident, $column:ident, $ty:ident, $hashes:ident, $h:expr) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let array = $column + .as_any() + .downcast_ref::<$array_type>() + .expect("downcast to expected array type failed"); let values = array.values(); if array.null_count() == 0 { @@ -102,7 +108,10 @@ fn hash_array( macro_rules! hash_array_decimal { ($array_type:ident, $column:ident, $hashes:ident, $h:expr) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let array = $column + .as_any() + .downcast_ref::<$array_type>() + .expect("downcast to expected array type failed"); if array.null_count() == 0 { for (i, hash) in $hashes.iter_mut().enumerate() { @@ -121,7 +130,10 @@ fn hash_array( match array.data_type() { DataType::Null => {} DataType::Boolean => { - let array = array.as_any().downcast_ref::().unwrap(); + let array = array + .as_any() + .downcast_ref::() + .expect("Expected a BooleanArray"); if array.null_count() == 0 { for (i, hash) in hashes_buffer.iter_mut().enumerate() { *hash = h( @@ -217,7 +229,10 @@ fn create_hashes_dictionary( hashes_buffer: &mut [T], h: impl Fn(&[u8], T) -> T + Copy, ) { - let dict_array = array.as_any().downcast_ref::>().unwrap(); + let dict_array = array + .as_any() + .downcast_ref::>() + .expect("Expected a DictionaryArray"); // Hash each dictionary value once, and then use that computed // hash for each key value to avoid a potentially expensive @@ -238,7 +253,10 @@ fn hash_one( ) { macro_rules! hash_one_primitive { ($array_type:ident, $column:ident, $ty:ident, $hash:ident, $idx:ident, $h:expr) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let array = $column + .as_any() + .downcast_ref::<$array_type>() + .expect("downcast to expected array type failed"); *$hash = $h( (array.value($idx as usize) as $ty).to_le_bytes().as_ref(), *$hash, @@ -248,14 +266,20 @@ fn hash_one( macro_rules! hash_one_binary { ($array_type:ident, $column:ident, $hash:ident, $idx:ident, $h:expr) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let array = $column + .as_any() + .downcast_ref::<$array_type>() + .expect("downcast to expected array type failed"); *$hash = $h(&array.value($idx as usize).as_ref(), *$hash); }; } macro_rules! hash_one_decimal { ($array_type:ident, $column:ident, $hash:ident, $idx:ident, $h:expr) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let array = $column + .as_any() + .downcast_ref::<$array_type>() + .expect("downcast to expected array type failed"); *$hash = $h(array.value($idx as usize).to_le_bytes().as_ref(), *$hash); }; } @@ -264,7 +288,10 @@ fn hash_one( match col.data_type() { DataType::Null => {} DataType::Boolean => { - let array = col.as_any().downcast_ref::().unwrap(); + let array = col + .as_any() + .downcast_ref::() + .expect("Expected a BooleanArray"); *hash = h( (if array.value(idx) { 1u32 } else { 0u32 }) .to_le_bytes() @@ -324,14 +351,20 @@ fn hash_one( hash_one_decimal!(Decimal128Array, col, hash, idx, h); } DataType::List(..) => { - let list_array = col.as_any().downcast_ref::().unwrap(); + let list_array = col + .as_any() + .downcast_ref::() + .expect("Expected a ListArray"); let value_array = list_array.value(idx); for i in 0..value_array.len() { hash_one(&value_array, i, hash, h); } } DataType::Map(..) => { - let map_array = col.as_any().downcast_ref::().unwrap(); + let map_array = col + .as_any() + .downcast_ref::() + .expect("Expected a MapArray"); let kv_array = map_array.value(idx); let key_array = kv_array.column(0); let value_array = kv_array.column(1); @@ -341,7 +374,10 @@ fn hash_one( } } DataType::Struct(_) => { - let struct_array = col.as_any().downcast_ref::().unwrap(); + let struct_array = col + .as_any() + .downcast_ref::() + .expect("Expected a StructArray"); for col in struct_array.columns() { hash_one(col, idx, hash, h); } @@ -353,7 +389,7 @@ fn hash_one( #[cfg(test)] mod tests { - use std::sync::Arc; + use std::{error::Error, sync::Arc}; use arrow::{ array::{ @@ -482,15 +518,14 @@ mod tests { } #[test] - fn test_list_array() { + fn test_list_array() -> Result<(), Box> { // Create inner array data: [1, 2, 3, 4, 5, 6] let value_data = ArrayData::builder(DataType::Int32) .len(6) .add_buffer(Buffer::from_slice_ref( &[1i32, 2, 3, 4, 5, 6].to_byte_slice(), )) - .build() - .unwrap(); + .build()?; // Create offset array to define list boundaries: [[1, 2], [3, 4, 5], [6]] let list_data_type = DataType::new_list(DataType::Int32, false); @@ -498,8 +533,7 @@ mod tests { .len(3) .add_buffer(Buffer::from_slice_ref(&[0i32, 2, 5, 6].to_byte_slice())) .add_child_data(value_data) - .build() - .unwrap(); + .build()?; let list_array = ListArray::from(list_data); let array_ref = Arc::new(list_array) as ArrayRef; @@ -507,26 +541,25 @@ mod tests { // Test Murmur3 hash let hashes = create_murmur3_hashes(3, &[array_ref.clone()], 42); assert_eq!(hashes, vec![-222940379, -374492525, -331964951]); + Ok(()) } #[test] - fn test_map_array() { + fn test_map_array() -> Result<(), Box> { // Construct key and values let key_data = ArrayData::builder(DataType::Int32) .len(8) .add_buffer(Buffer::from_slice_ref( &[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice(), )) - .build() - .unwrap(); + .build()?; let value_data = ArrayData::builder(DataType::UInt32) .len(8) .add_buffer(Buffer::from_slice_ref( &[0u32, 10, 20, 0, 40, 0, 60, 70].to_byte_slice(), )) .null_bit_buffer(Some(Buffer::from(&[0b11010110]))) - .build() - .unwrap(); + .build()?; // Construct a buffer for value offsets, for the nested array: // [[0, 1, 2], [3, 4, 5], [6, 7]] @@ -552,8 +585,7 @@ mod tests { .len(3) .add_buffer(entry_offsets) .add_child_data(entry_struct.into_data()) - .build() - .unwrap(); + .build()?; let map_array = MapArray::from(map_data); assert_eq!(&value_data, &map_array.values().to_data()); @@ -579,7 +611,7 @@ mod tests { unsafe { map_array.value_unchecked(0) } .as_any() .downcast_ref::() - .unwrap() + .expect("Expected a StructArray") ); for i in 0..3 { assert!(map_array.is_valid(i)); @@ -592,8 +624,7 @@ mod tests { .offset(1) .add_buffer(map_array.to_data().buffers()[0].clone()) .add_child_data(map_array.to_data().child_data()[0].clone()) - .build() - .unwrap(); + .build()?; let map_array = MapArray::from(map_data); assert_eq!(&value_data, &map_array.values().to_data()); @@ -613,14 +644,15 @@ mod tests { .value(0) .as_any() .downcast_ref::() - .unwrap() + .expect("Expected a StructArray") ); assert_eq!( &struct_array, unsafe { map_array.value_unchecked(0) } .as_any() .downcast_ref::() - .unwrap() + .expect("Expected a StructArray") ); + Ok(()) } } diff --git a/native-engine/datafusion-ext-exprs/src/bloom_filter_might_contain.rs b/native-engine/datafusion-ext-exprs/src/bloom_filter_might_contain.rs index 86af57376..663de05a6 100644 --- a/native-engine/datafusion-ext-exprs/src/bloom_filter_might_contain.rs +++ b/native-engine/datafusion-ext-exprs/src/bloom_filter_might_contain.rs @@ -137,7 +137,10 @@ impl PhysicalExpr for BloomFilterMightContainExpr { if bloom_filter.is_none() { return Ok(ColumnarValue::Scalar(ScalarValue::from(false))); } - let bloom_filter = bloom_filter.as_ref().as_ref().unwrap(); + let bloom_filter = bloom_filter + .as_ref() + .as_ref() + .expect("bloom_filter must be present"); // process with bloom filter let value = self.value_expr.evaluate(batch)?; diff --git a/native-engine/datafusion-ext-exprs/src/cast.rs b/native-engine/datafusion-ext-exprs/src/cast.rs index 12f6c5402..aaa859e00 100644 --- a/native-engine/datafusion-ext-exprs/src/cast.rs +++ b/native-engine/datafusion-ext-exprs/src/cast.rs @@ -102,7 +102,7 @@ impl PhysicalExpr for TryCastExpr { #[cfg(test)] mod test { - use std::sync::Arc; + use std::{error::Error, sync::Arc}; use arrow::{ array::{ArrayRef, Float32Array, Int32Array, StringArray}, @@ -114,7 +114,7 @@ mod test { use crate::cast::TryCastExpr; #[test] - fn test_ok_1() { + fn test_ok_1() -> Result<(), Box> { // input: Array // cast Float32 into Int32 let float_arr: ArrayRef = Arc::new(Float32Array::from(vec![ @@ -138,15 +138,14 @@ mod test { let cast_type = DataType::Int32; let expr = Arc::new(TryCastExpr::new( - phys_expr::col("col", &batch.schema()).unwrap(), + phys_expr::col("col", &batch.schema())?, cast_type, )); let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; let expected: ArrayRef = Arc::new(Int32Array::from(vec![ Some(7), @@ -157,10 +156,11 @@ mod test { None, ])); assert_eq!(&ret, &expected); + Ok(()) } #[test] - fn test_ok_2() { + fn test_ok_2() -> Result<(), Box> { // input: Array // cast Utf8 into Float32 let string_arr: ArrayRef = Arc::new(StringArray::from(vec![ @@ -179,15 +179,14 @@ mod test { let cast_type = DataType::Float32; let expr = Arc::new(TryCastExpr::new( - phys_expr::col("col", &batch.schema()).unwrap(), + phys_expr::col("col", &batch.schema())?, cast_type, )); let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; let expected: ArrayRef = Arc::new(Float32Array::from(vec![ Some(123.0), @@ -197,10 +196,11 @@ mod test { None, ])); assert_eq!(&ret, &expected); + Ok(()) } #[test] - fn test_ok_3() { + fn test_ok_3() -> Result<(), Box> { // input: Scalar // cast Utf8 into Float32 let string_arr: ArrayRef = Arc::new(StringArray::from(vec![ @@ -223,8 +223,7 @@ mod test { let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; let expected: ArrayRef = Arc::new(Float32Array::from(vec![ Some(123.4), @@ -234,5 +233,6 @@ mod test { Some(123.4), ])); assert_eq!(&ret, &expected); + Ok(()) } } diff --git a/native-engine/datafusion-ext-exprs/src/get_map_value.rs b/native-engine/datafusion-ext-exprs/src/get_map_value.rs index 892fbdba3..e2a968e0d 100644 --- a/native-engine/datafusion-ext-exprs/src/get_map_value.rs +++ b/native-engine/datafusion-ext-exprs/src/get_map_value.rs @@ -180,16 +180,14 @@ mod test { .add_buffer(Buffer::from_slice_ref( &[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice(), )) - .build() - .unwrap(); + .build()?; let value_data = ArrayData::builder(DataType::UInt32) .len(8) .add_buffer(Buffer::from_slice_ref( &[0u32, 10, 20, 0, 40, 0, 60, 70].to_byte_slice(), )) .null_bit_buffer(Some(Buffer::from_slice_ref(&[0b11010110]))) - .build() - .unwrap(); + .build()?; let entry_offsets = Buffer::from_slice_ref(&[0, 3, 6, 8].to_byte_slice()); @@ -214,8 +212,7 @@ mod test { .len(3) .add_buffer(entry_offsets) .add_child_data(entry_struct.into_data()) - .build() - .unwrap(); + .build()?; let map_array: ArrayRef = Arc::new(MapArray::from(map_data)); let input_batch = RecordBatch::try_from_iter_with_nullable(vec![("test col", map_array, true)])?; @@ -262,10 +259,11 @@ mod test { // [[a, b, c], [d, e, f], [g, h]] let entry_offsets = [0, 3, 6, 8]; - let map_array: ArrayRef = Arc::new( - MapArray::new_from_strings(keys.clone().into_iter(), &values_data, &entry_offsets) - .unwrap(), - ); + let map_array: ArrayRef = Arc::new(MapArray::new_from_strings( + keys.clone().into_iter(), + &values_data, + &entry_offsets, + )?); let input_batch = RecordBatch::try_from_iter_with_nullable(vec![("test col", map_array, true)])?; let get_indexed = Arc::new(GetMapValueExpr::new( diff --git a/native-engine/datafusion-ext-exprs/src/lib.rs b/native-engine/datafusion-ext-exprs/src/lib.rs index eb729e2dd..3a685a41f 100644 --- a/native-engine/datafusion-ext-exprs/src/lib.rs +++ b/native-engine/datafusion-ext-exprs/src/lib.rs @@ -31,10 +31,12 @@ pub mod string_starts_with; fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { if any.is::() { - any.downcast_ref::().unwrap().as_any() + any.downcast_ref::() + .expect("Expected a PhysicalExpr") + .as_any() } else if any.is::>() { any.downcast_ref::>() - .unwrap() + .expect("Expected a PhysicalExpr") .as_any() } else { any diff --git a/native-engine/datafusion-ext-exprs/src/string_contains.rs b/native-engine/datafusion-ext-exprs/src/string_contains.rs index 029041d65..f0f60cf0d 100644 --- a/native-engine/datafusion-ext-exprs/src/string_contains.rs +++ b/native-engine/datafusion-ext-exprs/src/string_contains.rs @@ -83,7 +83,10 @@ impl PhysicalExpr for StringContainsExpr { match expr { ColumnarValue::Array(array) => { - let string_array = array.as_any().downcast_ref::().unwrap(); + let string_array = array + .as_any() + .downcast_ref::() + .expect("Expected a StringArray"); let ret_array = Arc::new(BooleanArray::from_iter(string_array.iter().map( |maybe_string| maybe_string.map(|string| string.contains(&self.infix)), @@ -116,7 +119,7 @@ impl PhysicalExpr for StringContainsExpr { #[cfg(test)] mod test { - use std::sync::Arc; + use std::{error::Error, sync::Arc}; use arrow::{ array::{ArrayRef, BooleanArray, StringArray}, @@ -128,7 +131,7 @@ mod test { use crate::string_contains::StringContainsExpr; #[test] - fn test_ok() { + fn test_ok() -> Result<(), Box> { // create a StringArray from the vector let string_array: ArrayRef = Arc::new(StringArray::from(vec![ Some("abrr".to_string()), @@ -148,14 +151,13 @@ mod test { // test: col1 like 'ba%' let pattern = "ba".to_string(); let expr = Arc::new(StringContainsExpr::new( - phys_expr::col("col1", &batch.schema()).unwrap(), + phys_expr::col("col1", &batch.schema())?, pattern, )); let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; // verify result let expected: ArrayRef = Arc::new(BooleanArray::from(vec![ @@ -166,10 +168,11 @@ mod test { None, ])); assert_eq!(&ret, &expected); + Ok(()) } #[test] - fn test_scalar_string() { + fn test_scalar_string() -> Result<(), Box> { // create a StringArray from the vector let string_array: ArrayRef = Arc::new(StringArray::from(vec![ Some("abrr".to_string()), @@ -192,8 +195,7 @@ mod test { let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; // verify result let expected: ArrayRef = Arc::new(BooleanArray::from(vec![ @@ -204,5 +206,6 @@ mod test { Some(true), ])); assert_eq!(&ret, &expected); + Ok(()) } } diff --git a/native-engine/datafusion-ext-exprs/src/string_ends_with.rs b/native-engine/datafusion-ext-exprs/src/string_ends_with.rs index 1432a24cb..77b55f241 100644 --- a/native-engine/datafusion-ext-exprs/src/string_ends_with.rs +++ b/native-engine/datafusion-ext-exprs/src/string_ends_with.rs @@ -83,7 +83,10 @@ impl PhysicalExpr for StringEndsWithExpr { match expr { ColumnarValue::Array(array) => { - let string_array = array.as_any().downcast_ref::().unwrap(); + let string_array = array + .as_any() + .downcast_ref::() + .expect("Expected a StringArray"); let ret_array = Arc::new(BooleanArray::from_iter(string_array.iter().map( |maybe_string| maybe_string.map(|string| string.ends_with(&self.suffix)), ))); @@ -118,8 +121,7 @@ impl PhysicalExpr for StringEndsWithExpr { #[cfg(test)] mod test { - - use std::sync::Arc; + use std::{error::Error, sync::Arc}; use arrow::{ array::{ArrayRef, BooleanArray, StringArray}, @@ -131,7 +133,7 @@ mod test { use crate::string_ends_with::StringEndsWithExpr; #[test] - fn test_array() { + fn test_array() -> Result<(), Box> { let string_array: ArrayRef = Arc::new(StringArray::from(vec![ Some("abrrbrr".to_string()), Some("rrjndebcsabdji".to_string()), @@ -149,14 +151,13 @@ mod test { // test: col2 like '%rr' let pattern = "rr".to_string(); let expr = Arc::new(StringEndsWithExpr::new( - phys_expr::col("col2", &batch.schema()).unwrap(), + phys_expr::col("col2", &batch.schema())?, pattern, )); let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; // verify result let expected: ArrayRef = Arc::new(BooleanArray::from(vec![ @@ -167,10 +168,11 @@ mod test { Some(false), ])); assert_eq!(&ret, &expected); + Ok(()) } #[test] - fn test_scalar_string() { + fn test_scalar_string() -> Result<(), Box> { // create a StringArray from the vector let string_array: ArrayRef = Arc::new(StringArray::from(vec![ Some("Hello, Rust".to_string()), @@ -196,8 +198,7 @@ mod test { let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; // verify result let expected: ArrayRef = Arc::new(BooleanArray::from(vec![ @@ -208,5 +209,6 @@ mod test { Some(false), ])); assert_eq!(&ret, &expected); + Ok(()) } } diff --git a/native-engine/datafusion-ext-exprs/src/string_starts_with.rs b/native-engine/datafusion-ext-exprs/src/string_starts_with.rs index 4a47fcb22..68a2813ea 100644 --- a/native-engine/datafusion-ext-exprs/src/string_starts_with.rs +++ b/native-engine/datafusion-ext-exprs/src/string_starts_with.rs @@ -83,7 +83,10 @@ impl PhysicalExpr for StringStartsWithExpr { match expr { ColumnarValue::Array(array) => { - let string_array = array.as_any().downcast_ref::().unwrap(); + let string_array = array + .as_any() + .downcast_ref::() + .expect("Expected a StringArray"); let ret_array = Arc::new(BooleanArray::from_iter(string_array.iter().map( |maybe_string| maybe_string.map(|string| string.starts_with(&self.prefix)), ))); @@ -118,7 +121,7 @@ impl PhysicalExpr for StringStartsWithExpr { #[cfg(test)] mod test { - use std::sync::Arc; + use std::{error::Error, sync::Arc}; use arrow::{ array::{ArrayRef, BooleanArray, StringArray}, @@ -130,7 +133,7 @@ mod test { use crate::string_starts_with::StringStartsWithExpr; #[test] - fn test_ok() { + fn test_ok() -> Result<(), Box> { let string_array: ArrayRef = Arc::new(StringArray::from(vec![ None, Some("rabaok".to_string()), @@ -145,14 +148,13 @@ mod test { let pattern = "ra".to_string(); let expr = Arc::new(StringStartsWithExpr::new( - phys_expr::col("col3", &batch.schema()).unwrap(), + phys_expr::col("col3", &batch.schema())?, pattern, )); let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; let expected: ArrayRef = Arc::new(BooleanArray::from(vec![ None, @@ -163,10 +165,11 @@ mod test { // None ])); assert_eq!(&ret, &expected); + Ok(()) } #[test] - fn test_scalar_string() { + fn test_scalar_string() -> Result<(), Box> { let string_array: ArrayRef = Arc::new(StringArray::from(vec![ Some("Hello, Rust".to_string()), Some("Hello, He".to_string()), @@ -184,8 +187,7 @@ mod test { let ret = expr .evaluate(&batch) .expect("Error evaluating expr") - .into_array(batch.num_rows()) - .unwrap(); + .into_array(batch.num_rows())?; let expected: ArrayRef = Arc::new(BooleanArray::from(vec![ Some(true), @@ -195,5 +197,6 @@ mod test { Some(true), ])); assert_eq!(&ret, &expected); + Ok(()) } } diff --git a/native-engine/datafusion-ext-functions/src/spark_check_overflow.rs b/native-engine/datafusion-ext-functions/src/spark_check_overflow.rs index a72384603..5a67a8444 100644 --- a/native-engine/datafusion-ext-functions/src/spark_check_overflow.rs +++ b/native-engine/datafusion-ext-functions/src/spark_check_overflow.rs @@ -55,7 +55,10 @@ pub fn spark_check_overflow(args: &[ColumnarValue]) -> Result { _ => ColumnarValue::Scalar(ScalarValue::Decimal128(None, to_precision, to_scale)), }, ColumnarValue::Array(array) => { - let array = array.as_any().downcast_ref::().unwrap(); + let array = array + .as_any() + .downcast_ref::() + .expect("Expected a Decimal128Array"); let mut output = Decimal128Builder::with_capacity(array.len()); for v in array.into_iter() { diff --git a/native-engine/datafusion-ext-functions/src/spark_crypto.rs b/native-engine/datafusion-ext-functions/src/spark_crypto.rs index fe6dd1842..de406f307 100644 --- a/native-engine/datafusion-ext-functions/src/spark_crypto.rs +++ b/native-engine/datafusion-ext-functions/src/spark_crypto.rs @@ -99,7 +99,7 @@ fn hex_encode>(data: T) -> String { let mut s = String::with_capacity(data.as_ref().len() * 2); for b in data.as_ref() { // Writing to a string never errors, so we can unwrap here. - write!(&mut s, "{b:02x}").unwrap(); + write!(&mut s, "{b:02x}").expect("writing to String should not fail"); } s } diff --git a/native-engine/datafusion-ext-functions/src/spark_dates.rs b/native-engine/datafusion-ext-functions/src/spark_dates.rs index f712f9989..3ea5e23f6 100644 --- a/native-engine/datafusion-ext-functions/src/spark_dates.rs +++ b/native-engine/datafusion-ext-functions/src/spark_dates.rs @@ -207,7 +207,7 @@ mod tests { use super::*; #[test] - fn test_spark_year() { + fn test_spark_year() -> Result<()> { let input = Arc::new(Date32Array::from(vec![ Some(0), Some(1000), @@ -221,26 +221,22 @@ mod tests { Some(1975), None, ])); - assert_eq!( - &spark_year(&args).unwrap().into_array(1).unwrap(), - &expected_ret - ); + assert_eq!(&spark_year(&args)?.into_array(1)?, &expected_ret); + Ok(()) } #[test] - fn test_spark_month() { + fn test_spark_month() -> Result<()> { let input = Arc::new(Date32Array::from(vec![Some(0), Some(35), Some(65), None])); let args = vec![ColumnarValue::Array(input)]; let expected_ret: ArrayRef = Arc::new(Int32Array::from(vec![Some(1), Some(2), Some(3), None])); - assert_eq!( - &spark_month(&args).unwrap().into_array(1).unwrap(), - &expected_ret - ); + assert_eq!(&spark_month(&args)?.into_array(1)?, &expected_ret); + Ok(()) } #[test] - fn test_spark_day() { + fn test_spark_day() -> Result<()> { let input = Arc::new(Date32Array::from(vec![ Some(0), Some(10), @@ -258,14 +254,12 @@ mod tests { Some(10), None, ])); - assert_eq!( - &spark_day(&args).unwrap().into_array(1).unwrap(), - &expected_ret - ); + assert_eq!(&spark_day(&args)?.into_array(1)?, &expected_ret); + Ok(()) } #[test] - fn test_spark_quarter_basic() { + fn test_spark_quarter_basic() -> Result<()> { // Date32 days relative to 1970-01-01: // 0 -> 1970-01-01 (Q1) // 40 -> ~1970-02-10 (Q1) @@ -290,19 +284,21 @@ mod tests { None, ])); - let out = spark_quarter(&args).unwrap().into_array(1).unwrap(); + let out = spark_quarter(&args)?.into_array(1)?; assert_eq!(&out, &expected); + Ok(()) } #[test] - fn test_spark_quarter_null_only() { + fn test_spark_quarter_null_only() -> Result<()> { // Ensure NULL propagation let input = Arc::new(Date32Array::from(vec![None, None])); let args = vec![ColumnarValue::Array(input)]; let expected: ArrayRef = Arc::new(Int32Array::from(vec![None, None])); - let out = spark_quarter(&args).unwrap().into_array(1).unwrap(); + let out = spark_quarter(&args)?.into_array(1)?; assert_eq!(&out, &expected); + Ok(()) } #[inline] diff --git a/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs b/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs index 9a8f490a9..9d5c4078f 100644 --- a/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs +++ b/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs @@ -42,7 +42,7 @@ pub fn spark_get_json_object(args: &[ColumnarValue]) -> Result { let json_strings = json_string_array .as_any() .downcast_ref::() - .unwrap(); + .expect("Expected a StringArray"); let path_string = match &args[1] { ColumnarValue::Scalar(ScalarValue::Utf8(str)) => match str { Some(path) => path, @@ -87,7 +87,7 @@ pub fn spark_parse_json(args: &[ColumnarValue]) -> Result { let json_strings = json_string_array .as_any() .downcast_ref::() - .unwrap(); + .expect("Expected a StringArray"); let fallback_enabled = conf::PARSE_JSON_ERROR_FALLBACK.value().unwrap_or(false); let json_values: Vec>> = json_strings @@ -121,7 +121,10 @@ pub fn spark_parse_json(args: &[ColumnarValue]) -> Result { pub fn spark_get_parsed_json_object(args: &[ColumnarValue]) -> Result { let json_array = match &args[0] { - ColumnarValue::Array(array) => array.as_any().downcast_ref::().unwrap(), + ColumnarValue::Array(array) => array + .as_any() + .downcast_ref::() + .expect("Expected a UserDefinedArray"), ColumnarValue::Scalar(_) => unreachable!(), }; @@ -153,7 +156,9 @@ pub fn spark_get_parsed_json_object(args: &[ColumnarValue]) -> Result Option> { - let json_value = value.downcast_ref::().unwrap(); + let json_value = value + .downcast_ref::() + .expect("Expected a ParsedJsonValue"); match json_value { ParsedJsonValue::SerdeJson(v) => evaluator .evaluate_with_value_serde_json(v) @@ -165,9 +170,10 @@ pub fn spark_get_parsed_json_object(args: &[ColumnarValue]) -> Result { - fallback_results_iter.next().unwrap().map(Cow::from) - } + ParsedJsonValue::Fallback(_) => fallback_results_iter + .next() + .expect("fallback result iterator must have a next element") + .map(Cow::from), } }) })); @@ -186,7 +192,9 @@ pub fn spark_get_parsed_json_simple_field( let output = StringArray::from_iter(json_array.iter().map(|value| { value.as_ref().and_then(|value| { - let json_value = value.downcast_ref::().unwrap(); + let json_value = value + .downcast_ref::() + .expect("Expected a ParsedJsonValue"); match json_value { ParsedJsonValue::SerdeJson(v) => v .as_object() @@ -198,9 +206,10 @@ pub fn spark_get_parsed_json_simple_field( .and_then(|object| object.get(field)) .and_then(|v| sonic_value_to_string(v).unwrap_or_default()) .map(Cow::from), - ParsedJsonValue::Fallback(_) => { - fallback_results_iter.next().unwrap().map(Cow::from) - } + ParsedJsonValue::Fallback(_) => fallback_results_iter + .next() + .expect("fallback result iterator must have a next element") + .map(Cow::from), } }) })); @@ -214,7 +223,9 @@ fn parse_fallback(json_path: &str, json_array: &UserDefinedArray) -> Result Option<&str> { - let json_value = value.downcast_ref::().unwrap(); + let json_value = value + .downcast_ref::() + .expect("Expected a ParsedJsonValue"); if let ParsedJsonValue::Fallback(json) = json_value { return Some(json.as_ref()); } @@ -544,7 +555,9 @@ impl HiveGetJsonObjectMatcher { .flat_map(|r| { // keep consistent with hive UDFJson let iter: Box> = match r { - v if v.is_array() => Box::new(v.into_array().unwrap().into_iter()), + v if v.is_array() => Box::new( + v.into_array().expect("expected array value").into_iter(), + ), other => Box::new(std::iter::once(other)), }; iter @@ -613,108 +626,108 @@ mod test { let path = "$.owner"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some("amy".to_owned()) ); let path = "$. owner"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some("amy".to_owned()) ); let path = "$.store.bicycle.price"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some("19.95".to_owned()) ); let path = "$. store. bicycle. price"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some("19.95".to_owned()) ); let path = "$.store.fruit[0]"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some(r#"{"weight":8,"type":"apple"}"#.to_owned()) ); let path = "$. store. fruit[0]"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some(r#"{"weight":8,"type":"apple"}"#.to_owned()) ); let path = "$.store.fruit[1].weight"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some("9".to_owned()) ); let path = "$.store.fruit[*]"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some(r#"[{"weight":8,"type":"apple"},{"weight":9,"type":"pear"}]"#.to_owned()) ); let path = "$. store. fruit[*]"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some(r#"[{"weight":8,"type":"apple"},{"weight":9,"type":"pear"}]"#.to_owned()) ); let path = "$.store.fruit.[1].type"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some("pear".to_owned()) ); let path = "$. store. fruit. [1]. type"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), Some("pear".to_owned()) ); let path = "$.non_exist_key"; assert_eq!( HiveGetJsonObjectEvaluator::try_new(path) - .unwrap() + .expect("failed to create evaluator") .evaluate(input) - .unwrap(), + .expect("evaluation failed"), None ); Ok(()) @@ -748,36 +761,61 @@ mod test { } }"#; let input_array = Arc::new(StringArray::from(vec![input])); - let parsed = spark_parse_json(&[ColumnarValue::Array(input_array)]).unwrap(); + let parsed = spark_parse_json(&[ColumnarValue::Array(input_array)])?; let path = ColumnarValue::Scalar(ScalarValue::from("$.message.location.county")); let r = spark_get_parsed_json_object(&[parsed.clone(), path])?.into_array(1)?; - let v = r.as_string::().iter().next().unwrap(); + let v = r + .as_string::() + .iter() + .next() + .expect("missing first element"); assert_eq!(v, Some(r#"["浦东","西直门"]"#)); let path = ColumnarValue::Scalar(ScalarValue::from("$.message.location.NOT_EXISTED")); let r = spark_get_parsed_json_object(&[parsed.clone(), path])?.into_array(1)?; - let v = r.as_string::().iter().next().unwrap(); + let v = r + .as_string::() + .iter() + .next() + .expect("missing first element"); assert_eq!(v, None); let path = ColumnarValue::Scalar(ScalarValue::from("$.message.name")); let r = spark_get_parsed_json_object(&[parsed.clone(), path])?.into_array(1)?; - let v = r.as_string::().iter().next().unwrap(); - assert!(v.unwrap().contains("Asher")); + let v = r + .as_string::() + .iter() + .next() + .expect("missing first element") + .ok_or("value is NULL")?; + assert!(v.contains("Asher")); let path = ColumnarValue::Scalar(ScalarValue::from("$.message.location.city")); let r = spark_get_parsed_json_object(&[parsed.clone(), path])?.into_array(1)?; - let v = r.as_string::().iter().next().unwrap(); + let v = r + .as_string::() + .iter() + .next() + .expect("missing first element"); assert_eq!(v, Some(r#"["1.234",1.234]"#)); let path = ColumnarValue::Scalar(ScalarValue::from("$.message.location[0]")); let r = spark_get_parsed_json_object(&[parsed.clone(), path])?.into_array(1)?; - let v = r.as_string::().iter().next().unwrap(); + let v = r + .as_string::() + .iter() + .next() + .expect("missing first element"); assert_eq!(v, Some(r#"{"city":"1.234","county":"浦东"}"#)); let path = ColumnarValue::Scalar(ScalarValue::from("$.message.location[].county")); let r = spark_get_parsed_json_object(&[parsed.clone(), path])?.into_array(1)?; - let v = r.as_string::().iter().next().unwrap(); + let v = r + .as_string::() + .iter() + .next() + .expect("missing first element"); assert_eq!(v, Some(r#"["浦东","西直门"]"#)); Ok(()) } @@ -814,7 +852,11 @@ mod test { let path = ColumnarValue::Scalar(ScalarValue::from("$.i1.j2")); let r = spark_get_parsed_json_object(&[parsed.clone(), path])?.into_array(1)?; - let v = r.as_string::().iter().next().unwrap(); + let v = r + .as_string::() + .iter() + .next() + .expect("missing first element"); // NOTE: // standard jsonpath should output [[200,300],[400, 500],null,"other"] diff --git a/native-engine/datafusion-ext-functions/src/spark_make_decimal.rs b/native-engine/datafusion-ext-functions/src/spark_make_decimal.rs index 3d221bb2b..ad22e6d0a 100644 --- a/native-engine/datafusion-ext-functions/src/spark_make_decimal.rs +++ b/native-engine/datafusion-ext-functions/src/spark_make_decimal.rs @@ -45,7 +45,10 @@ pub fn spark_make_decimal(args: &[ColumnarValue]) -> Result { _ => ColumnarValue::Scalar(ScalarValue::Decimal128(None, precision, scale)), }, ColumnarValue::Array(array) => { - let array = array.as_any().downcast_ref::().unwrap(); + let array = array + .as_any() + .downcast_ref::() + .expect("Expected a Int64Array"); let mut output = Decimal128Builder::with_capacity(array.len()); for v in array.into_iter() { diff --git a/native-engine/datafusion-ext-functions/src/spark_null_if.rs b/native-engine/datafusion-ext-functions/src/spark_null_if.rs index 41af8e478..b8a999f70 100644 --- a/native-engine/datafusion-ext-functions/src/spark_null_if.rs +++ b/native-engine/datafusion-ext-functions/src/spark_null_if.rs @@ -93,7 +93,10 @@ pub fn spark_null_if_zero(args: &[ColumnarValue]) -> Result { macro_rules! handle_decimal { ($dt:ident, $precision:expr, $scale:expr) => {{ type T = paste::paste! {arrow::datatypes::[<$dt Type>]}; - let array = array.as_any().downcast_ref::>().unwrap(); + let array = array + .as_any() + .downcast_ref::>() + .expect("Expected a PrimitiveArray"); let _0 = ::Native::from_le_bytes([0; T::BYTE_LENGTH]); let filtered = array.iter().map(|v| v.filter(|v| *v != _0)); Arc::new( @@ -159,9 +162,8 @@ mod test { ))])? .into_array(1)?; - let expected = Decimal128Array::from(vec![Some(1230427389124691)]) - .with_precision_and_scale(20, 2) - .unwrap(); + let expected = + Decimal128Array::from(vec![Some(1230427389124691)]).with_precision_and_scale(20, 2)?; let expected: ArrayRef = Arc::new(expected); assert_eq!(&result, &expected); diff --git a/native-engine/datafusion-ext-functions/src/spark_strings.rs b/native-engine/datafusion-ext-functions/src/spark_strings.rs index 83f526f90..0e5a12816 100644 --- a/native-engine/datafusion-ext-functions/src/spark_strings.rs +++ b/native-engine/datafusion-ext-functions/src/spark_strings.rs @@ -143,7 +143,7 @@ pub fn string_concat(args: &[ColumnarValue]) -> Result { } ColumnarValue::Array(v) => { if v.is_valid(index) { - let v = as_string_array(v).unwrap(); + let v = as_string_array(v).expect("Expected a StringArray"); owned_string.push_str(v.value(index)); } else { is_not_null = false; @@ -293,7 +293,8 @@ pub fn string_concat_ws(args: &[ColumnarValue]) -> Result { } Arg::List(list) => { if list.is_valid(i) { - let strings = as_string_array(list.values()).unwrap(); + let strings = + as_string_array(list.values()).expect("Expected a StringArray"); let offsets = list.value_offsets(); let l = offsets[i] as usize; let r = offsets[i + 1] as usize; diff --git a/native-engine/datafusion-ext-functions/src/spark_unscaled_value.rs b/native-engine/datafusion-ext-functions/src/spark_unscaled_value.rs index 1ae2b0635..30a4b8582 100644 --- a/native-engine/datafusion-ext-functions/src/spark_unscaled_value.rs +++ b/native-engine/datafusion-ext-functions/src/spark_unscaled_value.rs @@ -31,7 +31,10 @@ pub fn spark_unscaled_value(args: &[ColumnarValue]) -> Result { _ => ColumnarValue::Scalar(ScalarValue::Int64(None)), }, ColumnarValue::Array(array) => { - let array = array.as_any().downcast_ref::().unwrap(); + let array = array + .as_any() + .downcast_ref::() + .expect("Expected a Decimal128Array"); let mut output = Int64Builder::new(); for v in array.into_iter() { diff --git a/native-engine/datafusion-ext-plans/src/agg/agg_ctx.rs b/native-engine/datafusion-ext-plans/src/agg/agg_ctx.rs index aa1ee0019..2794684f9 100644 --- a/native-engine/datafusion-ext-plans/src/agg/agg_ctx.rs +++ b/native-engine/datafusion-ext-plans/src/agg/agg_ctx.rs @@ -277,12 +277,13 @@ impl AggContext { let mut merging_acc_table = self.create_acc_table(0); if self.need_partial_merge { - let partial_merged_array = as_binary_array(batch.columns().last().unwrap())?; + let partial_merged_array = + as_binary_array(batch.columns().last().expect("last column"))?; let array = partial_merged_array .iter() .skip(batch_start_idx) .take(batch_end_idx - batch_start_idx) - .map(|bytes| bytes.unwrap()) + .map(|bytes| bytes.expect("non-null bytes")) .collect::>(); let mut cursors = array .iter() diff --git a/native-engine/datafusion-ext-plans/src/agg/agg_table.rs b/native-engine/datafusion-ext-plans/src/agg/agg_table.rs index 2e4333912..c432c892d 100644 --- a/native-engine/datafusion-ext-plans/src/agg/agg_table.rs +++ b/native-engine/datafusion-ext-plans/src/agg/agg_table.rs @@ -821,8 +821,8 @@ impl<'a> RecordsSpillCursor<'a> { )?; // load next bucket head - self.cur_bucket_idx = read_len(&mut self.input).unwrap(); - self.cur_bucket_count = read_len(&mut self.input).unwrap(); + self.cur_bucket_idx = read_len(&mut self.input).expect("read bucket index failed"); + self.cur_bucket_count = read_len(&mut self.input).expect("read bucket count failed"); Ok((acc_table, keys)) } } diff --git a/native-engine/datafusion-ext-plans/src/agg/bloom_filter.rs b/native-engine/datafusion-ext-plans/src/agg/bloom_filter.rs index 7c6f1da33..6aff3c849 100644 --- a/native-engine/datafusion-ext-plans/src/agg/bloom_filter.rs +++ b/native-engine/datafusion-ext-plans/src/agg/bloom_filter.rs @@ -127,7 +127,7 @@ impl Agg for AggBloomFilter { self.num_bits, )); } - bf.as_mut().unwrap() + bf.as_mut().expect("bloom_filter missing") } _ => return df_unimplemented_err!("AggBloomFilter only supports one bloom filter"), }; diff --git a/native-engine/datafusion-ext-plans/src/agg/collect.rs b/native-engine/datafusion-ext-plans/src/agg/collect.rs index 6ddeed832..225d4303d 100644 --- a/native-engine/datafusion-ext-plans/src/agg/collect.rs +++ b/native-engine/datafusion-ext-plans/src/agg/collect.rs @@ -442,7 +442,7 @@ impl AccList { } pub fn append(&mut self, value: &ScalarValue, nullable: bool) { - write_scalar(&value, nullable, &mut self.raw).unwrap(); + write_scalar(&value, nullable, &mut self.raw).expect("write scalar failed"); } pub fn merge(&mut self, other: &mut Self) { @@ -456,7 +456,9 @@ impl AccList { fn next(&mut self) -> Option { if self.0.position() < self.0.get_ref().len() as u64 { - return Some(read_scalar(&mut self.0, &self.1, self.2).unwrap()); + return Some( + read_scalar(&mut self.0, &self.1, self.2).expect("read scalar failed"), + ); } None } @@ -534,7 +536,7 @@ impl AccSet { pub fn append(&mut self, value: &ScalarValue, nullable: bool) { let old_raw_len = self.list.raw.len(); - write_scalar(value, nullable, &mut self.list.raw).unwrap(); + write_scalar(value, nullable, &mut self.list.raw).expect("write scalar failed"); self.append_raw_inline(old_raw_len); } @@ -713,7 +715,7 @@ mod tests { } #[test] - fn test_acc_set_spill() { + fn test_acc_set_spill() -> Result<()> { let mut acc_col = AccSetColumn::empty(DataType::Int32); acc_col.resize(3); acc_col.append_item(1, &ScalarValue::Int32(Some(1))); @@ -726,18 +728,15 @@ mod tests { let mut spill: Box = Box::new(vec![]); let mut spill_writer = spill.get_compressed_writer(); - acc_col - .spill(IdxSelection::Range(0, 3), &mut spill_writer) - .unwrap(); - spill_writer.finish().unwrap(); + acc_col.spill(IdxSelection::Range(0, 3), &mut spill_writer)?; + spill_writer.finish()?; let mut acc_col_unspill = AccSetColumn::empty(DataType::Int32); - acc_col_unspill - .unspill(3, &mut spill.get_compressed_reader()) - .unwrap(); + acc_col_unspill.unspill(3, &mut spill.get_compressed_reader())?; assert_eq!(acc_col.take_values(0), acc_col_unspill.take_values(0)); assert_eq!(acc_col.take_values(1), acc_col_unspill.take_values(1)); assert_eq!(acc_col.take_values(2), acc_col_unspill.take_values(2)); + Ok(()) } } diff --git a/native-engine/datafusion-ext-plans/src/agg/spark_udaf_wrapper.rs b/native-engine/datafusion-ext-plans/src/agg/spark_udaf_wrapper.rs index deba5d344..15dd82bfc 100644 --- a/native-engine/datafusion-ext-plans/src/agg/spark_udaf_wrapper.rs +++ b/native-engine/datafusion-ext-plans/src/agg/spark_udaf_wrapper.rs @@ -227,14 +227,14 @@ impl Agg for SparkUDAFWrapper { } fn create_acc_column(&self, num_rows: usize) -> AccColumnRef { - let jcontext = self.jcontext().unwrap(); + let jcontext = self.jcontext().expect("jcontext must be initialized"); let rows = jni_call!(SparkUDAFWrapperContext(jcontext.as_obj()).initialize( num_rows as i32, )-> JObject) - .unwrap(); + .expect("init rows failed"); - let jcontext = self.jcontext().unwrap(); - let obj = jni_new_global_ref!(rows.as_obj()).unwrap(); + let jcontext = self.jcontext().expect("jcontext must be initialized"); + let obj = jni_new_global_ref!(rows.as_obj()).expect("failed to create global ref for rows"); Box::new(AccUDAFBufferRowsColumn { obj, jcontext }) } diff --git a/native-engine/datafusion-ext-plans/src/agg_exec.rs b/native-engine/datafusion-ext-plans/src/agg_exec.rs index a77babd04..23ceacbf9 100644 --- a/native-engine/datafusion-ext-plans/src/agg_exec.rs +++ b/native-engine/datafusion-ext-plans/src/agg_exec.rs @@ -447,7 +447,7 @@ mod test { f: (&str, &Vec), g: (&str, &Vec), h: (&str, &Vec), - ) -> RecordBatch { + ) -> Result { let schema = Schema::new(vec![ Field::new(a.0, DataType::Int32, false), Field::new(b.0, DataType::Int32, false), @@ -459,7 +459,7 @@ mod test { Field::new(h.0, DataType::Int32, false), ]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![ Arc::new(Int32Array::from(a.1.clone())), @@ -471,8 +471,8 @@ mod test { Arc::new(Int32Array::from(g.1.clone())), Arc::new(Int32Array::from(h.1.clone())), ], - ) - .unwrap() + )?; + Ok(batch) } fn build_table( @@ -484,10 +484,11 @@ mod test { f: (&str, &Vec), g: (&str, &Vec), h: (&str, &Vec), - ) -> Arc { - let batch = build_table_i32(a, b, c, d, e, f, g, h); + ) -> Result> { + let batch = build_table_i32(a, b, c, d, e, f, g, h)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + let exec = TestMemoryExec::try_new(&[vec![batch]], schema, None)?; + Ok(Arc::new(exec)) } #[tokio::test(flavor = "multi_thread", worker_threads = 1)] @@ -503,7 +504,7 @@ mod test { ("f", &vec![0, 1, 2, 3, 4, 5, 6]), ("g", &vec![6, 3, 6, 3, 1, 5, 4]), ("h", &vec![6, 3, 6, 3, 1, 5, 4]), - ); + )?; let agg_expr_sum = create_agg( AggFunction::Sum, diff --git a/native-engine/datafusion-ext-plans/src/common/cached_exprs_evaluator.rs b/native-engine/datafusion-ext-plans/src/common/cached_exprs_evaluator.rs index 1c7ec79fe..a5c4d1ed0 100644 --- a/native-engine/datafusion-ext-plans/src/common/cached_exprs_evaluator.rs +++ b/native-engine/datafusion-ext-plans/src/common/cached_exprs_evaluator.rs @@ -479,7 +479,7 @@ fn prune_expr_cols(expr: &PhysicalExprRef) -> (PhysicalExprRef, Vec) { Ok(Transformed::yes(expr)) } }) - .unwrap() + .expect("expr transform failed") .data; let mapped_cols: Vec = used_cols diff --git a/native-engine/datafusion-ext-plans/src/common/column_pruning.rs b/native-engine/datafusion-ext-plans/src/common/column_pruning.rs index 267ffa190..143aaa170 100644 --- a/native-engine/datafusion-ext-plans/src/common/column_pruning.rs +++ b/native-engine/datafusion-ext-plans/src/common/column_pruning.rs @@ -114,7 +114,7 @@ pub fn extend_projection_by_expr( )) }) .map(|r| r.data) - .unwrap() + .expect("expr transform failed") } pub fn map_columns(expr: &PhysicalExprRef, mapping: &HashMap) -> PhysicalExprRef { @@ -130,5 +130,5 @@ pub fn map_columns(expr: &PhysicalExprRef, mapping: &HashMap) -> P )) }) .map(|r| r.data) - .unwrap() + .expect("expr transform failed") } diff --git a/native-engine/datafusion-ext-plans/src/common/execution_context.rs b/native-engine/datafusion-ext-plans/src/common/execution_context.rs index 15b6e69d0..8f2dd14fc 100644 --- a/native-engine/datafusion-ext-plans/src/common/execution_context.rs +++ b/native-engine/datafusion-ext-plans/src/common/execution_context.rs @@ -724,7 +724,7 @@ impl WrappedSender { send_time.inspect(|send_time| { exclude_time .as_ref() - .unwrap() + .expect("exclude_time must be set") .sub_duration(send_time.elapsed()); }); } diff --git a/native-engine/datafusion-ext-plans/src/common/row_null_checker.rs b/native-engine/datafusion-ext-plans/src/common/row_null_checker.rs index 83582068c..50a7cdfe1 100644 --- a/native-engine/datafusion-ext-plans/src/common/row_null_checker.rs +++ b/native-engine/datafusion-ext-plans/src/common/row_null_checker.rs @@ -75,9 +75,12 @@ impl RowNullChecker { encoded_length: 0, }, DataType::Boolean => FieldConfig::new_boolean(sort_options), - dt if dt.is_primitive() => { - FieldConfig::new_primitive(sort_options, 1 + dt.primitive_width().unwrap()) - } + dt if dt.is_primitive() => FieldConfig::new_primitive( + sort_options, + 1 + dt + .primitive_width() + .expect("primitive_width must be present"), + ), // DataType::Int8 => FieldConfig::new_primitive(sort_options, 2), // 1 byte null flag + // // 1 byte value // DataType::Int16 => FieldConfig::new_primitive(sort_options, 3), /* 1 byte null flag + @@ -445,7 +448,7 @@ impl FieldConfig { #[cfg(test)] mod tests { - use std::sync::Arc; + use std::{error::Error, sync::Arc}; use arrow::{ array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}, @@ -539,7 +542,7 @@ mod tests { } #[test] - fn test_roundtrip_with_record_batch() { + fn test_roundtrip_with_record_batch() -> Result<(), Box> { // Create a schema with multiple data types let schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, false), @@ -570,8 +573,7 @@ mod tests { Arc::new(name_array), Arc::new(active_array), ], - ) - .unwrap(); + )?; // Create RowNullChecker let checker = RowNullChecker::new( @@ -618,6 +620,7 @@ mod tests { // Verify that row count matches RecordBatch assert_eq!(record_batch.num_rows(), 4); assert_eq!(record_batch.num_columns(), 3); + Ok(()) } #[test] @@ -691,7 +694,7 @@ mod tests { } #[test] - fn test_has_nulls_with_rows() { + fn test_has_nulls_with_rows() -> Result<(), Box> { use arrow::{array::ArrayRef, row::RowConverter}; // Create a schema @@ -708,7 +711,7 @@ mod tests { let columns: Vec = vec![Arc::new(id_array), Arc::new(name_array)]; // Create RecordBatch - let batch = RecordBatch::try_new(schema.clone(), columns).unwrap(); + let batch = RecordBatch::try_new(schema.clone(), columns)?; // Create sort fields for RowConverter let sort_fields: Vec = schema @@ -720,8 +723,8 @@ mod tests { .collect(); // Convert RecordBatch to Rows - let converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converter = RowConverter::new(sort_fields.clone())?; + let rows = converter.convert_columns(&batch.columns())?; // Create field configs for RowNullChecker let field_configs: Vec<(DataType, SortOptions)> = schema @@ -746,10 +749,11 @@ mod tests { assert_eq!(null_buffer.is_valid(1), false); // Has null in name assert_eq!(null_buffer.is_valid(2), false); // Has null in id assert_eq!(null_buffer.is_valid(3), true); // No nulls + Ok(()) } #[test] - fn test_has_nulls_empty_rows() { + fn test_has_nulls_empty_rows() -> Result<(), Box> { // Test with empty rows let field_configs = vec![(DataType::Int32, SortOptions::default())]; let checker = RowNullChecker::new(&field_configs); @@ -760,7 +764,7 @@ mod tests { let id_array = Int32Array::from(Vec::>::new()); let columns: Vec = vec![Arc::new(id_array)]; - let batch = RecordBatch::try_new(schema.clone(), columns).unwrap(); + let batch = RecordBatch::try_new(schema.clone(), columns)?; let sort_fields: Vec = schema .fields() @@ -770,21 +774,22 @@ mod tests { }) .collect(); - let converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converter = RowConverter::new(sort_fields.clone())?; + let rows = converter.convert_columns(&batch.columns())?; let null_buffer = checker.has_nulls(&rows); assert_eq!(null_buffer.len(), 0); + Ok(()) } #[test] - fn test_has_nulls_all_nulls() { + fn test_has_nulls_all_nulls() -> Result<(), Box> { // Test with all rows containing nulls let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, true)])); let id_array = Int32Array::from(vec![None, None, None]); let columns: Vec = vec![Arc::new(id_array)]; - let batch = RecordBatch::try_new(schema.clone(), columns).unwrap(); + let batch = RecordBatch::try_new(schema.clone(), columns)?; let sort_fields: Vec = schema .fields() @@ -794,8 +799,8 @@ mod tests { }) .collect(); - let converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converter = RowConverter::new(sort_fields.clone())?; + let rows = converter.convert_columns(&batch.columns())?; let field_configs: Vec<(DataType, SortOptions)> = schema .fields() @@ -811,10 +816,11 @@ mod tests { for i in 0..3 { assert_eq!(null_buffer.is_valid(i), false); } + Ok(()) } #[test] - fn test_has_nulls_no_nulls() { + fn test_has_nulls_no_nulls() -> Result<(), Box> { // Test with no nulls in any row let schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, true), @@ -824,7 +830,7 @@ mod tests { let id_array = Int32Array::from(vec![Some(1), Some(2), Some(3)]); let name_array = StringArray::from(vec![Some("Alice"), Some("Bob"), Some("Charlie")]); let columns: Vec = vec![Arc::new(id_array), Arc::new(name_array)]; - let batch = RecordBatch::try_new(schema.clone(), columns).unwrap(); + let batch = RecordBatch::try_new(schema.clone(), columns)?; let sort_fields: Vec = schema .fields() @@ -834,8 +840,8 @@ mod tests { }) .collect(); - let converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converter = RowConverter::new(sort_fields.clone())?; + let rows = converter.convert_columns(&batch.columns())?; let field_configs: Vec<(DataType, SortOptions)> = schema .fields() @@ -851,5 +857,6 @@ mod tests { for i in 0..3 { assert_eq!(null_buffer.is_valid(i), true); } + Ok(()) } } diff --git a/native-engine/datafusion-ext-plans/src/expand_exec.rs b/native-engine/datafusion-ext-plans/src/expand_exec.rs index 9030e3cdf..5485e926f 100644 --- a/native-engine/datafusion-ext-plans/src/expand_exec.rs +++ b/native-engine/datafusion-ext-plans/src/expand_exec.rs @@ -208,133 +208,134 @@ mod test { use crate::expand_exec::ExpandExec; // build i32 table - fn build_table_i32(a: (&str, &Vec)) -> RecordBatch { + fn build_table_i32(a: (&str, &Vec)) -> Result { let schema = Schema::new(vec![Field::new(a.0, DataType::Int32, false)]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![Arc::new(Int32Array::from(a.1.clone()))], - ) - .unwrap() + )?; + Ok(batch) } - fn build_table_int(a: (&str, &Vec)) -> Arc { - let batch = build_table_i32(a); + fn build_table_int(a: (&str, &Vec)) -> Result> { + let batch = build_table_i32(a)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } // build f32 table - fn build_table_f32(a: (&str, &Vec)) -> RecordBatch { + fn build_table_f32(a: (&str, &Vec)) -> Result { let schema = Schema::new(vec![Field::new(a.0, DataType::Float32, false)]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![Arc::new(Float32Array::from(a.1.clone()))], - ) - .unwrap() + )?; + Ok(batch) } - fn build_table_float(a: (&str, &Vec)) -> Arc { - let batch = build_table_f32(a); + fn build_table_float(a: (&str, &Vec)) -> Result> { + let batch = build_table_f32(a)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } // build str table - fn build_table_str(a: (&str, &Vec)) -> RecordBatch { + fn build_table_str(a: (&str, &Vec)) -> Result { let schema = Schema::new(vec![Field::new(a.0, DataType::Utf8, false)]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![Arc::new(StringArray::from(a.1.clone()))], - ) - .unwrap() + )?; + Ok(batch) } - fn build_table_string(a: (&str, &Vec)) -> Arc { - let batch = build_table_str(a); + fn build_table_string(a: (&str, &Vec)) -> Result> { + let batch = build_table_str(a)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } // build boolean table - fn build_table_bool(a: (&str, &Vec)) -> RecordBatch { + fn build_table_bool(a: (&str, &Vec)) -> Result { let schema = Schema::new(vec![Field::new(a.0, DataType::Boolean, false)]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![Arc::new(BooleanArray::from(a.1.clone()))], - ) - .unwrap() + )?; + Ok(batch) } - fn build_table_boolean(a: (&str, &Vec)) -> Arc { - let batch = build_table_bool(a); + fn build_table_boolean(a: (&str, &Vec)) -> Result> { + let batch = build_table_bool(a)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } #[tokio::test] async fn test_expand_exec_i32() -> Result<()> { MemManager::init(10000); - let input = build_table_int(("a", &vec![-1, -2, 0, 3])); + let input = build_table_int(("a", &vec![-1, -2, 0, 3]))?; let schema = Schema::new(vec![Field::new("test_i32", DataType::Int32, false)]); let projections = vec![ - vec![ - binary( - col("test_i32", &schema).unwrap(), - Operator::Multiply, - lit(ScalarValue::from(2)), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_i32", &schema).unwrap(), - Operator::Plus, - lit(ScalarValue::from(100)), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_i32", &schema).unwrap(), - Operator::Divide, - lit(ScalarValue::from(-2)), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_i32", &schema).unwrap(), - Operator::Modulo, - lit(ScalarValue::from(2)), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_i32", &schema).unwrap(), - Operator::BitwiseShiftLeft, - lit(ScalarValue::from(1)), - &schema, - ) - .unwrap(), - ], + vec![binary( + col("test_i32", &schema)?, + Operator::Multiply, + lit(ScalarValue::from(2)), + &schema, + )?], + vec![binary( + col("test_i32", &schema)?, + Operator::Plus, + lit(ScalarValue::from(100)), + &schema, + )?], + vec![binary( + col("test_i32", &schema)?, + Operator::Divide, + lit(ScalarValue::from(-2)), + &schema, + )?], + vec![binary( + col("test_i32", &schema)?, + Operator::Modulo, + lit(ScalarValue::from(2)), + &schema, + )?], + vec![binary( + col("test_i32", &schema)?, + Operator::BitwiseShiftLeft, + lit(ScalarValue::from(1)), + &schema, + )?], ]; let expand_exec = ExpandExec::try_new(input.schema(), projections, input)?; let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let output = expand_exec.execute(0, task_ctx).unwrap(); + let output = expand_exec.execute(0, task_ctx)?; let batches = common::collect(output).await?; let expected = vec![ "+-----+", "| a |", "+-----+", "| -2 |", "| -4 |", "| 0 |", "| 6 |", "| 99 |", @@ -350,53 +351,41 @@ mod test { async fn test_expand_exec_f32() -> Result<()> { MemManager::init(10000); - let input = build_table_float(("a", &vec![-1.2, -2.3, 0.0, 3.4])); + let input = build_table_float(("a", &vec![-1.2, -2.3, 0.0, 3.4]))?; let schema = Schema::new(vec![Field::new("test_f32", DataType::Float32, false)]); let projections = vec![ - vec![ - binary( - col("test_f32", &schema).unwrap(), - Operator::Multiply, - lit(ScalarValue::from(2.1_f32)), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_f32", &schema).unwrap(), - Operator::Plus, - lit(ScalarValue::from(100_f32)), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_f32", &schema).unwrap(), - Operator::Divide, - lit(ScalarValue::from(-2_f32)), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_f32", &schema).unwrap(), - Operator::Modulo, - lit(ScalarValue::from(-2_f32)), - &schema, - ) - .unwrap(), - ], + vec![binary( + col("test_f32", &schema)?, + Operator::Multiply, + lit(ScalarValue::from(2.1_f32)), + &schema, + )?], + vec![binary( + col("test_f32", &schema)?, + Operator::Plus, + lit(ScalarValue::from(100_f32)), + &schema, + )?], + vec![binary( + col("test_f32", &schema)?, + Operator::Divide, + lit(ScalarValue::from(-2_f32)), + &schema, + )?], + vec![binary( + col("test_f32", &schema)?, + Operator::Modulo, + lit(ScalarValue::from(-2_f32)), + &schema, + )?], ]; let expand_exec = ExpandExec::try_new(input.schema(), projections, input)?; let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let output = expand_exec.execute(0, task_ctx).unwrap(); + let output = expand_exec.execute(0, task_ctx)?; let batches = common::collect(output).await?; let expected = vec![ "+-------------+", @@ -437,24 +426,21 @@ mod test { "rust".to_string(), "!".to_string(), ], - )); + ))?; let schema = Schema::new(vec![Field::new("test_str", DataType::Utf8, false)]); - let projections = vec![vec![ - binary( - col("test_str", &schema).unwrap(), - Operator::StringConcat, - lit(Some("app").unwrap()), - &schema, - ) - .unwrap(), - ]]; + let projections = vec![vec![binary( + col("test_str", &schema)?, + Operator::StringConcat, + lit(Some("app").expect("app")), + &schema, + )?]]; let expand_exec = ExpandExec::try_new(input.schema(), projections, input)?; let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let output = expand_exec.execute(0, task_ctx).unwrap(); + let output = expand_exec.execute(0, task_ctx)?; let batches = common::collect(output).await?; let expected = vec![ "+----------+", @@ -475,35 +461,29 @@ mod test { async fn test_expand_exec_bool() -> Result<()> { MemManager::init(10000); - let input = build_table_boolean(("a", &vec![true, false, true, false])); + let input = build_table_boolean(("a", &vec![true, false, true, false]))?; let schema = Schema::new(vec![Field::new("test_bool", DataType::Boolean, false)]); let projections = vec![ - vec![ - binary( - col("test_bool", &schema).unwrap(), - Operator::And, - lit(ScalarValue::Boolean(Some(true))), - &schema, - ) - .unwrap(), - ], - vec![ - binary( - col("test_bool", &schema).unwrap(), - Operator::Or, - lit(ScalarValue::Boolean(Some(true))), - &schema, - ) - .unwrap(), - ], + vec![binary( + col("test_bool", &schema)?, + Operator::And, + lit(ScalarValue::Boolean(Some(true))), + &schema, + )?], + vec![binary( + col("test_bool", &schema)?, + Operator::Or, + lit(ScalarValue::Boolean(Some(true))), + &schema, + )?], ]; let expand_exec = ExpandExec::try_new(input.schema(), projections, input)?; let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let output = expand_exec.execute(0, task_ctx).unwrap(); + let output = expand_exec.execute(0, task_ctx)?; let batches = common::collect(output).await?; let expected = vec![ "+-------+", diff --git a/native-engine/datafusion-ext-plans/src/generate_exec.rs b/native-engine/datafusion-ext-plans/src/generate_exec.rs index 34dcb4ffa..ae975237f 100644 --- a/native-engine/datafusion-ext-plans/src/generate_exec.rs +++ b/native-engine/datafusion-ext-plans/src/generate_exec.rs @@ -106,7 +106,7 @@ impl GenerateExec { self.generator_output_schema.clone(), outer, ) - .unwrap() + .expect("GeneratorExec::try_new failed") } } diff --git a/native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs b/native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs index 49be1d4e5..c4c51d6a7 100644 --- a/native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs +++ b/native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs @@ -146,8 +146,8 @@ impl Table { match len { 0 => unreachable!(), 1 => { - let single = mapped_indices.pop().unwrap(); - let _len = mapped_indices.pop().unwrap(); + let single = mapped_indices.pop().expect("missing index"); + let _len = mapped_indices.pop().expect("missing length"); MapValue::new_single(single) } _ => MapValue::new_range(start), diff --git a/native-engine/datafusion-ext-plans/src/joins/test.rs b/native-engine/datafusion-ext-plans/src/joins/test.rs index 9125ed53e..ccdff53aa 100644 --- a/native-engine/datafusion-ext-plans/src/joins/test.rs +++ b/native-engine/datafusion-ext-plans/src/joins/test.rs @@ -28,8 +28,7 @@ mod tests { use auron_memmgr::MemManager; use datafusion::{ assert_batches_sorted_eq, - common::JoinSide, - error::Result, + common::{JoinSide, Result}, physical_expr::expressions::Column, physical_plan::{ExecutionPlan, common, joins::utils::*, test::TestMemoryExec}, prelude::SessionContext, @@ -59,44 +58,51 @@ mod tests { a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> RecordBatch { + ) -> Result { let schema = Schema::new(vec![ Field::new(a.0, DataType::Int32, false), Field::new(b.0, DataType::Int32, false), Field::new(c.0, DataType::Int32, false), ]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![ Arc::new(Int32Array::from(a.1.clone())), Arc::new(Int32Array::from(b.1.clone())), Arc::new(Int32Array::from(c.1.clone())), ], - ) - .unwrap() + )?; + Ok(batch) } fn build_table( a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> Arc { - let batch = build_table_i32(a, b, c); + ) -> Result> { + let batch = build_table_i32(a, b, c)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } fn build_table_from_batches(batches: Vec) -> Arc { - let schema = batches.first().unwrap().schema(); - Arc::new(TestMemoryExec::try_new(&[batches], schema, None).unwrap()) + let schema = batches.first().expect("missing first batch").schema(); + Arc::new( + TestMemoryExec::try_new(&[batches], schema, None) + .expect("failed to create memory exec"), + ) } fn build_date_table( a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> Arc { + ) -> Result> { let schema = Schema::new(vec![ Field::new(a.0, DataType::Date32, false), Field::new(b.0, DataType::Date32, false), @@ -110,18 +116,21 @@ mod tests { Arc::new(Date32Array::from(b.1.clone())), Arc::new(Date32Array::from(c.1.clone())), ], - ) - .unwrap(); + )?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } fn build_date64_table( a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> Arc { + ) -> Result> { let schema = Schema::new(vec![ Field::new(a.0, DataType::Date64, false), Field::new(b.0, DataType::Date64, false), @@ -135,11 +144,14 @@ mod tests { Arc::new(Date64Array::from(b.1.clone())), Arc::new(Date64Array::from(c.1.clone())), ], - ) - .unwrap(); + )?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } /// returns a table with 3 columns of i32 in memory @@ -147,7 +159,7 @@ mod tests { a: (&str, &Vec>), b: (&str, &Vec>), c: (&str, &Vec>), - ) -> Arc { + ) -> Result> { let schema = Arc::new(Schema::new(vec![ Field::new(a.0, DataType::Int32, true), Field::new(b.0, DataType::Int32, true), @@ -160,9 +172,12 @@ mod tests { Arc::new(Int32Array::from(b.1.clone())), Arc::new(Int32Array::from(c.1.clone())), ], - ) - .unwrap(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + )?; + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } fn build_join_schema_for_test( @@ -279,12 +294,12 @@ mod tests { ("a1", &vec![1, 2, 3]), ("b1", &vec![4, 5, 5]), // this has a repetition ("c1", &vec![7, 8, 9]), - ); + )?; let right = build_table( ("a2", &vec![10, 20, 30]), ("b1", &vec![4, 5, 6]), ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), @@ -314,12 +329,12 @@ mod tests { ("a1", &vec![1, 2, 2]), ("b2", &vec![1, 2, 2]), ("c1", &vec![7, 8, 9]), - ); + )?; let right = build_table( ("a1", &vec![1, 2, 3]), ("b2", &vec![1, 2, 2]), ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![ ( Arc::new(Column::new_with_schema("a1", &left.schema())?), @@ -354,12 +369,12 @@ mod tests { ("a1", &vec![1, 1, 2]), ("b2", &vec![1, 1, 2]), ("c1", &vec![7, 8, 9]), - ); + )?; let right = build_table( ("a1", &vec![1, 1, 3]), ("b2", &vec![1, 1, 2]), ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![ ( Arc::new(Column::new_with_schema("a1", &left.schema())?), @@ -395,12 +410,12 @@ mod tests { ("a1", &vec![Some(1), Some(1), Some(2), Some(2)]), ("b2", &vec![None, Some(1), Some(2), Some(2)]), // null in key field ("c1", &vec![Some(1), None, Some(8), Some(9)]), // null in non-key field - ); + )?; let right = build_table_i32_nullable( ("a1", &vec![Some(1), Some(1), Some(2), Some(3)]), ("b2", &vec![None, Some(1), Some(2), Some(2)]), ("c2", &vec![Some(10), Some(70), Some(80), Some(90)]), - ); + )?; let on: JoinOn = vec![ ( Arc::new(Column::new_with_schema("a1", &left.schema())?), @@ -435,12 +450,12 @@ mod tests { ("a1", &vec![1, 2, 3]), ("b1", &vec![4, 5, 7]), // 7 does not exist on the right ("c1", &vec![7, 8, 9]), - ); + )?; let right = build_table( ("a2", &vec![10, 20, 30]), ("b1", &vec![4, 5, 6]), ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), Arc::new(Column::new_with_schema("b1", &right.schema())?), @@ -469,12 +484,12 @@ mod tests { ("a1", &vec![1, 2, 3]), ("b1", &vec![4, 5, 7]), ("c1", &vec![7, 8, 9]), - ); + )?; let right = build_table( ("a2", &vec![10, 20, 30]), ("b1", &vec![4, 5, 6]), // 6 does not exist on the left ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), Arc::new(Column::new_with_schema("b1", &right.schema())?), @@ -503,12 +518,12 @@ mod tests { ("a1", &vec![1, 2, 2, 3]), ("b1", &vec![4, 5, 5, 7]), // 7 does not exist on the right ("c1", &vec![7, 8, 80, 9]), - ); + )?; let right = build_table( ("a2", &vec![10, 20, 20, 30]), ("b2", &vec![4, 5, 5, 6]), ("c2", &vec![70, 80, 800, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), Arc::new(Column::new_with_schema("b2", &right.schema())?), @@ -540,12 +555,12 @@ mod tests { ("a1", &vec![1, 2, 2, 3, 5]), ("b1", &vec![4, 5, 5, 7, 7]), // 7 does not exist on the right ("c1", &vec![7, 8, 8, 9, 11]), - ); + )?; let right = build_table( ("a2", &vec![10, 20, 30]), ("b1", &vec![4, 5, 6]), ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), Arc::new(Column::new_with_schema("b1", &right.schema())?), @@ -573,12 +588,12 @@ mod tests { ("a1", &vec![1, 2, 2, 3]), ("b1", &vec![4, 5, 5, 7]), // 7 does not exist on the right ("c1", &vec![7, 8, 8, 9]), - ); + )?; let right = build_table( ("a2", &vec![10, 20, 30]), ("b1", &vec![4, 5, 6]), // 5 is double on the right ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), Arc::new(Column::new_with_schema("b1", &right.schema())?), @@ -607,12 +622,12 @@ mod tests { ("a", &vec![1, 2, 3]), ("b", &vec![4, 5, 7]), ("c", &vec![7, 8, 9]), - ); + )?; let right = build_table( ("a", &vec![10, 20, 30]), ("b", &vec![1, 2, 7]), ("c", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( // join on a=b so there are duplicate column names on unjoined columns Arc::new(Column::new_with_schema("a", &left.schema())?), @@ -641,12 +656,12 @@ mod tests { ("a1", &vec![1, 2, 3]), ("b1", &vec![19107, 19108, 19108]), // this has a repetition ("c1", &vec![7, 8, 9]), - ); + )?; let right = build_date_table( ("a2", &vec![10, 20, 30]), ("b1", &vec![19107, 19108, 19109]), ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), @@ -678,12 +693,12 @@ mod tests { ("b1", &vec![1650703441000, 1650903441000, 1650903441000]), /* this has a * repetition */ ("c1", &vec![7, 8, 9]), - ); + )?; let right = build_date64_table( ("a2", &vec![10, 20, 30]), ("b1", &vec![1650703441000, 1650503441000, 1650903441000]), ("c2", &vec![70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), @@ -714,12 +729,12 @@ mod tests { ("a1", &vec![0, 1, 2, 3, 4, 5]), ("b1", &vec![3, 4, 5, 6, 6, 7]), ("c1", &vec![4, 5, 6, 7, 8, 9]), - ); + )?; let right = build_table( ("a2", &vec![0, 10, 20, 30, 40]), ("b2", &vec![2, 4, 6, 6, 8]), ("c2", &vec![50, 60, 70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), Arc::new(Column::new_with_schema("b2", &right.schema())?), @@ -752,12 +767,12 @@ mod tests { ("a1", &vec![0, 1, 2, 3]), ("b1", &vec![3, 4, 5, 7]), ("c1", &vec![6, 7, 8, 9]), - ); + )?; let right = build_table( ("a2", &vec![0, 10, 20, 30]), ("b2", &vec![2, 4, 5, 6]), ("c2", &vec![60, 70, 80, 90]), - ); + )?; let on: JoinOn = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?), Arc::new(Column::new_with_schema("b2", &right.schema())?), @@ -786,22 +801,22 @@ mod tests { ("a1", &vec![0, 1, 2]), ("b1", &vec![3, 4, 5]), ("c1", &vec![4, 5, 6]), - ); + )?; let left_batch_2 = build_table_i32( ("a1", &vec![3, 4, 5, 6]), ("b1", &vec![6, 6, 7, 9]), ("c1", &vec![7, 8, 9, 9]), - ); + )?; let right_batch_1 = build_table_i32( ("a2", &vec![0, 10, 20]), ("b2", &vec![2, 4, 6]), ("c2", &vec![50, 60, 70]), - ); + )?; let right_batch_2 = build_table_i32( ("a2", &vec![30, 40]), ("b2", &vec![6, 8]), ("c2", &vec![80, 90]), - ); + )?; let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); let on: JoinOn = vec![( @@ -837,22 +852,22 @@ mod tests { ("a2", &vec![0, 1, 2]), ("b2", &vec![3, 4, 5]), ("c2", &vec![4, 5, 6]), - ); + )?; let right_batch_2 = build_table_i32( ("a2", &vec![3, 4, 5, 6]), ("b2", &vec![6, 6, 7, 9]), ("c2", &vec![7, 8, 9, 9]), - ); + )?; let left_batch_1 = build_table_i32( ("a1", &vec![0, 10, 20]), ("b1", &vec![2, 4, 6]), ("c1", &vec![50, 60, 70]), - ); + )?; let left_batch_2 = build_table_i32( ("a1", &vec![30, 40]), ("b1", &vec![6, 8]), ("c1", &vec![80, 90]), - ); + )?; let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); let on: JoinOn = vec![( @@ -888,22 +903,22 @@ mod tests { ("a1", &vec![0, 1, 2]), ("b1", &vec![3, 4, 5]), ("c1", &vec![4, 5, 6]), - ); + )?; let left_batch_2 = build_table_i32( ("a1", &vec![3, 4, 5, 6]), ("b1", &vec![6, 6, 7, 9]), ("c1", &vec![7, 8, 9, 9]), - ); + )?; let right_batch_1 = build_table_i32( ("a2", &vec![0, 10, 20]), ("b2", &vec![2, 4, 6]), ("c2", &vec![50, 60, 70]), - ); + )?; let right_batch_2 = build_table_i32( ("a2", &vec![30, 40]), ("b2", &vec![6, 8]), ("c2", &vec![80, 90]), - ); + )?; let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); let on: JoinOn = vec![( @@ -941,22 +956,22 @@ mod tests { ("a1", &vec![0, 1, 2]), ("b1", &vec![3, 4, 5]), ("c1", &vec![4, 5, 6]), - ); + )?; let left_batch_2 = build_table_i32( ("a1", &vec![3, 4, 5, 6]), ("b1", &vec![6, 6, 7, 9]), ("c1", &vec![7, 8, 9, 9]), - ); + )?; let right_batch_1 = build_table_i32( ("a2", &vec![0, 10, 20]), ("b2", &vec![2, 4, 6]), ("c2", &vec![50, 60, 70]), - ); + )?; let right_batch_2 = build_table_i32( ("a2", &vec![30, 40]), ("b2", &vec![6, 8]), ("c2", &vec![80, 90]), - ); + )?; let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); let on: JoinOn = vec![( diff --git a/native-engine/datafusion-ext-plans/src/limit_exec.rs b/native-engine/datafusion-ext-plans/src/limit_exec.rs index 1b9f5892e..dd1e40d99 100644 --- a/native-engine/datafusion-ext-plans/src/limit_exec.rs +++ b/native-engine/datafusion-ext-plans/src/limit_exec.rs @@ -167,32 +167,36 @@ mod test { a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> RecordBatch { + ) -> Result { let schema = Schema::new(vec![ Field::new(a.0, DataType::Int32, false), Field::new(b.0, DataType::Int32, false), Field::new(c.0, DataType::Int32, false), ]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![ Arc::new(Int32Array::from(a.1.clone())), Arc::new(Int32Array::from(b.1.clone())), Arc::new(Int32Array::from(c.1.clone())), ], - ) - .unwrap() + )?; + Ok(batch) } fn build_table( a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> Arc { - let batch = build_table_i32(a, b, c); + ) -> Result> { + let batch = build_table_i32(a, b, c)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } #[tokio::test] @@ -202,11 +206,11 @@ mod test { ("a", &vec![9, 8, 7, 6, 5, 4, 3, 2, 1, 0]), ("b", &vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]), ("c", &vec![5, 6, 7, 8, 9, 0, 1, 2, 3, 4]), - ); + )?; let limit_exec = LimitExec::new(input, 2_u64); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let output = limit_exec.execute(0, task_ctx).unwrap(); + let output = limit_exec.execute(0, task_ctx)?; let batches = common::collect(output).await?; let row_count = limit_exec.statistics()?.num_rows; diff --git a/native-engine/datafusion-ext-plans/src/parquet_exec.rs b/native-engine/datafusion-ext-plans/src/parquet_exec.rs index c80b86b21..da272c805 100644 --- a/native-engine/datafusion-ext-plans/src/parquet_exec.rs +++ b/native-engine/datafusion-ext-plans/src/parquet_exec.rs @@ -332,7 +332,7 @@ impl AsyncFileReader for ParquetFileReaderRef { continue; } - let last_merged_range = merged_ranges.last_mut().unwrap(); + let last_merged_range = merged_ranges.last_mut().expect("missing last range"); if range.start <= last_merged_range.end + max_over_read_size as u64 { last_merged_range.end = range.end.max(last_merged_range.end); } else { diff --git a/native-engine/datafusion-ext-plans/src/parquet_sink_exec.rs b/native-engine/datafusion-ext-plans/src/parquet_sink_exec.rs index a6102516b..647e7eff3 100644 --- a/native-engine/datafusion-ext-plans/src/parquet_sink_exec.rs +++ b/native-engine/datafusion-ext-plans/src/parquet_sink_exec.rs @@ -291,7 +291,7 @@ fn execute_parquet_sink( tokio::task::spawn_blocking(move || { let mut part_writer = part_writer.lock(); - let w = part_writer.as_mut().unwrap(); + let w = part_writer.as_mut().expect("missing partition writer"); w.write(&sub_batch) }) .await diff --git a/native-engine/datafusion-ext-plans/src/shuffle/buffered_data.rs b/native-engine/datafusion-ext-plans/src/shuffle/buffered_data.rs index 6b88ba8fd..f29f6d217 100644 --- a/native-engine/datafusion-ext-plans/src/shuffle/buffered_data.rs +++ b/native-engine/datafusion-ext-plans/src/shuffle/buffered_data.rs @@ -311,7 +311,8 @@ fn sort_batches_by_partition_id( part_ids } Partitioning::RangePartitioning(sort_expr, _, bounds) => { - evaluate_range_partition_ids(&batch, sort_expr, bounds).unwrap() + evaluate_range_partition_ids(&batch, sort_expr, bounds) + .expect("failed to evaluate range partition ids") } _ => unreachable!("unsupported partitioning: {:?}", partitioning), }; @@ -373,22 +374,22 @@ mod test { a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> RecordBatch { + ) -> Result { let schema = Schema::new(vec![ Field::new(a.0, DataType::Int32, false), Field::new(b.0, DataType::Int32, false), Field::new(c.0, DataType::Int32, false), ]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![ Arc::new(Int32Array::from(a.1.clone())), Arc::new(Int32Array::from(b.1.clone())), Arc::new(Int32Array::from(c.1.clone())), ], - ) - .unwrap() + )?; + Ok(batch) } #[tokio::test] @@ -397,7 +398,7 @@ mod test { ("a", &vec![19, 18, 17, 16, 15, 14, 13, 12, 11, 10]), ("b", &vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]), ("c", &vec![5, 6, 7, 8, 9, 0, 1, 2, 3, 4]), - ); + )?; let round_robin_partitioning = Partitioning::RoundRobinPartitioning(4); let (_parts, sorted_batch) = @@ -429,7 +430,7 @@ mod test { ("a", &vec![19, 18, 17, 16, 15, 14, 13, 12, 11, 10]), ("b", &vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]), ("c", &vec![5, 6, 7, 8, 9, 0, 1, 2, 3, 4]), - ); + )?; let sort_exprs = vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -449,7 +450,7 @@ mod test { .collect::>>()?, )?)); - let rows: Rows = sort_row_converter.lock().convert_columns(&bounds).unwrap(); + let rows: Rows = sort_row_converter.lock().convert_columns(&bounds)?; let partition_num = rows.num_rows() + 1; let range_repartitioning = @@ -483,7 +484,7 @@ mod test { ("a", &vec![19, 18, 17, 16, 15, 14, 13, 12, 11, 10]), ("b", &vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]), ("c", &vec![5, 6, 7, 8, 9, 0, 1, 2, 3, 4]), - ); + )?; let sort_exprs = vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), @@ -511,7 +512,7 @@ mod test { .collect::>>()?, )?)); - let rows: Rows = sort_row_converter.lock().convert_columns(&bounds).unwrap(); + let rows: Rows = sort_row_converter.lock().convert_columns(&bounds)?; let partition_num = rows.num_rows() + 1; let range_repartitioning = diff --git a/native-engine/datafusion-ext-plans/src/shuffle/single_repartitioner.rs b/native-engine/datafusion-ext-plans/src/shuffle/single_repartitioner.rs index d5f4f31dc..aceb1d00b 100644 --- a/native-engine/datafusion-ext-plans/src/shuffle/single_repartitioner.rs +++ b/native-engine/datafusion-ext-plans/src/shuffle/single_repartitioner.rs @@ -57,7 +57,7 @@ impl SingleShuffleRepartitioner { .wrap_writer(open_shuffle_file(&self.output_data_file)?), )); } - Ok(output_data.as_mut().unwrap()) + Ok(output_data.as_mut().expect("missing output writer")) } } diff --git a/native-engine/datafusion-ext-plans/src/sort_exec.rs b/native-engine/datafusion-ext-plans/src/sort_exec.rs index 24d0beb6b..96d9ef376 100644 --- a/native-engine/datafusion-ext-plans/src/sort_exec.rs +++ b/native-engine/datafusion-ext-plans/src/sort_exec.rs @@ -460,7 +460,7 @@ impl SortedBlock for InMemSortedBlock { if let Some(batch) = batch { self.mem_used -= batch.get_batch_mem_size(); self.mem_used -= self.sorted_keys[0].mem_size(); - self.sorted_keys.pop_front().unwrap(); + self.sorted_keys.pop_front().expect("missing key"); self.cur_row_idx = usize::MAX; Ok(Some(batch)) } else { @@ -1049,7 +1049,7 @@ fn create_zero_column_batch(num_rows: usize) -> RecordBatch { vec![], &RecordBatchOptions::new().with_row_count(Some(num_rows)), ) - .unwrap() + .expect("failed to create empty RecordBatch") } struct PruneSortKeysFromBatch { @@ -1293,7 +1293,7 @@ impl KeyCollector for SqueezeKeyCollector { fn add_key(&mut self, key: &[u8]) { self.sorted_key_writer .write_key(key, &mut self.store) - .unwrap(); + .expect("failed to write key"); } fn freeze(&mut self) { @@ -1425,32 +1425,36 @@ mod test { a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> RecordBatch { + ) -> Result { let schema = Schema::new(vec![ Field::new(a.0, DataType::Int32, false), Field::new(b.0, DataType::Int32, false), Field::new(c.0, DataType::Int32, false), ]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![ Arc::new(Int32Array::from(a.1.clone())), Arc::new(Int32Array::from(b.1.clone())), Arc::new(Int32Array::from(c.1.clone())), ], - ) - .unwrap() + )?; + Ok(batch) } fn build_table( a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> Arc { - let batch = build_table_i32(a, b, c); + ) -> Result> { + let batch = build_table_i32(a, b, c)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } #[tokio::test] @@ -1462,7 +1466,7 @@ mod test { ("a", &vec![9, 8, 7, 6, 5, 4, 3, 2, 1, 0]), ("b", &vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]), ("c", &vec![5, 6, 7, 8, 9, 0, 1, 2, 3, 4]), - ); + )?; let sort_exprs = vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -1592,7 +1596,7 @@ mod fuzztest { None, )?); let sort = Arc::new(datafusion::physical_plan::sorts::sort::SortExec::new( - LexOrdering::new(sort_exprs.iter().cloned()).unwrap(), + LexOrdering::new(sort_exprs.iter().cloned()).expect("invalid sort exprs"), input, )); let output = datafusion::physical_plan::collect(sort.clone(), task_ctx.clone()).await?; diff --git a/native-engine/datafusion-ext-plans/src/window/window_context.rs b/native-engine/datafusion-ext-plans/src/window/window_context.rs index 182b462f8..a76eb1253 100644 --- a/native-engine/datafusion-ext-plans/src/window/window_context.rs +++ b/native-engine/datafusion-ext-plans/src/window/window_context.rs @@ -137,7 +137,7 @@ impl WindowContext { Ok(self .partition_row_converter .lock() - .unwrap() + .expect("partition_row_converter lock poisoned") .convert_columns( &self .partition_spec @@ -151,16 +151,20 @@ impl WindowContext { } pub fn get_order_rows(&self, batch: &RecordBatch) -> Result { - Ok(self.order_row_converter.lock().unwrap().convert_columns( - &self - .order_spec - .iter() - .map(|expr: &PhysicalSortExpr| { - expr.expr - .evaluate(batch) - .and_then(|v| v.into_array(batch.num_rows())) - }) - .collect::>>()?, - )?) + Ok(self + .order_row_converter + .lock() + .expect("order_row_converter lock poisoned") + .convert_columns( + &self + .order_spec + .iter() + .map(|expr: &PhysicalSortExpr| { + expr.expr + .evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) + .collect::>>()?, + )?) } } diff --git a/native-engine/datafusion-ext-plans/src/window_exec.rs b/native-engine/datafusion-ext-plans/src/window_exec.rs index 1ba6f5355..5bb698eec 100644 --- a/native-engine/datafusion-ext-plans/src/window_exec.rs +++ b/native-engine/datafusion-ext-plans/src/window_exec.rs @@ -272,6 +272,7 @@ mod test { use arrow::{array::*, datatypes::*, record_batch::RecordBatch}; use datafusion::{ assert_batches_eq, + common::Result, physical_expr::{PhysicalSortExpr, expressions::Column}, physical_plan::{ExecutionPlan, test::TestMemoryExec}, prelude::SessionContext, @@ -287,32 +288,36 @@ mod test { a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> RecordBatch { + ) -> Result { let schema = Schema::new(vec![ Field::new(a.0, DataType::Int32, false), Field::new(b.0, DataType::Int32, false), Field::new(c.0, DataType::Int32, false), ]); - RecordBatch::try_new( + let batch = RecordBatch::try_new( Arc::new(schema), vec![ Arc::new(Int32Array::from(a.1.clone())), Arc::new(Int32Array::from(b.1.clone())), Arc::new(Int32Array::from(c.1.clone())), ], - ) - .unwrap() + )?; + Ok(batch) } fn build_table( a: (&str, &Vec), b: (&str, &Vec), c: (&str, &Vec), - ) -> Arc { - let batch = build_table_i32(a, b, c); + ) -> Result> { + let batch = build_table_i32(a, b, c)?; let schema = batch.schema(); - Arc::new(TestMemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + Ok(Arc::new(TestMemoryExec::try_new( + &[vec![batch]], + schema, + None, + )?)) } #[tokio::test] @@ -325,7 +330,7 @@ mod test { ("a1", &vec![1, 1, 1, 1, 2, 3, 3]), ("b1", &vec![1, 2, 2, 3, 4, 1, 1]), ("c1", &vec![0, 0, 0, 0, 0, 0, 0]), - ); + )?; let window_exprs = vec![ WindowExpr::new( WindowFunction::RankLike(WindowRankType::RowNumber), @@ -385,7 +390,7 @@ mod test { ("a1", &vec![1, 3, 3, 1, 1, 1, 2]), ("b1", &vec![1, 1, 1, 2, 2, 3, 4]), ("c1", &vec![0, 0, 0, 0, 0, 0, 0]), - ); + )?; let window_exprs = vec![ WindowExpr::new( WindowFunction::RankLike(WindowRankType::RowNumber), @@ -452,7 +457,7 @@ mod test { ("a1", &vec![1, 1, 1, 1, 2, 3, 3]), ("b1", &vec![1, 2, 2, 3, 4, 1, 1]), ("c1", &vec![0, 0, 0, 0, 0, 0, 0]), - ); + )?; let window_exprs = vec![WindowExpr::new( WindowFunction::RankLike(WindowRankType::RowNumber), vec![],