diff --git a/rust/arrow/examples/read_csv.rs b/rust/arrow/examples/read_csv.rs index 8c8dfa074985b..9e2b9c34c86a1 100644 --- a/rust/arrow/examples/read_csv.rs +++ b/rust/arrow/examples/read_csv.rs @@ -22,11 +22,10 @@ use std::sync::Arc; use arrow::csv; use arrow::datatypes::{DataType, Field, Schema}; -use arrow::error::Result; #[cfg(feature = "prettyprint")] use arrow::util::pretty::print_batches; -fn main() -> Result<()> { +fn main() { let schema = Schema::new(vec![ Field::new("city", DataType::Utf8, false), Field::new("lat", DataType::Float64, false), @@ -41,5 +40,4 @@ fn main() -> Result<()> { { print_batches(&[_batch]).unwrap(); } - Ok(()) } diff --git a/rust/arrow/examples/read_csv_infer_schema.rs b/rust/arrow/examples/read_csv_infer_schema.rs index 467c855b92e45..93253e72cff23 100644 --- a/rust/arrow/examples/read_csv_infer_schema.rs +++ b/rust/arrow/examples/read_csv_infer_schema.rs @@ -18,12 +18,11 @@ extern crate arrow; use arrow::csv; -use arrow::error::Result; #[cfg(feature = "prettyprint")] use arrow::util::pretty::print_batches; use std::fs::File; -fn main() -> Result<()> { +fn main() { let file = File::open("test/data/uk_cities_with_headers.csv").unwrap(); let builder = csv::ReaderBuilder::new() .has_header(true) @@ -34,5 +33,4 @@ fn main() -> Result<()> { { print_batches(&[_batch]).unwrap(); } - Ok(()) } diff --git a/rust/arrow/src/array/builder.rs b/rust/arrow/src/array/builder.rs index ad5b39f76d1ff..e7519aacac354 100644 --- a/rust/arrow/src/array/builder.rs +++ b/rust/arrow/src/array/builder.rs @@ -1499,6 +1499,7 @@ impl FieldData { } /// Appends a single value to this `FieldData`'s `values_buffer`. + #[allow(clippy::unnecessary_wraps)] fn append_to_values_buffer( &mut self, v: T::Native, @@ -1521,6 +1522,7 @@ impl FieldData { } /// Appends a null to this `FieldData`. + #[allow(clippy::unnecessary_wraps)] fn append_null(&mut self) -> Result<()> { if let Some(b) = &mut self.bitmap_builder { let values_buffer = self diff --git a/rust/arrow/src/array/transform/mod.rs b/rust/arrow/src/array/transform/mod.rs index 23d9dfe6c400a..010a9058da200 100644 --- a/rust/arrow/src/array/transform/mod.rs +++ b/rust/arrow/src/array/transform/mod.rs @@ -1075,7 +1075,7 @@ mod tests { } #[test] - fn test_fixed_size_binary_append() -> Result<()> { + fn test_fixed_size_binary_append() { let a = vec![Some(vec![1, 2]), Some(vec![3, 4]), Some(vec![5, 6])]; let a = FixedSizeBinaryArray::from(a).data(); @@ -1118,7 +1118,6 @@ mod tests { ]; let expected = FixedSizeBinaryArray::from(expected).data(); assert_eq!(&result, expected.as_ref()); - Ok(()) } /* diff --git a/rust/arrow/src/buffer.rs b/rust/arrow/src/buffer.rs index 1f3ef5897d4ee..63ac5a7fe4c25 100644 --- a/rust/arrow/src/buffer.rs +++ b/rust/arrow/src/buffer.rs @@ -1619,7 +1619,7 @@ mod tests { } #[test] - fn test_mutable_equal() -> Result<()> { + fn test_mutable_equal() { let mut buf = MutableBuffer::new(1); let mut buf2 = MutableBuffer::new(1); @@ -1632,8 +1632,6 @@ mod tests { buf2.reserve(65); assert!(buf != buf2); - - Ok(()) } #[test] diff --git a/rust/arrow/src/compute/kernels/cast.rs b/rust/arrow/src/compute/kernels/cast.rs index 3358e68faa41c..7099f3025ecf8 100644 --- a/rust/arrow/src/compute/kernels/cast.rs +++ b/rust/arrow/src/compute/kernels/cast.rs @@ -821,6 +821,7 @@ const EPOCH_DAYS_FROM_CE: i32 = 719_163; /// Arrays should have the same primitive data type, otherwise this should fail. /// We do not perform this check on primitive data types as we only use this /// function internally, where it is guaranteed to be infallible. +#[allow(clippy::unnecessary_wraps)] fn cast_array_data(array: &ArrayRef, to_type: DataType) -> Result where TO: ArrowNumericType, @@ -838,6 +839,7 @@ where } /// Convert Array into a PrimitiveArray of type, and apply numeric cast +#[allow(clippy::unnecessary_wraps)] fn cast_numeric_arrays(from: &ArrayRef) -> Result where FROM: ArrowNumericType, @@ -869,6 +871,7 @@ where } /// Cast numeric types to Utf8 +#[allow(clippy::unnecessary_wraps)] fn cast_numeric_to_string(array: &ArrayRef) -> Result where FROM: ArrowNumericType, @@ -893,6 +896,7 @@ where } /// Cast numeric types to Utf8 +#[allow(clippy::unnecessary_wraps)] fn cast_string_to_numeric(from: &ArrayRef) -> Result where T: ArrowNumericType, @@ -959,6 +963,7 @@ where /// Cast Boolean types to numeric /// /// `false` returns 0 while `true` returns 1 +#[allow(clippy::unnecessary_wraps)] fn cast_bool_to_numeric(from: &ArrayRef) -> Result where TO: ArrowNumericType, diff --git a/rust/arrow/src/compute/kernels/concat.rs b/rust/arrow/src/compute/kernels/concat.rs index 5df0bd53b1c98..9fde22cc93036 100644 --- a/rust/arrow/src/compute/kernels/concat.rs +++ b/rust/arrow/src/compute/kernels/concat.rs @@ -77,20 +77,18 @@ mod tests { use std::sync::Arc; #[test] - fn test_concat_empty_vec() -> Result<()> { + fn test_concat_empty_vec() { let re = concat(&[]); assert!(re.is_err()); - Ok(()) } #[test] - fn test_concat_incompatible_datatypes() -> Result<()> { + fn test_concat_incompatible_datatypes() { let re = concat(&[ &PrimitiveArray::::from(vec![Some(-1), Some(2), None]), &StringArray::from(vec![Some("hello"), Some("bar"), Some("world")]), ]); assert!(re.is_err()); - Ok(()) } #[test] diff --git a/rust/arrow/src/compute/kernels/length.rs b/rust/arrow/src/compute/kernels/length.rs index 5f2ed59e2629d..740bb2b68c8a0 100644 --- a/rust/arrow/src/compute/kernels/length.rs +++ b/rust/arrow/src/compute/kernels/length.rs @@ -24,6 +24,7 @@ use crate::{ }; use std::sync::Arc; +#[allow(clippy::unnecessary_wraps)] fn length_string(array: &Array, data_type: DataType) -> Result where OffsetSize: OffsetSizeTrait, @@ -178,11 +179,10 @@ mod tests { /// Tests that length is not valid for u64. #[test] - fn wrong_type() -> Result<()> { + fn wrong_type() { let array: UInt64Array = vec![1u64].into(); assert!(length(&array).is_err()); - Ok(()) } /// Tests with an offset diff --git a/rust/arrow/src/compute/kernels/sort.rs b/rust/arrow/src/compute/kernels/sort.rs index 70efef4cf5d4c..e33b76ed0a1ec 100644 --- a/rust/arrow/src/compute/kernels/sort.rs +++ b/rust/arrow/src/compute/kernels/sort.rs @@ -254,6 +254,7 @@ impl Default for SortOptions { } /// Sort primitive values +#[allow(clippy::unnecessary_wraps)] fn sort_boolean( values: &ArrayRef, value_indices: Vec, @@ -316,6 +317,7 @@ fn sort_boolean( } /// Sort primitive values +#[allow(clippy::unnecessary_wraps)] fn sort_primitive( values: &ArrayRef, value_indices: Vec, @@ -446,6 +448,7 @@ fn sort_string_dictionary( /// shared implementation between dictionary encoded and plain string arrays #[inline] +#[allow(clippy::unnecessary_wraps)] fn sort_string_helper<'a, A: Array, F>( values: &'a A, value_indices: Vec, @@ -481,6 +484,7 @@ where Ok(UInt32Array::from(valid_indices)) } +#[allow(clippy::unnecessary_wraps)] fn sort_list( values: &ArrayRef, value_indices: Vec, diff --git a/rust/arrow/src/compute/kernels/substring.rs b/rust/arrow/src/compute/kernels/substring.rs index 2ad9bc3c26273..c6f5cc710e768 100644 --- a/rust/arrow/src/compute/kernels/substring.rs +++ b/rust/arrow/src/compute/kernels/substring.rs @@ -24,6 +24,7 @@ use crate::{ }; use std::sync::Arc; +#[allow(clippy::unnecessary_wraps)] fn generic_substring( array: &GenericStringArray, start: OffsetSize, diff --git a/rust/arrow/src/compute/util.rs b/rust/arrow/src/compute/util.rs index cbe9ef43bb1c6..0e6de913dafb7 100644 --- a/rust/arrow/src/compute/util.rs +++ b/rust/arrow/src/compute/util.rs @@ -27,6 +27,7 @@ use std::ops::Add; /// Combines the null bitmaps of two arrays using a bitwise `and` operation. /// /// This function is useful when implementing operations on higher level arrays. +#[allow(clippy::unnecessary_wraps)] pub(super) fn combine_option_bitmap( left_data: &ArrayDataRef, right_data: &ArrayDataRef, @@ -60,6 +61,7 @@ pub(super) fn combine_option_bitmap( /// Compares the null bitmaps of two arrays using a bitwise `or` operation. /// /// This function is useful when implementing operations on higher level arrays. +#[allow(clippy::unnecessary_wraps)] pub(super) fn compare_option_bitmap( left_data: &ArrayDataRef, right_data: &ArrayDataRef, diff --git a/rust/arrow/src/csv/reader.rs b/rust/arrow/src/csv/reader.rs index 7effe71086ae4..9ad3691d4fc6c 100644 --- a/rust/arrow/src/csv/reader.rs +++ b/rust/arrow/src/csv/reader.rs @@ -1153,7 +1153,7 @@ mod tests { } #[test] - fn test_bounded() -> Result<()> { + fn test_bounded() { let schema = Schema::new(vec![Field::new("int", DataType::UInt32, false)]); let data = vec![ vec!["0"], @@ -1196,7 +1196,6 @@ mod tests { assert_eq!(a, &UInt32Array::from(vec![4, 5])); assert!(csv.next().is_none()); - Ok(()) } #[test] diff --git a/rust/arrow/src/json/reader.rs b/rust/arrow/src/json/reader.rs index a26cf84379d13..af9a585fd7925 100644 --- a/rust/arrow/src/json/reader.rs +++ b/rust/arrow/src/json/reader.rs @@ -764,6 +764,7 @@ impl Decoder { } #[inline(always)] + #[allow(clippy::unnecessary_wraps)] fn build_string_dictionary_builder( &self, row_len: usize, @@ -835,6 +836,7 @@ impl Decoder { Ok(Arc::new(builder.finish())) } + #[allow(clippy::unnecessary_wraps)] fn build_primitive_array( &self, rows: &[Value], diff --git a/rust/datafusion/src/execution/context.rs b/rust/datafusion/src/execution/context.rs index 4b8a4f71fe624..a6fd020fa0bfc 100644 --- a/rust/datafusion/src/execution/context.rs +++ b/rust/datafusion/src/execution/context.rs @@ -172,7 +172,7 @@ impl ExecutionContext { // create a query planner let state = self.state.lock().unwrap().clone(); let query_planner = SqlToRel::new(&state); - Ok(query_planner.statement_to_plan(&statements[0])?) + query_planner.statement_to_plan(&statements[0]) } /// Registers a variable provider within this context. diff --git a/rust/datafusion/src/logical_plan/builder.rs b/rust/datafusion/src/logical_plan/builder.rs index 9f5d781d978f0..a573b3f09749e 100644 --- a/rust/datafusion/src/logical_plan/builder.rs +++ b/rust/datafusion/src/logical_plan/builder.rs @@ -469,7 +469,7 @@ mod tests { } #[test] - fn stringified_plan() -> Result<()> { + fn stringified_plan() { let stringified_plan = StringifiedPlan::new(PlanType::LogicalPlan, "...the plan..."); assert!(stringified_plan.should_display(true)); @@ -488,7 +488,5 @@ mod tests { ); assert!(stringified_plan.should_display(true)); assert!(!stringified_plan.should_display(false)); - - Ok(()) } } diff --git a/rust/datafusion/src/logical_plan/dfschema.rs b/rust/datafusion/src/logical_plan/dfschema.rs index 0305f07acdc23..9adb22b43d075 100644 --- a/rust/datafusion/src/logical_plan/dfschema.rs +++ b/rust/datafusion/src/logical_plan/dfschema.rs @@ -237,9 +237,11 @@ where Self: Sized, { /// Attempt to create a DSSchema + #[allow(clippy::wrong_self_convention)] fn to_dfschema(self) -> Result; /// Attempt to create a DSSchemaRef + #[allow(clippy::wrong_self_convention)] fn to_dfschema_ref(self) -> Result { Ok(Arc::new(self.to_dfschema()?)) } diff --git a/rust/datafusion/src/logical_plan/expr.rs b/rust/datafusion/src/logical_plan/expr.rs index 63915381a7bb6..7f358cb31b0c4 100644 --- a/rust/datafusion/src/logical_plan/expr.rs +++ b/rust/datafusion/src/logical_plan/expr.rs @@ -1137,11 +1137,10 @@ mod tests { } #[test] - fn case_when_different_literal_then_types() -> Result<()> { + fn case_when_different_literal_then_types() { let maybe_expr = when(col("state").eq(lit("CO")), lit(303)) .when(col("state").eq(lit("NY")), lit("212")) .end(); assert!(maybe_expr.is_err()); - Ok(()) } } diff --git a/rust/datafusion/src/logical_plan/operators.rs b/rust/datafusion/src/logical_plan/operators.rs index dac48d40b4809..624635e6d9ae2 100644 --- a/rust/datafusion/src/logical_plan/operators.rs +++ b/rust/datafusion/src/logical_plan/operators.rs @@ -111,11 +111,10 @@ impl ops::Div for Expr { #[cfg(test)] mod tests { - use crate::error::Result; use crate::prelude::lit; #[test] - fn test_operators() -> Result<()> { + fn test_operators() { assert_eq!( format!("{:?}", lit(1u32) + lit(2u32)), "UInt32(1) Plus UInt32(2)" @@ -132,7 +131,5 @@ mod tests { format!("{:?}", lit(1u32) / lit(2u32)), "UInt32(1) Divide UInt32(2)" ); - - Ok(()) } } diff --git a/rust/datafusion/src/optimizer/hash_build_probe_order.rs b/rust/datafusion/src/optimizer/hash_build_probe_order.rs index 5d63926a4ab02..e6ad905e73d15 100644 --- a/rust/datafusion/src/optimizer/hash_build_probe_order.rs +++ b/rust/datafusion/src/optimizer/hash_build_probe_order.rs @@ -226,7 +226,7 @@ mod tests { } #[test] - fn test_swap_order() -> Result<()> { + fn test_swap_order() { let lp_left = LogicalPlan::TableScan { table_name: "left".to_string(), projection: None, @@ -245,7 +245,5 @@ mod tests { assert!(should_swap_join_order(&lp_left, &lp_right)); assert!(!should_swap_join_order(&lp_right, &lp_left)); - - Ok(()) } } diff --git a/rust/datafusion/src/physical_plan/aggregates.rs b/rust/datafusion/src/physical_plan/aggregates.rs index 9ea5c17e3d4f8..59aa730fdea63 100644 --- a/rust/datafusion/src/physical_plan/aggregates.rs +++ b/rust/datafusion/src/physical_plan/aggregates.rs @@ -205,10 +205,9 @@ mod tests { } #[test] - fn test_sum_no_utf8() -> Result<()> { + fn test_sum_no_utf8() { let observed = return_type(&AggregateFunction::Sum, &[DataType::Utf8]); assert!(observed.is_err()); - Ok(()) } #[test] @@ -239,9 +238,8 @@ mod tests { } #[test] - fn test_avg_no_utf8() -> Result<()> { + fn test_avg_no_utf8() { let observed = return_type(&AggregateFunction::Avg, &[DataType::Utf8]); assert!(observed.is_err()); - Ok(()) } } diff --git a/rust/datafusion/src/physical_plan/coalesce_batches.rs b/rust/datafusion/src/physical_plan/coalesce_batches.rs index 472b10a6a82a6..9f36fd8f794a7 100644 --- a/rust/datafusion/src/physical_plan/coalesce_batches.rs +++ b/rust/datafusion/src/physical_plan/coalesce_batches.rs @@ -241,7 +241,7 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn test_concat_batches() -> Result<()> { let schema = test_schema(); - let partition = create_vec_batches(&schema, 10)?; + let partition = create_vec_batches(&schema, 10); let partitions = vec![partition]; let output_partitions = coalesce_batches(&schema, partitions, 20).await?; @@ -263,16 +263,13 @@ mod tests { Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) } - fn create_vec_batches( - schema: &Arc, - num_batches: usize, - ) -> Result> { + fn create_vec_batches(schema: &Arc, num_batches: usize) -> Vec { let batch = create_batch(schema); let mut vec = Vec::with_capacity(num_batches); for _ in 0..num_batches { vec.push(batch.clone()); } - Ok(vec) + vec } fn create_batch(schema: &Arc) -> RecordBatch { diff --git a/rust/datafusion/src/physical_plan/datetime_expressions.rs b/rust/datafusion/src/physical_plan/datetime_expressions.rs index 247f899f50ecf..a70ebcc4fef55 100644 --- a/rust/datafusion/src/physical_plan/datetime_expressions.rs +++ b/rust/datafusion/src/physical_plan/datetime_expressions.rs @@ -420,7 +420,7 @@ mod tests { } #[test] - fn string_to_timestamp_invalid() -> Result<()> { + fn string_to_timestamp_invalid() { // Test parsing invalid formats // It would be nice to make these messages better @@ -430,8 +430,6 @@ mod tests { "Wed, 18 Feb 2015 23:16:09 GMT", "Error parsing 'Wed, 18 Feb 2015 23:16:09 GMT' as timestamp", ); - - Ok(()) } // Parse a timestamp to timestamp int with a useful human readable error message diff --git a/rust/datafusion/src/physical_plan/expressions/cast.rs b/rust/datafusion/src/physical_plan/expressions/cast.rs index ed5cd27aff4dc..bcf54dc14bcac 100644 --- a/rust/datafusion/src/physical_plan/expressions/cast.rs +++ b/rust/datafusion/src/physical_plan/expressions/cast.rs @@ -214,11 +214,10 @@ mod tests { } #[test] - fn invalid_cast() -> Result<()> { + fn invalid_cast() { // Ensure a useful error happens at plan time if invalid casts are used let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let result = cast(col("a"), &schema, DataType::LargeBinary); result.expect_err("expected Invalid CAST"); - Ok(()) } } diff --git a/rust/datafusion/src/physical_plan/expressions/coercion.rs b/rust/datafusion/src/physical_plan/expressions/coercion.rs index 62c30fc1274d0..e9949f5199e88 100644 --- a/rust/datafusion/src/physical_plan/expressions/coercion.rs +++ b/rust/datafusion/src/physical_plan/expressions/coercion.rs @@ -183,10 +183,9 @@ pub fn order_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Result<()> { + fn test_dictionary_type_coersion() { use DataType::*; // TODO: In the future, this would ideally return Dictionary types and avoid unpacking @@ -205,7 +204,5 @@ mod tests { let lhs_type = Utf8; let rhs_type = Dictionary(Box::new(Int8), Box::new(Utf8)); assert_eq!(dictionary_coercion(&lhs_type, &rhs_type), Some(Utf8)); - - Ok(()) } } diff --git a/rust/datafusion/src/physical_plan/expressions/in_list.rs b/rust/datafusion/src/physical_plan/expressions/in_list.rs index 311a8845ff6ff..41f111006ea2a 100644 --- a/rust/datafusion/src/physical_plan/expressions/in_list.rs +++ b/rust/datafusion/src/physical_plan/expressions/in_list.rs @@ -129,6 +129,7 @@ impl InListExpr { } /// Compare for specific utf8 types + #[allow(clippy::unnecessary_wraps)] fn compare_utf8( &self, array: ArrayRef, diff --git a/rust/datafusion/src/physical_plan/expressions/not.rs b/rust/datafusion/src/physical_plan/expressions/not.rs index 07502b5419cc5..23a1a46651dee 100644 --- a/rust/datafusion/src/physical_plan/expressions/not.rs +++ b/rust/datafusion/src/physical_plan/expressions/not.rs @@ -149,12 +149,10 @@ mod tests { /// verify that expression errors when the input expression is not a boolean. #[test] - fn neg_op_not_null() -> Result<()> { + fn neg_op_not_null() { let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); let expr = not(col("a"), &schema); assert!(expr.is_err()); - - Ok(()) } } diff --git a/rust/datafusion/src/physical_plan/functions.rs b/rust/datafusion/src/physical_plan/functions.rs index c9c2cde2a9e7a..ca597c9e6ae63 100644 --- a/rust/datafusion/src/physical_plan/functions.rs +++ b/rust/datafusion/src/physical_plan/functions.rs @@ -474,7 +474,7 @@ pub fn create_physical_expr( BuiltinScalarFunction::DateTrunc => { |args| Ok(Arc::new(datetime_expressions::date_trunc(args)?)) } - BuiltinScalarFunction::Array => |args| Ok(array_expressions::array(args)?), + BuiltinScalarFunction::Array => |args| array_expressions::array(args), }); // coerce let args = coerce(args, input_schema, &signature(fun))?; diff --git a/rust/datafusion/src/physical_plan/group_scalar.rs b/rust/datafusion/src/physical_plan/group_scalar.rs index 51e7c4b807b30..f73658dc219cf 100644 --- a/rust/datafusion/src/physical_plan/group_scalar.rs +++ b/rust/datafusion/src/physical_plan/group_scalar.rs @@ -117,7 +117,7 @@ impl From<&GroupByScalar> for ScalarValue { mod tests { use super::*; - use crate::error::{DataFusionError, Result}; + use crate::error::DataFusionError; macro_rules! scalar_eq_test { ($TYPE:expr, $VALUE:expr) => {{ @@ -132,12 +132,10 @@ mod tests { } #[test] - fn test_scalar_ne_non_std() -> Result<()> { + fn test_scalar_ne_non_std() { // Test only Scalars with non native Eq, Hash scalar_eq_test!(ScalarValue::Float32, Some(1.0)); scalar_eq_test!(ScalarValue::Float64, Some(1.0)); - - Ok(()) } macro_rules! scalar_ne_test { @@ -153,16 +151,14 @@ mod tests { } #[test] - fn test_scalar_eq_non_std() -> Result<()> { + fn test_scalar_eq_non_std() { // Test only Scalars with non native Eq, Hash scalar_ne_test!(ScalarValue::Float32, Some(1.0), Some(2.0)); scalar_ne_test!(ScalarValue::Float64, Some(1.0), Some(2.0)); - - Ok(()) } #[test] - fn from_scalar_holding_none() -> Result<()> { + fn from_scalar_holding_none() { let scalar_value = ScalarValue::Int8(None); let result = GroupByScalar::try_from(&scalar_value); @@ -173,12 +169,10 @@ mod tests { ), _ => panic!("Unexpected result"), } - - Ok(()) } #[test] - fn from_scalar_unsupported() -> Result<()> { + fn from_scalar_unsupported() { // Use any ScalarValue type not supported by GroupByScalar. let scalar_value = ScalarValue::LargeUtf8(Some("1.1".to_string())); let result = GroupByScalar::try_from(&scalar_value); @@ -192,8 +186,6 @@ mod tests { ), _ => panic!("Unexpected result"), } - - Ok(()) } #[test] diff --git a/rust/datafusion/src/physical_plan/planner.rs b/rust/datafusion/src/physical_plan/planner.rs index 0f0f53d2e5a44..93bb94327a4a9 100644 --- a/rust/datafusion/src/physical_plan/planner.rs +++ b/rust/datafusion/src/physical_plan/planner.rs @@ -866,7 +866,7 @@ mod tests { } #[test] - fn default_extension_planner() -> Result<()> { + fn default_extension_planner() { let ctx_state = make_ctx_state(); let planner = DefaultPhysicalPlanner::default(); let logical_plan = LogicalPlan::Extension { @@ -884,11 +884,10 @@ mod tests { expected_error ), } - Ok(()) } #[test] - fn bad_extension_planner() -> Result<()> { + fn bad_extension_planner() { // Test that creating an execution plan whose schema doesn't // match the logical plan's schema generates an error. let ctx_state = make_ctx_state(); @@ -930,7 +929,6 @@ mod tests { expected_error ), } - Ok(()) } #[test] diff --git a/rust/datafusion/src/physical_plan/repartition.rs b/rust/datafusion/src/physical_plan/repartition.rs index deb78b2a47fc1..20e7122de1268 100644 --- a/rust/datafusion/src/physical_plan/repartition.rs +++ b/rust/datafusion/src/physical_plan/repartition.rs @@ -247,7 +247,7 @@ mod tests { async fn one_to_many_round_robin() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50)?; + let partition = create_vec_batches(&schema, 50); let partitions = vec![partition]; // repartition from 1 input to 4 output @@ -267,7 +267,7 @@ mod tests { async fn many_to_one_round_robin() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50)?; + let partition = create_vec_batches(&schema, 50); let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; // repartition from 3 input to 1 output @@ -284,7 +284,7 @@ mod tests { async fn many_to_many_round_robin() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50)?; + let partition = create_vec_batches(&schema, 50); let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; // repartition from 3 input to 5 output @@ -305,13 +305,13 @@ mod tests { Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) } - fn create_vec_batches(schema: &Arc, n: usize) -> Result> { + fn create_vec_batches(schema: &Arc, n: usize) -> Vec { let batch = create_batch(schema); let mut vec = Vec::with_capacity(n); for _ in 0..n { vec.push(batch.clone()); } - Ok(vec) + vec } fn create_batch(schema: &Arc) -> RecordBatch { diff --git a/rust/datafusion/src/physical_plan/type_coercion.rs b/rust/datafusion/src/physical_plan/type_coercion.rs index ebc6bf79c402a..a84707a48dfa5 100644 --- a/rust/datafusion/src/physical_plan/type_coercion.rs +++ b/rust/datafusion/src/physical_plan/type_coercion.rs @@ -189,7 +189,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; #[test] - fn test_maybe_data_types() -> Result<()> { + fn test_maybe_data_types() { // this vec contains: arg1, arg2, expected result let cases = vec![ // 2 entries, same values @@ -223,7 +223,6 @@ mod tests { for case in cases { assert_eq!(maybe_data_types(&case.0, &case.1), case.2) } - Ok(()) } #[test] diff --git a/rust/datafusion/src/scalar.rs b/rust/datafusion/src/scalar.rs index 35b15907fdefc..31ca1da8edf6e 100644 --- a/rust/datafusion/src/scalar.rs +++ b/rust/datafusion/src/scalar.rs @@ -624,19 +624,17 @@ mod tests { use super::*; #[test] - fn scalar_list_null_to_array() -> Result<()> { + fn scalar_list_null_to_array() { let list_array_ref = ScalarValue::List(None, DataType::UInt64).to_array(); let list_array = list_array_ref.as_any().downcast_ref::().unwrap(); assert!(list_array.is_null(0)); assert_eq!(list_array.len(), 1); assert_eq!(list_array.values().len(), 0); - - Ok(()) } #[test] - fn scalar_list_to_array() -> Result<()> { + fn scalar_list_to_array() { let list_array_ref = ScalarValue::List( Some(vec![ ScalarValue::UInt64(Some(100)), @@ -660,7 +658,5 @@ mod tests { assert_eq!(prim_array.value(0), 100); assert!(prim_array.is_null(1)); assert_eq!(prim_array.value(2), 101); - - Ok(()) } } diff --git a/rust/datafusion/src/sql/parser.rs b/rust/datafusion/src/sql/parser.rs index dc31581a0e6e1..3637e882d2fd2 100644 --- a/rust/datafusion/src/sql/parser.rs +++ b/rust/datafusion/src/sql/parser.rs @@ -311,7 +311,7 @@ mod tests { } /// Parses sql and asserts that the expected error message was found - fn expect_parse_error(sql: &str, expected_error: &str) -> Result<(), ParserError> { + fn expect_parse_error(sql: &str, expected_error: &str) { match DFParser::parse_sql(sql) { Ok(statements) => { panic!( @@ -329,7 +329,6 @@ mod tests { ); } } - Ok(()) } fn make_column_def(name: impl Into, data_type: DataType) -> ColumnDef { @@ -374,7 +373,7 @@ mod tests { expect_parse_error( sql, "Expected one of PARQUET, NDJSON, or CSV, found: UNKNOWN_TYPE", - )?; + ); Ok(()) } diff --git a/rust/datafusion/tests/sql.rs b/rust/datafusion/tests/sql.rs index 653d1baa978a7..8c1850e1df062 100644 --- a/rust/datafusion/tests/sql.rs +++ b/rust/datafusion/tests/sql.rs @@ -554,6 +554,7 @@ async fn csv_query_sqrt_sqrt() -> Result<()> { Ok(()) } +#[allow(clippy::unnecessary_wraps)] fn create_ctx() -> Result { let mut ctx = ExecutionContext::new(); diff --git a/rust/datafusion/tests/user_defined_plan.rs b/rust/datafusion/tests/user_defined_plan.rs index 817aadad8c615..4f43c46c71793 100644 --- a/rust/datafusion/tests/user_defined_plan.rs +++ b/rust/datafusion/tests/user_defined_plan.rs @@ -427,6 +427,7 @@ fn remove_lowest_value(top_values: &mut BTreeMap) { } } +#[allow(clippy::unnecessary_wraps)] fn accumulate_batch( input_batch: &RecordBatch, mut top_values: BTreeMap, diff --git a/rust/integration-testing/src/flight_client_scenarios/middleware.rs b/rust/integration-testing/src/flight_client_scenarios/middleware.rs index 607eab1018a55..cbca879dca514 100644 --- a/rust/integration-testing/src/flight_client_scenarios/middleware.rs +++ b/rust/integration-testing/src/flight_client_scenarios/middleware.rs @@ -75,6 +75,7 @@ pub async fn run_scenario(host: &str, port: &str) -> Result { Ok(()) } +#[allow(clippy::unnecessary_wraps)] fn middleware_interceptor(mut req: Request<()>) -> Result, Status> { let metadata = req.metadata_mut(); metadata.insert("x-middleware", "expected value".parse().unwrap()); diff --git a/rust/parquet/src/arrow/arrow_reader.rs b/rust/parquet/src/arrow/arrow_reader.rs index 1559c97e4cfad..288e043b64291 100644 --- a/rust/parquet/src/arrow/arrow_reader.rs +++ b/rust/parquet/src/arrow/arrow_reader.rs @@ -147,7 +147,7 @@ impl ArrowReader for ParquetFileArrowReader { self.file_reader.clone(), )?; - Ok(ParquetRecordBatchReader::try_new(batch_size, array_reader)?) + ParquetRecordBatchReader::try_new(batch_size, array_reader) } } diff --git a/rust/parquet/src/arrow/record_reader.rs b/rust/parquet/src/arrow/record_reader.rs index 8591911922199..d58d563621f2f 100644 --- a/rust/parquet/src/arrow/record_reader.rs +++ b/rust/parquet/src/arrow/record_reader.rs @@ -369,6 +369,7 @@ impl RecordReader { /// Split values into records according repetition definition and returns number of /// records read. + #[allow(clippy::unnecessary_wraps)] fn split_records(&mut self, records_to_read: usize) -> Result { let rep_levels = self.rep_levels.as_ref().map(|buf| { let (prefix, rep_levels, suffix) = @@ -410,6 +411,7 @@ impl RecordReader { } } + #[allow(clippy::unnecessary_wraps)] fn set_values_written(&mut self, new_values_written: usize) -> Result<()> { self.values_written = new_values_written; self.records diff --git a/rust/parquet/src/compression.rs b/rust/parquet/src/compression.rs index 67cb275e2e18e..16fe487189f5c 100644 --- a/rust/parquet/src/compression.rs +++ b/rust/parquet/src/compression.rs @@ -208,7 +208,7 @@ mod brotli_codec { BROTLI_DEFAULT_COMPRESSION_QUALITY, BROTLI_DEFAULT_LG_WINDOW_SIZE, ); - encoder.write_all(&input_buf[..])?; + encoder.write_all(input_buf)?; encoder.flush().map_err(|e| e.into()) } } @@ -308,7 +308,7 @@ mod zstd_codec { fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { let mut encoder = zstd::Encoder::new(output_buf, ZSTD_COMPRESSION_LEVEL)?; - encoder.write_all(&input_buf[..])?; + encoder.write_all(input_buf)?; match encoder.finish() { Ok(_) => Ok(()), Err(e) => Err(e.into()), diff --git a/rust/parquet/src/data_type.rs b/rust/parquet/src/data_type.rs index 92afc8a2e9857..ecce77a8910dd 100644 --- a/rust/parquet/src/data_type.rs +++ b/rust/parquet/src/data_type.rs @@ -460,6 +460,7 @@ impl AsBytes for [u8] { macro_rules! gen_as_bytes { ($source_ty:ident) => { impl AsBytes for $source_ty { + #[allow(clippy::size_of_in_element_count)] fn as_bytes(&self) -> &[u8] { unsafe { std::slice::from_raw_parts( @@ -472,6 +473,7 @@ macro_rules! gen_as_bytes { impl SliceAsBytes for $source_ty { #[inline] + #[allow(clippy::size_of_in_element_count)] fn slice_as_bytes(self_: &[Self]) -> &[u8] { unsafe { std::slice::from_raw_parts( @@ -482,6 +484,7 @@ macro_rules! gen_as_bytes { } #[inline] + #[allow(clippy::size_of_in_element_count)] unsafe fn slice_as_bytes_mut(self_: &mut [Self]) -> &mut [u8] { std::slice::from_raw_parts_mut( self_.as_mut_ptr() as *mut u8, diff --git a/rust/parquet/src/encodings/decoding.rs b/rust/parquet/src/encodings/decoding.rs index 02f0a1c9c5938..ee7ad5ae95c4d 100644 --- a/rust/parquet/src/encodings/decoding.rs +++ b/rust/parquet/src/encodings/decoding.rs @@ -995,7 +995,7 @@ mod tests { let mut decoder: PlainDecoder = PlainDecoder::new(type_length); let result = decoder.set_data(data, num_values); assert!(result.is_ok()); - let result = decoder.get(&mut buffer[..]); + let result = decoder.get(buffer); assert!(result.is_ok()); assert_eq!(decoder.values_left(), 0); assert_eq!(buffer, expected); @@ -1013,7 +1013,7 @@ mod tests { let mut decoder: PlainDecoder = PlainDecoder::new(type_length); let result = decoder.set_data(data, num_values); assert!(result.is_ok()); - let result = decoder.get_spaced(&mut buffer[..], num_nulls, valid_bits); + let result = decoder.get_spaced(buffer, num_nulls, valid_bits); assert!(result.is_ok()); assert_eq!(num_values + num_nulls, result.unwrap()); assert_eq!(decoder.values_left(), 0); @@ -1317,6 +1317,7 @@ mod tests { /// A util trait to convert slices of different types to byte arrays trait ToByteArray { + #[allow(clippy::wrong_self_convention)] fn to_byte_array(data: &[T::T]) -> Vec; } diff --git a/rust/parquet/src/encodings/encoding.rs b/rust/parquet/src/encodings/encoding.rs index d429e27843361..fdd616e9e27ed 100644 --- a/rust/parquet/src/encodings/encoding.rs +++ b/rust/parquet/src/encodings/encoding.rs @@ -270,6 +270,7 @@ impl DictEncoder { } #[inline] + #[allow(clippy::unnecessary_wraps)] fn put_one(&mut self, value: &T::T) -> Result<()> { let mut j = (hash_util::hash(value, 0) & self.mod_bitmask) as usize; let mut index = self.hash_slots[j]; diff --git a/rust/parquet/src/file/serialized_reader.rs b/rust/parquet/src/file/serialized_reader.rs index 9fbff41918605..b0d1d0c7b314c 100644 --- a/rust/parquet/src/file/serialized_reader.rs +++ b/rust/parquet/src/file/serialized_reader.rs @@ -451,7 +451,7 @@ mod tests { } #[test] - fn test_file_reader_into_iter() -> Result<()> { + fn test_file_reader_into_iter() { let path = get_test_path("alltypes_plain.parquet"); let vec = vec![path.clone(), path] .iter() @@ -463,12 +463,10 @@ mod tests { // rows in the parquet file are not sorted by "id" // each file contains [id:4, id:5, id:6, id:7, id:2, id:3, id:0, id:1] assert_eq!(vec, vec![4, 5, 6, 7, 2, 3, 0, 1, 4, 5, 6, 7, 2, 3, 0, 1]); - - Ok(()) } #[test] - fn test_file_reader_into_iter_project() -> Result<()> { + fn test_file_reader_into_iter_project() { let path = get_test_path("alltypes_plain.parquet"); let result = vec![path] .iter() @@ -487,8 +485,6 @@ mod tests { result, "{id: 4},{id: 5},{id: 6},{id: 7},{id: 2},{id: 3},{id: 0},{id: 1}" ); - - Ok(()) } #[test] diff --git a/rust/parquet/src/record/reader.rs b/rust/parquet/src/record/reader.rs index 0b02bc8ed46af..5f42d37bac0f4 100644 --- a/rust/parquet/src/record/reader.rs +++ b/rust/parquet/src/record/reader.rs @@ -1522,7 +1522,7 @@ mod tests { } #[test] - fn test_file_reader_iter() -> Result<()> { + fn test_file_reader_iter() { let path = get_test_path("alltypes_plain.parquet"); let vec = vec![path] .iter() @@ -1532,12 +1532,10 @@ mod tests { .collect::>(); assert_eq!(vec, vec![4, 5, 6, 7, 2, 3, 0, 1]); - - Ok(()) } #[test] - fn test_file_reader_iter_projection() -> Result<()> { + fn test_file_reader_iter_projection() { let path = get_test_path("alltypes_plain.parquet"); let values = vec![path] .iter() @@ -1553,8 +1551,6 @@ mod tests { .join(", "); assert_eq!(values, "id:4, id:5, id:6, id:7, id:2, id:3, id:0, id:1"); - - Ok(()) } #[test] diff --git a/rust/parquet/src/schema/parser.rs b/rust/parquet/src/schema/parser.rs index 66f7b72e659d0..9f14a5502411d 100644 --- a/rust/parquet/src/schema/parser.rs +++ b/rust/parquet/src/schema/parser.rs @@ -345,7 +345,7 @@ impl<'a> Parser<'a> { if let Some(id) = id { builder = builder.with_id(id); } - Ok(builder.build()?) + builder.build() } }