Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,7 @@ or_fun_call = "warn"
unnecessary_lazy_evaluations = "warn"
uninlined_format_args = "warn"
inefficient_to_string = "warn"
needless_pass_by_value = "warn"

[workspace.lints.rust]
unexpected_cfgs = { level = "warn", check-cfg = [
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ struct CustomExec {
}

impl CustomExec {
#[expect(clippy::needless_pass_by_value)]
fn new(
projections: Option<&Vec<usize>>,
schema: SchemaRef,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,7 @@ impl ExternalBatchBufferer {
}
}

#[expect(clippy::needless_pass_by_value)]
fn add_batch(&mut self, batch_data: Vec<u8>) -> Result<()> {
let additional_memory = batch_data.len();

Expand Down
1 change: 1 addition & 0 deletions datafusion-examples/examples/flight/server.rs
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,7 @@ impl FlightService for FlightServiceImpl {
}
}

#[expect(clippy::needless_pass_by_value)]
fn to_tonic_err(e: datafusion::error::DataFusionError) -> Status {
Status::internal(format!("{e:?}"))
}
Expand Down
1 change: 1 addition & 0 deletions datafusion-examples/examples/query_planning/pruning.rs
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,7 @@ impl PruningStatistics for MyCatalog {
}
}

#[expect(clippy::needless_pass_by_value)]
fn create_pruning_predicate(expr: Expr, schema: &SchemaRef) -> PruningPredicate {
let df_schema = DFSchema::try_from(Arc::clone(schema)).unwrap();
let props = ExecutionProps::new();
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/aggregate_query_sql.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ use std::hint::black_box;
use std::sync::Arc;
use tokio::runtime::Runtime;

#[expect(clippy::needless_pass_by_value)]
fn query(ctx: Arc<Mutex<SessionContext>>, rt: &Runtime, sql: &str) {
let df = rt.block_on(ctx.lock().sql(sql)).unwrap();
black_box(rt.block_on(df.collect()).unwrap());
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/csv_load.rs
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ use std::time::Duration;
use test_utils::AccessLogGenerator;
use tokio::runtime::Runtime;

#[expect(clippy::needless_pass_by_value)]
fn load_csv(
ctx: Arc<Mutex<SessionContext>>,
rt: &Runtime,
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/data_utils/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,7 @@ fn create_record_batch(

/// Create record batches of `partitions_len` partitions and `batch_size` for each batch,
/// with a total number of `array_len` records
#[expect(clippy::needless_pass_by_value)]
pub fn create_record_batches(
schema: SchemaRef,
array_len: usize,
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/dataframe.rs
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ fn create_context(field_count: u32) -> datafusion_common::Result<Arc<SessionCont
Ok(Arc::new(ctx))
}

#[expect(clippy::needless_pass_by_value)]
fn run(column_count: u32, ctx: Arc<SessionContext>, rt: &Runtime) {
black_box(rt.block_on(async {
let mut data_frame = ctx.table("t").await.unwrap();
Expand Down
2 changes: 2 additions & 0 deletions datafusion/core/benches/distinct_query_sql.rs
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ use std::hint::black_box;
use std::{sync::Arc, time::Duration};
use tokio::runtime::Runtime;

#[expect(clippy::needless_pass_by_value)]
fn query(ctx: Arc<Mutex<SessionContext>>, rt: &Runtime, sql: &str) {
let df = rt.block_on(ctx.lock().sql(sql)).unwrap();
black_box(rt.block_on(df.collect()).unwrap());
Expand Down Expand Up @@ -124,6 +125,7 @@ async fn distinct_with_limit(
Ok(())
}

#[expect(clippy::needless_pass_by_value)]
fn run(rt: &Runtime, plan: Arc<dyn ExecutionPlan>, ctx: Arc<TaskContext>) {
black_box(rt.block_on(distinct_with_limit(plan.clone(), ctx.clone()))).unwrap();
}
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/math_query_sql.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ use datafusion::datasource::MemTable;
use datafusion::error::Result;
use datafusion::execution::context::SessionContext;

#[expect(clippy::needless_pass_by_value)]
fn query(ctx: Arc<Mutex<SessionContext>>, rt: &Runtime, sql: &str) {
// execute the query
let df = rt.block_on(ctx.lock().sql(sql)).unwrap();
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/physical_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering;

// Initialize the operator using the provided record batches and the sort key
// as inputs. All record batches must have the same schema.
#[expect(clippy::needless_pass_by_value)]
fn sort_preserving_merge_operator(
session_ctx: Arc<SessionContext>,
rt: &Runtime,
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/sort_limit_query_sql.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ use datafusion::execution::context::SessionContext;

use tokio::runtime::Runtime;

#[expect(clippy::needless_pass_by_value)]
fn query(ctx: Arc<Mutex<SessionContext>>, rt: &Runtime, sql: &str) {
// execute the query
let df = rt.block_on(ctx.lock().sql(sql)).unwrap();
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/sql_planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ fn create_context() -> SessionContext {

/// Register the table definitions as a MemTable with the context and return the
/// context
#[expect(clippy::needless_pass_by_value)]
fn register_defs(ctx: SessionContext, defs: Vec<TableDef>) -> SessionContext {
defs.iter().for_each(|TableDef { name, schema }| {
ctx.register_table(
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/topk_aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ async fn create_context(
Ok((physical_plan, ctx.task_ctx()))
}

#[expect(clippy::needless_pass_by_value)]
fn run(rt: &Runtime, plan: Arc<dyn ExecutionPlan>, ctx: Arc<TaskContext>, asc: bool) {
black_box(rt.block_on(async { aggregate(plan.clone(), ctx.clone(), asc).await }))
.unwrap();
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/benches/window_query_sql.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ use std::hint::black_box;
use std::sync::Arc;
use tokio::runtime::Runtime;

#[expect(clippy::needless_pass_by_value)]
fn query(ctx: Arc<Mutex<SessionContext>>, rt: &Runtime, sql: &str) {
let df = rt.block_on(ctx.lock().sql(sql)).unwrap();
black_box(rt.block_on(df.collect()).unwrap());
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/src/bin/print_functions_docs.rs
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ fn save_doc_code_text(documentation: &Documentation, name: &str) {
file.write_all(attr_text.as_bytes()).unwrap();
}

#[expect(clippy::needless_pass_by_value)]
fn print_docs(
providers: Vec<Box<dyn DocProvider>>,
doc_sections: Vec<DocSection>,
Expand Down
1 change: 1 addition & 0 deletions datafusion/core/tests/custom_sources_cases/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -316,6 +316,7 @@ async fn optimizers_catch_all_statistics() {
assert_eq!(format!("{:?}", actual[0]), format!("{expected:?}"));
}

#[expect(clippy::needless_pass_by_value)]
fn contains_place_holder_exec(plan: Arc<dyn ExecutionPlan>) -> bool {
if plan.as_any().is::<PlaceholderRowExec>() {
true
Expand Down
2 changes: 2 additions & 0 deletions datafusion/core/tests/expr_api/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ use datafusion_optimizer::simplify_expressions::ExprSimplifier;
use std::sync::{Arc, LazyLock};

mod parse_sql_expr;
#[expect(clippy::needless_pass_by_value)]
mod simplification;

#[test]
Expand Down Expand Up @@ -384,6 +385,7 @@ async fn evaluate_agg_test(expr: Expr, expected_lines: Vec<&str>) {

/// Converts the `Expr` to a `PhysicalExpr`, evaluates it against the provided
/// `RecordBatch` and compares the result to the expected result.
#[expect(clippy::needless_pass_by_value)]
fn evaluate_expr_test(expr: Expr, expected_lines: Vec<&str>) {
let batch = &TEST_BATCH;
let df_schema = DFSchema::try_from(batch.schema()).unwrap();
Expand Down
6 changes: 6 additions & 0 deletions datafusion/core/tests/fuzz_cases/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,20 +15,26 @@
// specific language governing permissions and limitations
// under the License.

#[expect(clippy::needless_pass_by_value)]
mod aggregate_fuzz;
mod distinct_count_string_fuzz;
#[expect(clippy::needless_pass_by_value)]
mod join_fuzz;
mod merge_fuzz;
#[expect(clippy::needless_pass_by_value)]
mod sort_fuzz;
#[expect(clippy::needless_pass_by_value)]
mod sort_query_fuzz;
mod topk_filter_pushdown;

mod aggregation_fuzzer;
#[expect(clippy::needless_pass_by_value)]
mod equivalence;

mod pruning;

mod limit_fuzz;
#[expect(clippy::needless_pass_by_value)]
mod sort_preserving_repartition_fuzz;
mod window_fuzz;

Expand Down
1 change: 1 addition & 0 deletions datafusion/core/tests/parquet/encryption.rs
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ async fn read_parquet_test_data<'a, T: Into<String>>(
.unwrap()
}

#[expect(clippy::needless_pass_by_value)]
pub fn write_batches(
path: PathBuf,
props: WriterProperties,
Expand Down
4 changes: 4 additions & 0 deletions datafusion/core/tests/parquet/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -652,6 +652,7 @@ fn make_date_batch(offset: Duration) -> RecordBatch {
/// of the column. It is *not* a table named service.name
///
/// name | service.name
#[expect(clippy::needless_pass_by_value)]
fn make_bytearray_batch(
name: &str,
string_values: Vec<&str>,
Expand Down Expand Up @@ -707,6 +708,7 @@ fn make_bytearray_batch(
/// of the column. It is *not* a table named service.name
///
/// name | service.name
#[expect(clippy::needless_pass_by_value)]
fn make_names_batch(name: &str, service_name_values: Vec<&str>) -> RecordBatch {
let num_rows = service_name_values.len();
let name: StringArray = std::iter::repeat_n(Some(name), num_rows).collect();
Expand Down Expand Up @@ -791,6 +793,7 @@ fn make_utf8_batch(value: Vec<Option<&str>>) -> RecordBatch {
.unwrap()
}

#[expect(clippy::needless_pass_by_value)]
fn make_dictionary_batch(strings: Vec<&str>, integers: Vec<i32>) -> RecordBatch {
let keys = Int32Array::from_iter(0..strings.len() as i32);
let small_keys = Int16Array::from_iter(0..strings.len() as i16);
Expand Down Expand Up @@ -839,6 +842,7 @@ fn make_dictionary_batch(strings: Vec<&str>, integers: Vec<i32>) -> RecordBatch
.unwrap()
}

#[expect(clippy::needless_pass_by_value)]
fn create_data_batch(scenario: Scenario) -> Vec<RecordBatch> {
match scenario {
Scenario::Timestamps => {
Expand Down
5 changes: 5 additions & 0 deletions datafusion/core/tests/physical_optimizer/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,18 +17,23 @@

//! Physical Optimizer integration tests

#[expect(clippy::needless_pass_by_value)]
mod aggregate_statistics;
mod combine_partial_final_agg;
#[expect(clippy::needless_pass_by_value)]
mod enforce_distribution;
mod enforce_sorting;
mod enforce_sorting_monotonicity;
#[expect(clippy::needless_pass_by_value)]
mod filter_pushdown;
mod join_selection;
#[expect(clippy::needless_pass_by_value)]
mod limit_pushdown;
mod limited_distinct_aggregation;
mod partition_statistics;
mod projection_pushdown;
mod replace_with_order_preserving_variants;
mod sanity_checker;
#[expect(clippy::needless_pass_by_value)]
mod test_utils;
mod window_optimize;
Original file line number Diff line number Diff line change
Expand Up @@ -569,6 +569,7 @@ impl TimeSum {
Self { sum: 0, test_state }
}

#[expect(clippy::needless_pass_by_value)]
fn register(ctx: &mut SessionContext, test_state: Arc<TestState>, name: &str) {
let timestamp_type = DataType::Timestamp(TimeUnit::Nanosecond, None);
let input_type = vec![timestamp_type.clone()];
Expand Down
1 change: 1 addition & 0 deletions datafusion/functions-aggregate/benches/array_agg.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ pub fn seedable_rng() -> StdRng {
StdRng::seed_from_u64(42)
}

#[expect(clippy::needless_pass_by_value)]
fn merge_batch_bench(c: &mut Criterion, name: &str, values: ArrayRef) {
let list_item_data_type = values.as_list::<i32>().values().data_type().clone();
c.bench_function(name, |b| {
Expand Down
1 change: 1 addition & 0 deletions datafusion/functions-aggregate/benches/count.rs
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ fn prepare_accumulator() -> Box<dyn Accumulator> {
count_fn.accumulator(accumulator_args).unwrap()
}

#[expect(clippy::needless_pass_by_value)]
fn convert_to_state_bench(
c: &mut Criterion,
name: &str,
Expand Down
1 change: 1 addition & 0 deletions datafusion/functions-aggregate/benches/sum.rs
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ fn prepare_accumulator(data_type: &DataType) -> Box<dyn GroupsAccumulator> {
sum_fn.create_groups_accumulator(accumulator_args).unwrap()
}

#[expect(clippy::needless_pass_by_value)]
fn convert_to_state_bench(
c: &mut Criterion,
name: &str,
Expand Down
1 change: 1 addition & 0 deletions datafusion/functions/benches/pad.rs
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,7 @@ fn create_args<O: OffsetSizeTrait>(
}
}

#[expect(clippy::needless_pass_by_value)]
fn invoke_pad_with_args(
args: Vec<ColumnarValue>,
number_rows: usize,
Expand Down
1 change: 1 addition & 0 deletions datafusion/functions/benches/substr.rs
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,7 @@ fn create_args_with_count<O: OffsetSizeTrait>(
}
}

#[expect(clippy::needless_pass_by_value)]
fn invoke_substr_with_args(
args: Vec<ColumnarValue>,
number_rows: usize,
Expand Down
1 change: 1 addition & 0 deletions datafusion/physical-expr/benches/binary_op.rs
Original file line number Diff line number Diff line change
Expand Up @@ -286,6 +286,7 @@ fn generate_test_strings(num_rows: usize) -> (Vec<String>, Vec<String>) {
/// Creates record batches with boolean arrays that test different short-circuit scenarios.
/// When TEST_ALL_FALSE = true: creates data for AND operator benchmarks (needs early false exit)
/// When TEST_ALL_FALSE = false: creates data for OR operator benchmarks (needs early true exit)
#[expect(clippy::needless_pass_by_value)]
fn create_record_batch<const TEST_ALL_FALSE: bool>(
schema: Arc<Schema>,
b_values: &[String],
Expand Down
6 changes: 3 additions & 3 deletions datafusion/physical-expr/benches/case_when.rs
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,7 @@ fn create_random_string_generator(
/// `null_percentage` is the percentage of null values
/// The rest of the values will be outside the specified range
fn generate_values_for_lookup<T, A>(
options: Options<T>,
options: &Options<T>,
generate_other_value: impl Fn(&mut StdRng, &[T]) -> T,
) -> A
where
Expand Down Expand Up @@ -416,7 +416,7 @@ fn benchmark_lookup_table_case_when(c: &mut Criterion, batch_size: usize) {
&input,
|b, input| {
let array: Int32Array = generate_values_for_lookup(
Options::<i32> {
&Options::<i32> {
number_of_rows: batch_size,
range_of_values: when_thens_primitive_to_string
.iter()
Expand Down Expand Up @@ -469,7 +469,7 @@ fn benchmark_lookup_table_case_when(c: &mut Criterion, batch_size: usize) {
&input,
|b, input| {
let array: StringArray = generate_values_for_lookup(
Options::<String> {
&Options::<String> {
number_of_rows: batch_size,
range_of_values: when_thens_string_to_primitive
.iter()
Expand Down
1 change: 1 addition & 0 deletions datafusion/physical-plan/benches/aggregate_vectorized.rs
Original file line number Diff line number Diff line change
Expand Up @@ -271,6 +271,7 @@ fn bench_single_primitive<const NULLABLE: bool>(
}

/// Test `vectorized_equal_to` with different number of true in the initial results
#[expect(clippy::needless_pass_by_value)]
fn vectorized_equal_to<GroupColumnBuilder: GroupColumn>(
group: &mut BenchmarkGroup<WallTime>,
mut builder: GroupColumnBuilder,
Expand Down
1 change: 1 addition & 0 deletions datafusion/physical-plan/benches/spill_io.rs
Original file line number Diff line number Diff line change
Expand Up @@ -490,6 +490,7 @@ fn bench_spill_compression(c: &mut Criterion) {
group.finish();
}

#[expect(clippy::needless_pass_by_value)]
fn benchmark_spill_batches_for_all_codec(
group: &mut BenchmarkGroup<'_, WallTime>,
batch_label: &str,
Expand Down
4 changes: 4 additions & 0 deletions datafusion/sql/tests/sql_integration.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,10 @@
// specific language governing permissions and limitations
// under the License.

// This lint violation is acceptable for tests, so suppress for now
// Issue: <https://github.com/apache/datafusion/issues/18503>
#![expect(clippy::needless_pass_by_value)]

use std::any::Any;
use std::hash::Hash;
#[cfg(test)]
Expand Down
1 change: 1 addition & 0 deletions datafusion/sqllogictest/bin/sqllogictests.rs
Original file line number Diff line number Diff line change
Expand Up @@ -407,6 +407,7 @@ async fn run_file_in_runner<D: AsyncDB, M: MakeConnection<Conn = D>>(
Ok(())
}

#[expect(clippy::needless_pass_by_value)]
fn get_record_count(path: &PathBuf, label: String) -> u64 {
let records: Vec<Record<<DataFusion as AsyncDB>::ColumnType>> =
parse_file(path).unwrap();
Expand Down
1 change: 1 addition & 0 deletions datafusion/sqllogictest/src/engines/conversion.rs
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,7 @@ pub(crate) fn decimal_to_str(value: Decimal) -> String {
/// Converts a `BigDecimal` to its plain string representation, optionally rounding to a specified number of decimal places.
///
/// If `round_digits` is `None`, the value is rounded to 12 decimal places by default.
#[expect(clippy::needless_pass_by_value)]
pub(crate) fn big_decimal_to_str(value: BigDecimal, round_digits: Option<i64>) -> String {
// Round the value to limit the number of decimal places
let value = value.round(round_digits.unwrap_or(12)).normalized();
Expand Down