From ac876dbc9729b16e272e00496c51e53d9f649173 Mon Sep 17 00:00:00 2001 From: "xudong.w" Date: Thu, 29 Dec 2022 23:22:51 +0800 Subject: [PATCH] Make nightly clippy happy (#4515) * Make nightly clippy happy * update --- benchmarks/src/bin/h2o.rs | 4 +- benchmarks/src/bin/nyctaxi.rs | 6 +- benchmarks/src/bin/parquet_filter_pushdown.rs | 6 +- benchmarks/src/bin/tpch.rs | 29 ++- benchmarks/src/tpch.rs | 16 +- datafusion-examples/examples/csv_sql.rs | 2 +- datafusion-examples/examples/dataframe.rs | 2 +- .../examples/deserialize_to_struct.rs | 4 +- datafusion-examples/examples/flight_client.rs | 6 +- datafusion-examples/examples/flight_server.rs | 10 +- datafusion-examples/examples/parquet_sql.rs | 2 +- .../examples/parquet_sql_multiple_files.rs | 2 +- datafusion-examples/examples/query-aws-s3.rs | 5 +- datafusion/common/src/dfschema.rs | 11 +- datafusion/common/src/error.rs | 38 ++- datafusion/common/src/parsers.rs | 29 +-- datafusion/common/src/scalar.rs | 140 +++++------ datafusion/common/src/table_reference.rs | 2 +- datafusion/core/benches/merge.rs | 2 +- datafusion/core/benches/parquet_query_sql.rs | 10 +- datafusion/core/benches/physical_plan.rs | 6 +- .../core/benches/sort_limit_query_sql.rs | 2 +- datafusion/core/benches/sql_planner.rs | 2 +- datafusion/core/src/bin/print_config_docs.rs | 2 +- .../core/src/catalog/information_schema.rs | 2 +- datafusion/core/src/catalog/schema.rs | 7 +- datafusion/core/src/config.rs | 4 +- datafusion/core/src/dataframe.rs | 4 +- .../src/datasource/file_format/file_type.rs | 5 +- .../core/src/datasource/file_format/mod.rs | 2 +- .../src/datasource/file_format/parquet.rs | 16 +- .../core/src/datasource/listing/helpers.rs | 6 +- .../core/src/datasource/listing/table.rs | 11 +- datafusion/core/src/datasource/listing/url.rs | 5 +- datafusion/core/src/datasource/memory.rs | 18 +- .../core/src/datasource/object_store.rs | 6 +- datafusion/core/src/execution/context.rs | 54 ++--- datafusion/core/src/execution/disk_manager.rs | 5 +- .../core/src/execution/memory_pool/mod.rs | 2 +- .../src/physical_optimizer/enforcement.rs | 14 +- .../src/physical_optimizer/join_selection.rs | 2 +- .../src/physical_optimizer/optimize_sorts.rs | 30 +-- .../src/physical_optimizer/pipeline_fixer.rs | 3 +- .../core/src/physical_optimizer/pruning.rs | 73 +++--- .../core/src/physical_optimizer/test_utils.rs | 2 +- .../core/src/physical_plan/aggregates/hash.rs | 2 +- .../core/src/physical_plan/aggregates/mod.rs | 14 +- .../physical_plan/aggregates/no_grouping.rs | 2 +- .../src/physical_plan/aggregates/row_hash.rs | 2 +- datafusion/core/src/physical_plan/analyze.rs | 6 +- .../src/physical_plan/coalesce_partitions.rs | 3 +- datafusion/core/src/physical_plan/common.rs | 10 +- datafusion/core/src/physical_plan/display.rs | 4 +- datafusion/core/src/physical_plan/empty.rs | 2 +- datafusion/core/src/physical_plan/explain.rs | 3 +- .../core/src/physical_plan/file_format/csv.rs | 13 +- .../src/physical_plan/file_format/json.rs | 13 +- .../src/physical_plan/file_format/parquet.rs | 47 ++-- .../file_format/parquet/page_filter.rs | 3 +- .../file_format/parquet/row_filter.rs | 9 +- datafusion/core/src/physical_plan/filter.rs | 3 +- .../core/src/physical_plan/joins/hash_join.rs | 6 +- .../core/src/physical_plan/joins/utils.rs | 4 +- datafusion/core/src/physical_plan/limit.rs | 3 +- datafusion/core/src/physical_plan/memory.rs | 3 +- .../core/src/physical_plan/metrics/mod.rs | 4 +- .../core/src/physical_plan/metrics/tracker.rs | 5 +- .../core/src/physical_plan/metrics/value.rs | 25 +- datafusion/core/src/physical_plan/planner.rs | 109 ++++----- .../core/src/physical_plan/projection.rs | 2 +- .../core/src/physical_plan/repartition.rs | 12 +- .../core/src/physical_plan/sorts/sort.rs | 14 +- .../sorts/sort_preserving_merge.rs | 9 +- .../core/src/physical_plan/streaming.rs | 3 +- datafusion/core/src/physical_plan/udaf.rs | 2 +- datafusion/core/src/physical_plan/union.rs | 3 +- datafusion/core/src/physical_plan/values.rs | 6 +- .../core/src/physical_plan/windows/mod.rs | 4 +- datafusion/core/src/scheduler/mod.rs | 7 +- .../src/scheduler/pipeline/repartition.rs | 6 +- datafusion/core/src/test/exec.rs | 14 +- datafusion/core/src/test/mod.rs | 2 +- datafusion/core/src/test_util.rs | 4 +- datafusion/core/tests/custom_sources.rs | 12 +- datafusion/core/tests/dataframe.rs | 2 +- datafusion/core/tests/fifo.rs | 4 +- datafusion/core/tests/merge_fuzz.rs | 2 +- datafusion/core/tests/order_spill_fuzz.rs | 2 +- .../core/tests/parquet/custom_reader.rs | 5 +- .../core/tests/parquet/filter_pushdown.rs | 11 +- datafusion/core/tests/parquet/mod.rs | 8 +- datafusion/core/tests/parquet/page_pruning.rs | 2 +- datafusion/core/tests/path_partition.rs | 13 +- .../core/tests/provider_filter_pushdown.rs | 11 +- datafusion/core/tests/row.rs | 2 +- datafusion/core/tests/sort_key_cursor.rs | 3 +- datafusion/core/tests/sql/aggregates.rs | 2 +- datafusion/core/tests/sql/errors.rs | 5 +- datafusion/core/tests/sql/explain.rs | 4 +- datafusion/core/tests/sql/explain_analyze.rs | 68 +++--- datafusion/core/tests/sql/expr.rs | 4 +- datafusion/core/tests/sql/joins.rs | 135 +++++------ datafusion/core/tests/sql/json.rs | 6 +- datafusion/core/tests/sql/limit.rs | 4 +- datafusion/core/tests/sql/mod.rs | 19 +- datafusion/core/tests/sql/parquet.rs | 6 +- datafusion/core/tests/sql/parquet_schema.rs | 2 +- datafusion/core/tests/sql/partitioned_csv.rs | 2 +- datafusion/core/tests/sql/predicates.rs | 8 +- datafusion/core/tests/sql/projection.rs | 9 +- datafusion/core/tests/sql/references.rs | 2 +- datafusion/core/tests/sql/udf.rs | 2 +- datafusion/core/tests/sql/window.rs | 78 +++---- .../core/tests/sqllogictests/src/setup.rs | 9 +- datafusion/core/tests/user_defined_plan.rs | 11 +- datafusion/expr/src/aggregate_function.rs | 5 +- datafusion/expr/src/built_in_function.rs | 5 +- .../expr/src/conditional_expressions.rs | 3 +- datafusion/expr/src/expr.rs | 176 +++++++------- datafusion/expr/src/expr_rewriter.rs | 4 +- datafusion/expr/src/expr_schema.rs | 3 +- datafusion/expr/src/field_util.rs | 3 +- datafusion/expr/src/function.rs | 6 +- datafusion/expr/src/logical_plan/builder.rs | 40 ++-- datafusion/expr/src/logical_plan/display.rs | 5 +- datafusion/expr/src/logical_plan/plan.rs | 82 +++---- datafusion/expr/src/operator.rs | 2 +- .../expr/src/type_coercion/aggregates.rs | 21 +- datafusion/expr/src/type_coercion/binary.rs | 8 +- datafusion/expr/src/utils.rs | 15 +- datafusion/expr/src/window_frame.rs | 10 +- datafusion/expr/src/window_function.rs | 3 +- datafusion/jit/src/api.rs | 4 +- datafusion/jit/src/ast.rs | 72 +++--- datafusion/jit/src/compile.rs | 8 +- datafusion/jit/src/jit.rs | 6 +- .../optimizer/src/common_subexpr_eliminate.rs | 4 +- .../optimizer/src/eliminate_cross_join.rs | 3 +- datafusion/optimizer/src/eliminate_limit.rs | 4 +- datafusion/optimizer/src/optimizer.rs | 9 +- .../optimizer/src/propagate_empty_relation.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 64 ++--- .../optimizer/src/push_down_projection.rs | 12 +- .../src/simplify_expressions/context.rs | 3 +- .../simplify_expressions/expr_simplifier.rs | 3 +- .../simplify_expressions/simplify_exprs.rs | 4 +- .../src/simplify_expressions/utils.rs | 11 +- .../src/single_distinct_to_groupby.rs | 2 +- datafusion/optimizer/src/test/mod.rs | 4 +- datafusion/optimizer/src/type_coercion.rs | 26 +-- .../src/unwrap_cast_in_comparison.rs | 24 +- datafusion/optimizer/src/utils.rs | 4 +- .../optimizer/tests/integration-test.rs | 40 ++-- datafusion/physical-expr/benches/in_list.rs | 13 +- .../src/aggregate/approx_distinct.rs | 5 +- .../src/aggregate/approx_percentile_cont.rs | 9 +- .../physical-expr/src/aggregate/build_in.rs | 3 +- .../src/aggregate/count_distinct.rs | 12 +- .../physical-expr/src/aggregate/median.rs | 6 +- datafusion/physical-expr/src/aggregate/mod.rs | 6 +- datafusion/physical-expr/src/aggregate/sum.rs | 9 +- .../src/aggregate/sum_distinct.rs | 3 +- .../physical-expr/src/aggregate/tdigest.rs | 4 +- .../physical-expr/src/array_expressions.rs | 3 +- .../physical-expr/src/crypto_expressions.rs | 17 +- .../physical-expr/src/datetime_expressions.rs | 22 +- .../physical-expr/src/expressions/binary.rs | 5 +- .../physical-expr/src/expressions/case.rs | 9 +- .../physical-expr/src/expressions/cast.rs | 3 +- .../physical-expr/src/expressions/column.rs | 6 +- .../physical-expr/src/expressions/datetime.rs | 35 ++- .../src/expressions/get_indexed_field.rs | 8 +- .../physical-expr/src/expressions/in_list.rs | 8 +- .../physical-expr/src/expressions/literal.rs | 2 +- .../physical-expr/src/expressions/mod.rs | 2 +- .../physical-expr/src/expressions/negative.rs | 2 +- .../physical-expr/src/expressions/try_cast.rs | 3 +- datafusion/physical-expr/src/functions.rs | 102 +++----- .../physical-expr/src/math_expressions.rs | 6 +- datafusion/physical-expr/src/planner.rs | 3 +- .../physical-expr/src/regex_expressions.rs | 11 +- .../physical-expr/src/scalar_function.rs | 2 +- datafusion/physical-expr/src/sort_expr.rs | 3 +- .../physical-expr/src/string_expressions.rs | 28 +-- .../physical-expr/src/struct_expressions.rs | 5 +- datafusion/physical-expr/src/type_coercion.rs | 7 +- .../physical-expr/src/unicode_expressions.rs | 25 +- .../src/window/window_frame_state.rs | 12 +- datafusion/proto/build.rs | 4 +- .../proto/examples/logical_plan_serde.rs | 2 +- .../proto/examples/physical_plan_serde.rs | 4 +- datafusion/proto/src/bytes/mod.rs | 16 +- datafusion/proto/src/bytes/registry.rs | 4 +- .../proto/src/logical_plan/from_proto.rs | 21 +- datafusion/proto/src/logical_plan/mod.rs | 58 ++--- datafusion/proto/src/logical_plan/to_proto.rs | 21 +- .../proto/src/physical_plan/from_proto.rs | 10 +- datafusion/proto/src/physical_plan/mod.rs | 38 +-- .../proto/src/physical_plan/to_proto.rs | 6 +- datafusion/row/src/layout.rs | 4 +- datafusion/sql/examples/sql.rs | 2 +- datafusion/sql/src/parser.rs | 9 +- datafusion/sql/src/planner.rs | 218 ++++++++---------- datafusion/sql/src/utils.rs | 11 +- parquet-test-utils/src/lib.rs | 2 +- test-utils/src/data_gen.rs | 7 +- 206 files changed, 1260 insertions(+), 1721 deletions(-) diff --git a/benchmarks/src/bin/h2o.rs b/benchmarks/src/bin/h2o.rs index 88f4084e87a5..446210f683f5 100644 --- a/benchmarks/src/bin/h2o.rs +++ b/benchmarks/src/bin/h2o.rs @@ -55,7 +55,7 @@ struct GroupBy { #[tokio::main] async fn main() -> Result<()> { let opt = Opt::from_args(); - println!("Running benchmarks with the following options: {:?}", opt); + println!("Running benchmarks with the following options: {opt:?}"); match opt { Opt::GroupBy(config) => group_by(&config).await, } @@ -107,7 +107,7 @@ async fn group_by(opt: &GroupBy) -> Result<()> { _ => unimplemented!(), }; - println!("Executing {}", sql); + println!("Executing {sql}"); let start = Instant::now(); let df = ctx.sql(sql).await?; let batches = df.collect().await?; diff --git a/benchmarks/src/bin/nyctaxi.rs b/benchmarks/src/bin/nyctaxi.rs index 57024460d28a..266335228332 100644 --- a/benchmarks/src/bin/nyctaxi.rs +++ b/benchmarks/src/bin/nyctaxi.rs @@ -66,7 +66,7 @@ struct Opt { #[tokio::main] async fn main() -> Result<()> { let opt = Opt::from_args(); - println!("Running benchmarks with the following options: {:?}", opt); + println!("Running benchmarks with the following options: {opt:?}"); let config = SessionConfig::new() .with_target_partitions(opt.partitions) @@ -86,7 +86,7 @@ async fn main() -> Result<()> { .await? } other => { - println!("Invalid file format '{}'", other); + println!("Invalid file format '{other}'"); process::exit(-1); } } @@ -102,7 +102,7 @@ async fn datafusion_sql_benchmarks( let mut queries = HashMap::new(); queries.insert("fare_amt_by_passenger", "SELECT passenger_count, MIN(fare_amount), MAX(fare_amount), SUM(fare_amount) FROM tripdata GROUP BY passenger_count"); for (name, sql) in &queries { - println!("Executing '{}'", name); + println!("Executing '{name}'"); for i in 0..iterations { let start = Instant::now(); execute_sql(ctx, sql, debug).await?; diff --git a/benchmarks/src/bin/parquet_filter_pushdown.rs b/benchmarks/src/bin/parquet_filter_pushdown.rs index 33b490b5a636..d49cb839dafc 100644 --- a/benchmarks/src/bin/parquet_filter_pushdown.rs +++ b/benchmarks/src/bin/parquet_filter_pushdown.rs @@ -67,7 +67,7 @@ struct Opt { #[tokio::main] async fn main() -> Result<()> { let opt: Opt = Opt::from_args(); - println!("Running benchmarks with the following options: {:?}", opt); + println!("Running benchmarks with the following options: {opt:?}"); let path = opt.path.join("logs.parquet"); @@ -137,9 +137,9 @@ async fn run_benchmarks(opt: Opt, test_file: &TestParquetFile) -> Result<()> { ]; for filter_expr in &filter_matrix { - println!("Executing with filter '{}'", filter_expr); + println!("Executing with filter '{filter_expr}'"); for scan_options in &scan_options_matrix { - println!("Using scan options {:?}", scan_options); + println!("Using scan options {scan_options:?}"); for i in 0..opt.iterations { let start = Instant::now(); diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index d74e9b64d185..1b1bb876c80d 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -169,8 +169,7 @@ async fn main() -> Result<()> { "zstd" => Compression::ZSTD, other => { return Err(DataFusionError::NotImplemented(format!( - "Invalid compression format: {}", - other + "Invalid compression format: {other}" ))); } }; @@ -193,7 +192,7 @@ const TPCH_QUERY_END_ID: usize = 22; async fn benchmark_datafusion( opt: DataFusionBenchmarkOpt, ) -> Result>> { - println!("Running benchmarks with the following options: {:?}", opt); + println!("Running benchmarks with the following options: {opt:?}"); let query_range = match opt.query { Some(query_id) => query_id..=query_id, None => TPCH_QUERY_START_ID..=TPCH_QUERY_END_ID, @@ -257,14 +256,13 @@ async fn benchmark_query( millis.push(elapsed); let row_count = result.iter().map(|b| b.num_rows()).sum(); println!( - "Query {} iteration {} took {:.1} ms and returned {} rows", - query_id, i, elapsed, row_count + "Query {query_id} iteration {i} took {elapsed:.1} ms and returned {row_count} rows" ); benchmark_run.add_result(elapsed, row_count); } let avg = millis.iter().sum::() / millis.len() as f64; - println!("Query {} avg time: {:.2} ms", query_id, avg); + println!("Query {query_id} avg time: {avg:.2} ms"); Ok((benchmark_run, result)) } @@ -286,7 +284,7 @@ async fn register_tables( }; if opt.mem_table { - println!("Loading table '{}' into memory", table); + println!("Loading table '{table}' into memory"); let start = Instant::now(); let memtable = MemTable::load(table_provider, Some(opt.partitions), &ctx.state()) @@ -328,12 +326,12 @@ async fn execute_query( let (state, plan) = plan.into_parts(); if debug { - println!("=== Logical plan ===\n{:?}\n", plan); + println!("=== Logical plan ===\n{plan:?}\n"); } let plan = state.optimize(&plan)?; if debug { - println!("=== Optimized logical plan ===\n{:?}\n", plan); + println!("=== Optimized logical plan ===\n{plan:?}\n"); } let physical_plan = state.create_physical_plan(&plan).await?; if debug { @@ -378,7 +376,7 @@ async fn get_table( match table_format { // dbgen creates .tbl ('|' delimited) files without header "tbl" => { - let path = format!("{}/{}.tbl", path, table); + let path = format!("{path}/{table}.tbl"); let format = CsvFormat::default() .with_delimiter(b'|') @@ -387,7 +385,7 @@ async fn get_table( (Arc::new(format), path, ".tbl") } "csv" => { - let path = format!("{}/{}", path, table); + let path = format!("{path}/{table}"); let format = CsvFormat::default() .with_delimiter(b',') .with_has_header(true); @@ -395,7 +393,7 @@ async fn get_table( (Arc::new(format), path, DEFAULT_CSV_EXTENSION) } "parquet" => { - let path = format!("{}/{}", path, table); + let path = format!("{path}/{table}"); let format = ParquetFormat::default().with_enable_pruning(Some(true)); (Arc::new(format), path, DEFAULT_PARQUET_EXTENSION) @@ -651,8 +649,8 @@ mod tests { } let possibilities = vec![ - format!("expected-plans/q{}.txt", query), - format!("benchmarks/expected-plans/q{}.txt", query), + format!("expected-plans/q{query}.txt"), + format!("benchmarks/expected-plans/q{query}.txt"), ]; let mut found = false; @@ -661,8 +659,7 @@ mod tests { if let Ok(expected) = read_text_file(path) { assert_eq!(expected, actual, // generate output that is easier to copy/paste/update - "\n\nMismatch of expected content in: {:?}\nExpected:\n\n{}\n\nActual:\n\n{}\n\n", - path, expected, actual); + "\n\nMismatch of expected content in: {path:?}\nExpected:\n\n{expected}\n\nActual:\n\n{actual}\n\n"); found = true; break; } diff --git a/benchmarks/src/tpch.rs b/benchmarks/src/tpch.rs index 4ebaf5ef5762..deaecdd93db1 100644 --- a/benchmarks/src/tpch.rs +++ b/benchmarks/src/tpch.rs @@ -292,8 +292,8 @@ pub fn get_answer_schema(n: usize) -> Schema { pub fn get_query_sql(query: usize) -> Result> { if query > 0 && query < 23 { let possibilities = vec![ - format!("queries/q{}.sql", query), - format!("benchmarks/queries/q{}.sql", query), + format!("queries/q{query}.sql"), + format!("benchmarks/queries/q{query}.sql"), ]; let mut errors = vec![]; for filename in possibilities { @@ -306,12 +306,11 @@ pub fn get_query_sql(query: usize) -> Result> { .map(|s| s.to_string()) .collect()); } - Err(e) => errors.push(format!("{}: {}", filename, e)), + Err(e) => errors.push(format!("{filename}: {e}")), }; } Err(DataFusionError::Plan(format!( - "invalid query. Could not find query: {:?}", - errors + "invalid query. Could not find query: {errors:?}" ))) } else { Err(DataFusionError::Plan( @@ -334,7 +333,7 @@ pub async fn convert_tbl( let start = Instant::now(); let schema = get_tpch_table_schema(table); - let input_path = format!("{}/{}.tbl", input_path, table); + let input_path = format!("{input_path}/{table}.tbl"); let options = CsvReadOptions::new() .schema(&schema) .has_header(false) @@ -372,8 +371,7 @@ pub async fn convert_tbl( } other => { return Err(DataFusionError::NotImplemented(format!( - "Invalid output format: {}", - other + "Invalid output format: {other}" ))); } } @@ -447,7 +445,7 @@ fn col_to_scalar(column: &ArrayRef, row_index: usize) -> ScalarValue { let array = as_string_array(column).unwrap(); ScalarValue::Utf8(Some(array.value(row_index).to_string())) } - other => panic!("unexpected data type in benchmark: {}", other), + other => panic!("unexpected data type in benchmark: {other}"), } } diff --git a/datafusion-examples/examples/csv_sql.rs b/datafusion-examples/examples/csv_sql.rs index 0dfc288c7510..ce602e0e4816 100644 --- a/datafusion-examples/examples/csv_sql.rs +++ b/datafusion-examples/examples/csv_sql.rs @@ -30,7 +30,7 @@ async fn main() -> Result<()> { // register csv file with the execution context ctx.register_csv( "aggregate_test_100", - &format!("{}/csv/aggregate_test_100.csv", testdata), + &format!("{testdata}/csv/aggregate_test_100.csv"), CsvReadOptions::new(), ) .await?; diff --git a/datafusion-examples/examples/dataframe.rs b/datafusion-examples/examples/dataframe.rs index 9ec03c594bfa..f52ff8925612 100644 --- a/datafusion-examples/examples/dataframe.rs +++ b/datafusion-examples/examples/dataframe.rs @@ -29,7 +29,7 @@ async fn main() -> Result<()> { let testdata = datafusion::test_util::parquet_test_data(); - let filename = &format!("{}/alltypes_plain.parquet", testdata); + let filename = &format!("{testdata}/alltypes_plain.parquet"); // define the query using the DataFrame trait let df = ctx diff --git a/datafusion-examples/examples/deserialize_to_struct.rs b/datafusion-examples/examples/deserialize_to_struct.rs index dba2e04b1bf6..3bfc7d096abe 100644 --- a/datafusion-examples/examples/deserialize_to_struct.rs +++ b/datafusion-examples/examples/deserialize_to_struct.rs @@ -26,7 +26,7 @@ use serde::Deserialize; #[tokio::main] async fn main() -> Result<()> { let data_list = Data::new().await?; - println!("{:#?}", data_list); + println!("{data_list:#?}"); Ok(()) } @@ -48,7 +48,7 @@ impl Data { ctx.register_parquet( "alltypes_plain", - &format!("{}/alltypes_plain.parquet", testdata), + &format!("{testdata}/alltypes_plain.parquet"), ParquetReadOptions::default(), ) .await?; diff --git a/datafusion-examples/examples/flight_client.rs b/datafusion-examples/examples/flight_client.rs index 9fea1ab7db7e..7a43af9a45b5 100644 --- a/datafusion-examples/examples/flight_client.rs +++ b/datafusion-examples/examples/flight_client.rs @@ -41,12 +41,12 @@ async fn main() -> Result<(), Box> { let request = tonic::Request::new(FlightDescriptor { r#type: flight_descriptor::DescriptorType::Path as i32, cmd: vec![], - path: vec![format!("{}/alltypes_plain.parquet", testdata)], + path: vec![format!("{testdata}/alltypes_plain.parquet")], }); let schema_result = client.get_schema(request).await?.into_inner(); let schema = Schema::try_from(&schema_result)?; - println!("Schema: {:?}", schema); + println!("Schema: {schema:?}"); // Call do_get to execute a SQL query and receive results let request = tonic::Request::new(Ticket { @@ -59,7 +59,7 @@ async fn main() -> Result<(), Box> { let flight_data = stream.message().await?.unwrap(); // convert FlightData to a stream let schema = Arc::new(Schema::try_from(&flight_data)?); - println!("Schema: {:?}", schema); + println!("Schema: {schema:?}"); // all the remaining stream messages should be dictionary and record batches let mut results = vec![]; diff --git a/datafusion-examples/examples/flight_server.rs b/datafusion-examples/examples/flight_server.rs index 26507661cb9e..9b6dc39e4eab 100644 --- a/datafusion-examples/examples/flight_server.rs +++ b/datafusion-examples/examples/flight_server.rs @@ -77,7 +77,7 @@ impl FlightService for FlightServiceImpl { let ticket = request.into_inner(); match std::str::from_utf8(&ticket.ticket) { Ok(sql) => { - println!("do_get: {}", sql); + println!("do_get: {sql}"); // create local execution context let ctx = SessionContext::new(); @@ -87,7 +87,7 @@ impl FlightService for FlightServiceImpl { // register parquet file with the execution context ctx.register_parquet( "alltypes_plain", - &format!("{}/alltypes_plain.parquet", testdata), + &format!("{testdata}/alltypes_plain.parquet"), ParquetReadOptions::default(), ) .await @@ -131,7 +131,7 @@ impl FlightService for FlightServiceImpl { Ok(Response::new(Box::pin(output) as Self::DoGetStream)) } - Err(e) => Err(Status::invalid_argument(format!("Invalid ticket: {:?}", e))), + Err(e) => Err(Status::invalid_argument(format!("Invalid ticket: {e:?}"))), } } @@ -186,7 +186,7 @@ impl FlightService for FlightServiceImpl { } fn to_tonic_err(e: datafusion::error::DataFusionError) -> Status { - Status::internal(format!("{:?}", e)) + Status::internal(format!("{e:?}")) } /// This example shows how to wrap DataFusion with `FlightService` to support looking up schema information for @@ -199,7 +199,7 @@ async fn main() -> Result<(), Box> { let svc = FlightServiceServer::new(service); - println!("Listening on {:?}", addr); + println!("Listening on {addr:?}"); Server::builder().add_service(svc).serve(addr).await?; diff --git a/datafusion-examples/examples/parquet_sql.rs b/datafusion-examples/examples/parquet_sql.rs index bcaa05d7e85a..fb438a7832cb 100644 --- a/datafusion-examples/examples/parquet_sql.rs +++ b/datafusion-examples/examples/parquet_sql.rs @@ -30,7 +30,7 @@ async fn main() -> Result<()> { // register parquet file with the execution context ctx.register_parquet( "alltypes_plain", - &format!("{}/alltypes_plain.parquet", testdata), + &format!("{testdata}/alltypes_plain.parquet"), ParquetReadOptions::default(), ) .await?; diff --git a/datafusion-examples/examples/parquet_sql_multiple_files.rs b/datafusion-examples/examples/parquet_sql_multiple_files.rs index 241449dc47b2..d4893818efcf 100644 --- a/datafusion-examples/examples/parquet_sql_multiple_files.rs +++ b/datafusion-examples/examples/parquet_sql_multiple_files.rs @@ -40,7 +40,7 @@ async fn main() -> Result<()> { // for the query ctx.register_listing_table( "my_table", - &format!("file://{}", testdata), + &format!("file://{testdata}"), listing_options, None, None, diff --git a/datafusion-examples/examples/query-aws-s3.rs b/datafusion-examples/examples/query-aws-s3.rs index 5969eb73e026..37881673123b 100644 --- a/datafusion-examples/examples/query-aws-s3.rs +++ b/datafusion-examples/examples/query-aws-s3.rs @@ -51,10 +51,7 @@ async fn main() -> Result<()> { // cannot query the parquet files from this bucket because the path contains a whitespace // and we don't support that yet // https://github.com/apache/arrow-rs/issues/2799 - let path = format!( - "s3://{}/csv_backup/yellow_tripdata_2022-02.csv", - bucket_name - ); + let path = format!("s3://{bucket_name}/csv_backup/yellow_tripdata_2022-02.csv"); ctx.register_csv("trips", &path, CsvReadOptions::default()) .await?; diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 764a5743f014..110a1ec41852 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -174,9 +174,8 @@ impl DFSchema { Some(qualifier) => { if (qualifier.to_owned() + "." + self.fields[i].name()) == name { return Err(DataFusionError::Plan(format!( - "Fully qualified field name '{}' was supplied to `index_of` \ - which is deprecated. Please use `index_of_column_by_name` instead", - name + "Fully qualified field name '{name}' was supplied to `index_of` \ + which is deprecated. Please use `index_of_column_by_name` instead" ))); } } @@ -669,7 +668,7 @@ mod tests { let err = schema.index_of_column_by_name(None, "t1.c0").err().unwrap(); assert_eq!( "Schema error: No field named 't1.c0'. Valid fields are 't1'.'c0', 't1'.'c1'.", - &format!("{}", err) + &format!("{err}") ); Ok(()) } @@ -1024,7 +1023,7 @@ mod tests { impl<'a> TestCase<'a> { fn run(self) { - println!("Running {:#?}", self); + println!("Running {self:#?}"); let schema1 = to_df_schema(self.fields1); let schema2 = to_df_schema(self.fields2); assert_eq!( @@ -1123,7 +1122,7 @@ mod tests { fn test_metadata_n(n: usize) -> HashMap { (0..n) .into_iter() - .map(|i| (format!("k{}", i), format!("v{}", i))) + .map(|i| (format!("k{i}"), format!("v{i}"))) .collect() } } diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index 47b026d598ab..0d8e9b652994 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -182,22 +182,20 @@ impl Display for SchemaError { Self::DuplicateQualifiedField { qualifier, name } => { write!( f, - "Schema contains duplicate qualified field name '{}'.'{}'", - qualifier, name + "Schema contains duplicate qualified field name '{qualifier}'.'{name}'" ) } Self::DuplicateUnqualifiedField { name } => { write!( f, - "Schema contains duplicate unqualified field name '{}'", - name + "Schema contains duplicate unqualified field name '{name}'" ) } Self::AmbiguousReference { qualifier, name } => { if let Some(q) = qualifier { - write!(f, "Schema contains qualified field name '{}'.'{}' and unqualified field name '{}' which would be ambiguous", q, name, name) + write!(f, "Schema contains qualified field name '{q}'.'{name}' and unqualified field name '{name}' which would be ambiguous") } else { - write!(f, "Ambiguous reference to unqualified field '{}'", name) + write!(f, "Ambiguous reference to unqualified field '{name}'") } } } @@ -276,48 +274,48 @@ impl From for DataFusionError { impl Display for DataFusionError { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match *self { - DataFusionError::ArrowError(ref desc) => write!(f, "Arrow error: {}", desc), + DataFusionError::ArrowError(ref desc) => write!(f, "Arrow error: {desc}"), #[cfg(feature = "parquet")] DataFusionError::ParquetError(ref desc) => { - write!(f, "Parquet error: {}", desc) + write!(f, "Parquet error: {desc}") } #[cfg(feature = "avro")] DataFusionError::AvroError(ref desc) => { write!(f, "Avro error: {}", desc) } - DataFusionError::IoError(ref desc) => write!(f, "IO error: {}", desc), + DataFusionError::IoError(ref desc) => write!(f, "IO error: {desc}"), DataFusionError::SQL(ref desc) => { - write!(f, "SQL error: {:?}", desc) + write!(f, "SQL error: {desc:?}") } DataFusionError::NotImplemented(ref desc) => { - write!(f, "This feature is not implemented: {}", desc) + write!(f, "This feature is not implemented: {desc}") } DataFusionError::Internal(ref desc) => { - write!(f, "Internal error: {}. This was likely caused by a bug in DataFusion's \ - code and we would welcome that you file an bug report in our issue tracker", desc) + write!(f, "Internal error: {desc}. This was likely caused by a bug in DataFusion's \ + code and we would welcome that you file an bug report in our issue tracker") } DataFusionError::Plan(ref desc) => { - write!(f, "Error during planning: {}", desc) + write!(f, "Error during planning: {desc}") } DataFusionError::SchemaError(ref desc) => { - write!(f, "Schema error: {}", desc) + write!(f, "Schema error: {desc}") } DataFusionError::Execution(ref desc) => { - write!(f, "Execution error: {}", desc) + write!(f, "Execution error: {desc}") } DataFusionError::ResourcesExhausted(ref desc) => { - write!(f, "Resources exhausted: {}", desc) + write!(f, "Resources exhausted: {desc}") } DataFusionError::External(ref desc) => { - write!(f, "External error: {}", desc) + write!(f, "External error: {desc}") } #[cfg(feature = "jit")] DataFusionError::JITError(ref desc) => { - write!(f, "JIT error: {}", desc) + write!(f, "JIT error: {desc}") } #[cfg(feature = "object_store")] DataFusionError::ObjectStore(ref desc) => { - write!(f, "Object Store error: {}", desc) + write!(f, "Object Store error: {desc}") } DataFusionError::Context(ref desc, ref err) => { write!(f, "{}\ncaused by\n{}", desc, *err) diff --git a/datafusion/common/src/parsers.rs b/datafusion/common/src/parsers.rs index aced7c78a3fa..1c31d61d143c 100644 --- a/datafusion/common/src/parsers.rs +++ b/datafusion/common/src/parsers.rs @@ -49,8 +49,7 @@ impl FromStr for CompressionTypeVariant { "XZ" => Ok(Self::XZ), "" => Ok(Self::UNCOMPRESSED), _ => Err(ParserError::ParserError(format!( - "Unsupported file compression type {}", - s + "Unsupported file compression type {s}" ))), } } @@ -105,8 +104,7 @@ impl FromStr for IntervalType { "second" | "seconds" => Ok(Self::Second), "millisecond" | "milliseconds" => Ok(Self::Millisecond), _ => Err(DataFusionError::NotImplemented(format!( - "Unknown interval type: {}", - s + "Unknown interval type: {s}" ))), } } @@ -143,31 +141,27 @@ pub fn parse_interval(leading_field: &str, value: &str) -> Result { Ok(n) => n, Err(_) => { return Err(DataFusionError::NotImplemented(format!( - "Unsupported Interval Expression with value {:?}", - value + "Unsupported Interval Expression with value {value:?}" ))); } }; if interval_period > (i64::MAX as f64) { return Err(DataFusionError::NotImplemented(format!( - "Interval field value out of range: {:?}", - value + "Interval field value out of range: {value:?}" ))); } let it = IntervalType::from_str(interval_type).map_err(|_| { DataFusionError::NotImplemented(format!( - "Invalid input syntax for type interval: {:?}", - value + "Invalid input syntax for type interval: {value:?}" )) })?; // Disallow duplicate interval types if used_interval_types & (it as u16) != 0 { return Err(DataFusionError::SQL(ParserError::ParserError(format!( - "Invalid input syntax for type interval: {:?}. Repeated type '{}'", - value, interval_type + "Invalid input syntax for type interval: {value:?}. Repeated type '{interval_type}'" )))); } else { used_interval_types |= it as u16; @@ -220,8 +214,7 @@ pub fn parse_interval(leading_field: &str, value: &str) -> Result { if result_month > (i32::MAX as i64) { return Err(DataFusionError::NotImplemented(format!( - "Interval field value out of range: {:?}", - value + "Interval field value out of range: {value:?}" ))); } @@ -229,8 +222,7 @@ pub fn parse_interval(leading_field: &str, value: &str) -> Result { if result_days > (i32::MAX as i64) { return Err(DataFusionError::NotImplemented(format!( - "Interval field value out of range: {:?}", - value + "Interval field value out of range: {value:?}" ))); } @@ -238,8 +230,7 @@ pub fn parse_interval(leading_field: &str, value: &str) -> Result { if result_nanos > (i64::MAX as i128) { return Err(DataFusionError::NotImplemented(format!( - "Interval field value out of range: {:?}", - value + "Interval field value out of range: {value:?}" ))); } } @@ -394,7 +385,7 @@ mod test { .expect_err("parsing interval should have failed"); assert_eq!( r#"SQL(ParserError("Invalid input syntax for type interval: \"1 month 1 second 1 second\". Repeated type 'second'"))"#, - format!("{:?}", err) + format!("{err:?}") ); } } diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 96bf7f5ed8c8..11a369fcf307 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -598,8 +598,7 @@ fn do_date_time_math( ) -> Result { let prior = NaiveDateTime::from_timestamp_opt(secs, nsecs).ok_or_else(|| { DataFusionError::Internal(format!( - "Could not conert to NaiveDateTime: secs {} nsecs {} scalar {:?} sign {}", - secs, nsecs, scalar, sign + "Could not conert to NaiveDateTime: secs {secs} nsecs {nsecs} scalar {scalar:?} sign {sign}" )) })?; do_date_math(prior, scalar, sign) @@ -614,8 +613,7 @@ where ScalarValue::IntervalYearMonth(Some(i)) => shift_months(prior, *i * sign), ScalarValue::IntervalMonthDayNano(Some(i)) => add_m_d_nano(prior, *i, sign), other => Err(DataFusionError::Execution(format!( - "DateIntervalExpr does not support non-interval type {:?}", - other + "DateIntervalExpr does not support non-interval type {other:?}" )))?, }) } @@ -986,8 +984,7 @@ impl ScalarValue { return Ok(ScalarValue::Decimal128(Some(value), precision, scale)); } Err(DataFusionError::Internal(format!( - "Can not new a decimal type ScalarValue for precision {} and scale {}", - precision, scale + "Can not new a decimal type ScalarValue for precision {precision} and scale {scale}" ))) } @@ -1100,8 +1097,7 @@ impl ScalarValue { Ok(ScalarValue::Decimal128(Some(-v), *precision, *scale)) } value => Err(DataFusionError::Internal(format!( - "Can not run arithmetic negative on scalar value {:?}", - value + "Can not run arithmetic negative on scalar value {value:?}" ))), } } @@ -1511,8 +1507,7 @@ impl ScalarValue { }; } else { return Err(DataFusionError::Internal(format!( - "Expected Struct but found: {}", - scalar + "Expected Struct but found: {scalar}" ))); }; } @@ -1537,13 +1532,12 @@ impl ScalarValue { if &inner_key_type == key_type { Ok(*scalar) } else { - panic!("Expected inner key type of {} but found: {}, value was ({:?})", key_type, inner_key_type, scalar); + panic!("Expected inner key type of {key_type} but found: {inner_key_type}, value was ({scalar:?})"); } } _ => { Err(DataFusionError::Internal(format!( - "Expected scalar of type {} but found: {} {:?}", - value_type, scalar, scalar + "Expected scalar of type {value_type} but found: {scalar} {scalar:?}" ))) } }) @@ -1572,8 +1566,7 @@ impl ScalarValue { } else { Err(DataFusionError::Internal(format!( "Inconsistent types in ScalarValue::iter_to_array. \ - Expected {:?}, got {:?}", - data_type, sv + Expected {data_type:?}, got {sv:?}" ))) } }) @@ -1677,8 +1670,7 @@ impl ScalarValue { } } else { return Err(DataFusionError::Internal(format!( - "Expected ScalarValue::List element. Received {:?}", - scalar + "Expected ScalarValue::List element. Received {scalar:?}" ))); } } @@ -2141,8 +2133,7 @@ impl ScalarValue { } other => { return Err(DataFusionError::NotImplemented(format!( - "Can't create a scalar from array of type \"{:?}\"", - other + "Can't create a scalar from array of type \"{other:?}\"" ))); } }) @@ -2624,8 +2615,7 @@ impl TryFrom<&DataType> for ScalarValue { DataType::Null => ScalarValue::Null, _ => { return Err(DataFusionError::NotImplemented(format!( - "Can't create a scalar from data_type \"{:?}\"", - datatype + "Can't create a scalar from data_type \"{datatype:?}\"" ))); } }) @@ -2645,7 +2635,7 @@ impl fmt::Display for ScalarValue { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { ScalarValue::Decimal128(v, p, s) => { - write!(f, "{:?},{:?},{:?}", v, p, s)?; + write!(f, "{v:?},{p:?},{s:?}")?; } ScalarValue::Boolean(e) => format_option!(f, e)?, ScalarValue::Float32(e) => format_option!(f, e)?, @@ -2669,7 +2659,7 @@ impl fmt::Display for ScalarValue { f, "{}", l.iter() - .map(|v| format!("{}", v)) + .map(|v| format!("{v}")) .collect::>() .join(",") )?, @@ -2680,7 +2670,7 @@ impl fmt::Display for ScalarValue { f, "{}", l.iter() - .map(|v| format!("{}", v)) + .map(|v| format!("{v}")) .collect::>() .join(",") )?, @@ -2691,7 +2681,7 @@ impl fmt::Display for ScalarValue { f, "{}", l.iter() - .map(|v| format!("{}", v)) + .map(|v| format!("{v}")) .collect::>() .join(",") )?, @@ -2702,7 +2692,7 @@ impl fmt::Display for ScalarValue { f, "{}", l.iter() - .map(|v| format!("{}", v)) + .map(|v| format!("{v}")) .collect::>() .join(",") )?, @@ -2729,7 +2719,7 @@ impl fmt::Display for ScalarValue { )?, None => write!(f, "NULL")?, }, - ScalarValue::Dictionary(_k, v) => write!(f, "{}", v)?, + ScalarValue::Dictionary(_k, v) => write!(f, "{v}")?, ScalarValue::Null => write!(f, "NULL")?, }; Ok(()) @@ -2739,65 +2729,65 @@ impl fmt::Display for ScalarValue { impl fmt::Debug for ScalarValue { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { - ScalarValue::Decimal128(_, _, _) => write!(f, "Decimal128({})", self), - ScalarValue::Boolean(_) => write!(f, "Boolean({})", self), - ScalarValue::Float32(_) => write!(f, "Float32({})", self), - ScalarValue::Float64(_) => write!(f, "Float64({})", self), - ScalarValue::Int8(_) => write!(f, "Int8({})", self), - ScalarValue::Int16(_) => write!(f, "Int16({})", self), - ScalarValue::Int32(_) => write!(f, "Int32({})", self), - ScalarValue::Int64(_) => write!(f, "Int64({})", self), - ScalarValue::UInt8(_) => write!(f, "UInt8({})", self), - ScalarValue::UInt16(_) => write!(f, "UInt16({})", self), - ScalarValue::UInt32(_) => write!(f, "UInt32({})", self), - ScalarValue::UInt64(_) => write!(f, "UInt64({})", self), + ScalarValue::Decimal128(_, _, _) => write!(f, "Decimal128({self})"), + ScalarValue::Boolean(_) => write!(f, "Boolean({self})"), + ScalarValue::Float32(_) => write!(f, "Float32({self})"), + ScalarValue::Float64(_) => write!(f, "Float64({self})"), + ScalarValue::Int8(_) => write!(f, "Int8({self})"), + ScalarValue::Int16(_) => write!(f, "Int16({self})"), + ScalarValue::Int32(_) => write!(f, "Int32({self})"), + ScalarValue::Int64(_) => write!(f, "Int64({self})"), + ScalarValue::UInt8(_) => write!(f, "UInt8({self})"), + ScalarValue::UInt16(_) => write!(f, "UInt16({self})"), + ScalarValue::UInt32(_) => write!(f, "UInt32({self})"), + ScalarValue::UInt64(_) => write!(f, "UInt64({self})"), ScalarValue::TimestampSecond(_, tz_opt) => { - write!(f, "TimestampSecond({}, {:?})", self, tz_opt) + write!(f, "TimestampSecond({self}, {tz_opt:?})") } ScalarValue::TimestampMillisecond(_, tz_opt) => { - write!(f, "TimestampMillisecond({}, {:?})", self, tz_opt) + write!(f, "TimestampMillisecond({self}, {tz_opt:?})") } ScalarValue::TimestampMicrosecond(_, tz_opt) => { - write!(f, "TimestampMicrosecond({}, {:?})", self, tz_opt) + write!(f, "TimestampMicrosecond({self}, {tz_opt:?})") } ScalarValue::TimestampNanosecond(_, tz_opt) => { - write!(f, "TimestampNanosecond({}, {:?})", self, tz_opt) - } - ScalarValue::Utf8(None) => write!(f, "Utf8({})", self), - ScalarValue::Utf8(Some(_)) => write!(f, "Utf8(\"{}\")", self), - ScalarValue::LargeUtf8(None) => write!(f, "LargeUtf8({})", self), - ScalarValue::LargeUtf8(Some(_)) => write!(f, "LargeUtf8(\"{}\")", self), - ScalarValue::Binary(None) => write!(f, "Binary({})", self), - ScalarValue::Binary(Some(_)) => write!(f, "Binary(\"{}\")", self), + write!(f, "TimestampNanosecond({self}, {tz_opt:?})") + } + ScalarValue::Utf8(None) => write!(f, "Utf8({self})"), + ScalarValue::Utf8(Some(_)) => write!(f, "Utf8(\"{self}\")"), + ScalarValue::LargeUtf8(None) => write!(f, "LargeUtf8({self})"), + ScalarValue::LargeUtf8(Some(_)) => write!(f, "LargeUtf8(\"{self}\")"), + ScalarValue::Binary(None) => write!(f, "Binary({self})"), + ScalarValue::Binary(Some(_)) => write!(f, "Binary(\"{self}\")"), ScalarValue::FixedSizeBinary(size, None) => { - write!(f, "FixedSizeBinary({}, {})", size, self) + write!(f, "FixedSizeBinary({size}, {self})") } ScalarValue::FixedSizeBinary(size, Some(_)) => { - write!(f, "FixedSizeBinary({}, \"{}\")", size, self) - } - ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({})", self), - ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{}\")", self), - ScalarValue::List(_, _) => write!(f, "List([{}])", self), - ScalarValue::Date32(_) => write!(f, "Date32(\"{}\")", self), - ScalarValue::Date64(_) => write!(f, "Date64(\"{}\")", self), - ScalarValue::Time32Second(_) => write!(f, "Time32Second(\"{}\")", self), + write!(f, "FixedSizeBinary({size}, \"{self}\")") + } + ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"), + ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{self}\")"), + ScalarValue::List(_, _) => write!(f, "List([{self}])"), + ScalarValue::Date32(_) => write!(f, "Date32(\"{self}\")"), + ScalarValue::Date64(_) => write!(f, "Date64(\"{self}\")"), + ScalarValue::Time32Second(_) => write!(f, "Time32Second(\"{self}\")"), ScalarValue::Time32Millisecond(_) => { - write!(f, "Time32Millisecond(\"{}\")", self) + write!(f, "Time32Millisecond(\"{self}\")") } ScalarValue::Time64Microsecond(_) => { - write!(f, "Time64Microsecond(\"{}\")", self) + write!(f, "Time64Microsecond(\"{self}\")") } ScalarValue::Time64Nanosecond(_) => { - write!(f, "Time64Nanosecond(\"{}\")", self) + write!(f, "Time64Nanosecond(\"{self}\")") } ScalarValue::IntervalDayTime(_) => { - write!(f, "IntervalDayTime(\"{}\")", self) + write!(f, "IntervalDayTime(\"{self}\")") } ScalarValue::IntervalYearMonth(_) => { - write!(f, "IntervalYearMonth(\"{}\")", self) + write!(f, "IntervalYearMonth(\"{self}\")") } ScalarValue::IntervalMonthDayNano(_) => { - write!(f, "IntervalMonthDayNano(\"{}\")", self) + write!(f, "IntervalMonthDayNano(\"{self}\")") } ScalarValue::Struct(e, fields) => { // Use Debug representation of field values @@ -2814,7 +2804,7 @@ impl fmt::Debug for ScalarValue { None => write!(f, "Struct(NULL)"), } } - ScalarValue::Dictionary(k, v) => write!(f, "Dictionary({:?}, {:?})", k, v), + ScalarValue::Dictionary(k, v) => write!(f, "Dictionary({k:?}, {v:?})"), ScalarValue::Null => write!(f, "NULL"), } } @@ -3512,16 +3502,13 @@ mod tests { println!("**** Test Case *****"); let TestCase { array, scalars } = case; println!("Input array type: {}", array.data_type()); - println!("Input scalars: {:#?}", scalars); + println!("Input scalars: {scalars:#?}"); assert_eq!(array.len(), scalars.len()); for (index, scalar) in scalars.into_iter().enumerate() { assert!( scalar.eq_array(&array, index), - "Expected {:?} to be equal to {:?} at index {}", - scalar, - array, - index + "Expected {scalar:?} to be equal to {array:?} at index {index}" ); // test that all other elements are *not* equal @@ -3529,10 +3516,7 @@ mod tests { if index != other_index { assert!( !scalar.eq_array(&array, other_index), - "Expected {:?} to be NOT equal to {:?} at index {}", - scalar, - array, - other_index + "Expected {scalar:?} to be NOT equal to {array:?} at index {other_index}" ); } } @@ -3669,13 +3653,13 @@ mod tests { // Check Display assert_eq!( - format!("{}", scalar), + format!("{scalar}"), String::from("{A:23,B:false,C:Hello,D:{e:2,f:3}}") ); // Check Debug assert_eq!( - format!("{:?}", scalar), + format!("{scalar:?}"), String::from( r#"Struct({A:Int32(23),B:Boolean(false),C:Utf8("Hello"),D:Struct({e:Int16(2),f:Int64(3)})})"# ) @@ -3721,7 +3705,7 @@ mod tests { // None version let none_scalar = ScalarValue::try_from(array.data_type()).unwrap(); assert!(none_scalar.is_null()); - assert_eq!(format!("{:?}", none_scalar), String::from("Struct(NULL)")); + assert_eq!(format!("{none_scalar:?}"), String::from("Struct(NULL)")); // Construct with convenience From> let constructed = ScalarValue::from(vec![ diff --git a/datafusion/common/src/table_reference.rs b/datafusion/common/src/table_reference.rs index d8ad835caec4..e547d7c03a60 100644 --- a/datafusion/common/src/table_reference.rs +++ b/datafusion/common/src/table_reference.rs @@ -101,7 +101,7 @@ impl OwnedTableReference { impl std::fmt::Display for OwnedTableReference { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { - OwnedTableReference::Bare { table } => write!(f, "{}", table), + OwnedTableReference::Bare { table } => write!(f, "{table}"), OwnedTableReference::Partial { schema, table } => { write!(f, "{schema}.{table}") } diff --git a/datafusion/core/benches/merge.rs b/datafusion/core/benches/merge.rs index 0b993ca64441..a7ac6cd41dac 100644 --- a/datafusion/core/benches/merge.rs +++ b/datafusion/core/benches/merge.rs @@ -459,7 +459,7 @@ impl DataGenerator { /// array of low cardinality (100 distinct) values fn utf8_low_cardinality_values(&mut self) -> Vec>> { let strings = (0..100) - .map(|s| format!("value{}", s).into()) + .map(|s| format!("value{s}").into()) .collect::>(); // pick from the 100 strings randomly diff --git a/datafusion/core/benches/parquet_query_sql.rs b/datafusion/core/benches/parquet_query_sql.rs index e9204f86e52c..9b0d809629d2 100644 --- a/datafusion/core/benches/parquet_query_sql.rs +++ b/datafusion/core/benches/parquet_query_sql.rs @@ -99,9 +99,7 @@ fn generate_string_dictionary( valid_percent: f64, ) -> ArrayRef { let mut rng = thread_rng(); - let strings: Vec<_> = (0..cardinality) - .map(|x| format!("{}#{}", prefix, x)) - .collect(); + let strings: Vec<_> = (0..cardinality).map(|x| format!("{prefix}#{x}")).collect(); Arc::new(DictionaryArray::::from_iter((0..len).map( |_| { @@ -192,7 +190,7 @@ fn criterion_benchmark(c: &mut Criterion) { }; assert!(Path::new(&file_path).exists(), "path not found"); - println!("Using parquet file {}", file_path); + println!("Using parquet file {file_path}"); let partitions = 4; let config = SessionConfig::new().with_target_partitions(partitions); @@ -230,7 +228,7 @@ fn criterion_benchmark(c: &mut Criterion) { continue; } - c.bench_function(&format!("tokio: {}", query), |b| { + c.bench_function(&format!("tokio: {query}"), |b| { b.iter(|| { let query = query.clone(); let context = context.clone(); @@ -252,7 +250,7 @@ fn criterion_benchmark(c: &mut Criterion) { }); }); - c.bench_function(&format!("scheduled: {}", query), |b| { + c.bench_function(&format!("scheduled: {query}"), |b| { b.iter(|| { let query = query.clone(); let context = context.clone(); diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index ebf36ec56e5e..59fb726a3c13 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -97,9 +97,9 @@ fn batches( j = i; } - col_a.push(Some(format!("a-{:?}", j))); - col_b.push(Some(format!("b-{:?}", j))); - col_c.push(Some(format!("c-{:?}", j))); + col_a.push(Some(format!("a-{j:?}"))); + col_b.push(Some(format!("b-{j:?}"))); + col_c.push(Some(format!("c-{j:?}"))); col_d.push(Some((i + curr_batch_offset) as i64)); current_rows_per_sort_key += 1; diff --git a/datafusion/core/benches/sort_limit_query_sql.rs b/datafusion/core/benches/sort_limit_query_sql.rs index efee5de13274..62160067143e 100644 --- a/datafusion/core/benches/sort_limit_query_sql.rs +++ b/datafusion/core/benches/sort_limit_query_sql.rs @@ -65,7 +65,7 @@ fn create_context() -> Arc> { let testdata = datafusion::test_util::arrow_test_data(); - let path = format!("{}/csv/aggregate_test_100.csv", testdata); + let path = format!("{testdata}/csv/aggregate_test_100.csv"); let table_path = ListingTableUrl::parse(path).unwrap(); // create CSV data source diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index f949f30b912a..1feba8045a49 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -38,7 +38,7 @@ fn plan(ctx: Arc>, sql: &str) { /// Create schema representing a large table pub fn create_schema(column_prefix: &str) -> Schema { let fields = (0..200) - .map(|i| Field::new(format!("{}{}", column_prefix, i), DataType::Int32, true)) + .map(|i| Field::new(format!("{column_prefix}{i}"), DataType::Int32, true)) .collect(); Schema::new(fields) } diff --git a/datafusion/core/src/bin/print_config_docs.rs b/datafusion/core/src/bin/print_config_docs.rs index 6e66fea5fe51..0a6415c62766 100644 --- a/datafusion/core/src/bin/print_config_docs.rs +++ b/datafusion/core/src/bin/print_config_docs.rs @@ -19,5 +19,5 @@ use datafusion::config::BuiltInConfigs; fn main() { let docs = BuiltInConfigs::generate_config_markdown(); - println!("{}", docs); + println!("{docs}"); } diff --git a/datafusion/core/src/catalog/information_schema.rs b/datafusion/core/src/catalog/information_schema.rs index 96e5585e0fb4..ec86acae3925 100644 --- a/datafusion/core/src/catalog/information_schema.rs +++ b/datafusion/core/src/catalog/information_schema.rs @@ -511,7 +511,7 @@ impl InformationSchemaColumnsBuilder { self.is_nullables.append_value(nullable_str); // "System supplied type" --> Use debug format of the datatype - self.data_types.append_value(format!("{:?}", data_type)); + self.data_types.append_value(format!("{data_type:?}")); // "If data_type identifies a character or bit string type, the // declared maximum length; null for all other data types or diff --git a/datafusion/core/src/catalog/schema.rs b/datafusion/core/src/catalog/schema.rs index df0ef78807b1..41187c62965b 100644 --- a/datafusion/core/src/catalog/schema.rs +++ b/datafusion/core/src/catalog/schema.rs @@ -108,8 +108,7 @@ impl SchemaProvider for MemorySchemaProvider { ) -> Result>> { if self.table_exist(name.as_str()) { return Err(DataFusionError::Execution(format!( - "The table {} already exists", - name + "The table {name} already exists" ))); } Ok(self.tables.insert(name, table)) @@ -160,9 +159,9 @@ mod tests { async fn test_schema_register_listing_table() { let testdata = crate::test_util::parquet_test_data(); let testdir = if testdata.starts_with('/') { - format!("file://{}", testdata) + format!("file://{testdata}") } else { - format!("file:///{}", testdata) + format!("file:///{testdata}") }; let filename = if testdir.ends_with('/') { format!("{}{}", testdir, "alltypes_plain.parquet") diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 976b8c07376a..d6504c3d0f0a 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -317,13 +317,13 @@ impl BuiltInConfigs { small batches will be coalesced into larger batches. This is helpful when there \ are highly selective filters or joins that could produce tiny output batches. The \ target batch size is determined by the configuration setting \ - '{}'.", OPT_COALESCE_TARGET_BATCH_SIZE), + '{OPT_COALESCE_TARGET_BATCH_SIZE}'."), true, ), ConfigDefinition::new_u64( OPT_COALESCE_TARGET_BATCH_SIZE, format!("Target batch size when coalescing batches. Uses in conjunction with the \ - configuration setting '{}'.", OPT_COALESCE_BATCHES), + configuration setting '{OPT_COALESCE_BATCHES}'."), 4096, ), ConfigDefinition::new_string( diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 12267ec1e0d3..f7241cb966cd 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -1148,7 +1148,7 @@ mod tests { /// Compare the formatted string representation of two plans for equality fn assert_same_plan(plan1: &LogicalPlan, plan2: &LogicalPlan) { - assert_eq!(format!("{:?}", plan1), format!("{:?}", plan2)); + assert_eq!(format!("{plan1:?}"), format!("{plan2:?}")); } /// Create a logical plan from a SQL query @@ -1176,7 +1176,7 @@ mod tests { let testdata = test_util::arrow_test_data(); ctx.register_csv( table_name, - &format!("{}/csv/aggregate_test_100.csv", testdata), + &format!("{testdata}/csv/aggregate_test_100.csv"), CsvReadOptions::new().schema(schema.as_ref()), ) .await?; diff --git a/datafusion/core/src/datasource/file_format/file_type.rs b/datafusion/core/src/datasource/file_format/file_type.rs index 3aa802e45189..1a9973c68a64 100644 --- a/datafusion/core/src/datasource/file_format/file_type.rs +++ b/datafusion/core/src/datasource/file_format/file_type.rs @@ -78,7 +78,7 @@ impl FromStr for FileCompressionType { fn from_str(s: &str) -> Result { let variant = CompressionTypeVariant::from_str(s).map_err(|_| { - DataFusionError::NotImplemented(format!("Unknown FileCompressionType: {}", s)) + DataFusionError::NotImplemented(format!("Unknown FileCompressionType: {s}")) })?; Ok(Self { variant }) } @@ -208,8 +208,7 @@ impl FromStr for FileType { "CSV" => Ok(FileType::CSV), "JSON" | "NDJSON" => Ok(FileType::JSON), _ => Err(DataFusionError::NotImplemented(format!( - "Unknown FileType: {}", - s + "Unknown FileType: {s}" ))), } } diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index b73b3881a68f..f49db0b20b62 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -111,7 +111,7 @@ pub(crate) mod test_util { limit: Option, ) -> Result> { let store = Arc::new(LocalFileSystem::new()) as _; - let meta = local_unpartitioned_file(format!("{}/{}", store_root, file_name)); + let meta = local_unpartitioned_file(format!("{store_root}/{file_name}")); let file_schema = format.infer_schema(state, &store, &[meta.clone()]).await?; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 9f238a43cab8..cc9661e95215 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -993,7 +993,7 @@ mod tests { assert_eq!( "[true, false, true, false, true, false, true, false]", - format!("{:?}", values) + format!("{values:?}") ); Ok(()) @@ -1018,7 +1018,7 @@ mod tests { values.push(array.value(i)); } - assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{:?}", values)); + assert_eq!("[4, 5, 6, 7, 2, 3, 0, 1]", format!("{values:?}")); Ok(()) } @@ -1042,7 +1042,7 @@ mod tests { values.push(array.value(i)); } - assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{:?}", values)); + assert_eq!("[1235865600000000000, 1235865660000000000, 1238544000000000000, 1238544060000000000, 1233446400000000000, 1233446460000000000, 1230768000000000000, 1230768060000000000]", format!("{values:?}")); Ok(()) } @@ -1068,7 +1068,7 @@ mod tests { assert_eq!( "[0.0, 1.1, 0.0, 1.1, 0.0, 1.1, 0.0, 1.1]", - format!("{:?}", values) + format!("{values:?}") ); Ok(()) @@ -1095,7 +1095,7 @@ mod tests { assert_eq!( "[0.0, 10.1, 0.0, 10.1, 0.0, 10.1, 0.0, 10.1]", - format!("{:?}", values) + format!("{values:?}") ); Ok(()) @@ -1122,7 +1122,7 @@ mod tests { assert_eq!( "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", - format!("{:?}", values) + format!("{values:?}") ); Ok(()) @@ -1176,7 +1176,7 @@ mod tests { #[tokio::test] async fn test_read_parquet_page_index() -> Result<()> { let testdata = crate::test_util::parquet_test_data(); - let path = format!("{}/alltypes_tiny_pages.parquet", testdata); + let path = format!("{testdata}/alltypes_tiny_pages.parquet"); let file = File::open(path).await.unwrap(); let options = ArrowReaderOptions::new().with_page_index(true); let builder = @@ -1187,7 +1187,7 @@ mod tests { .clone(); check_page_index_validation(builder.page_indexes(), builder.offset_indexes()); - let path = format!("{}/alltypes_tiny_pages_plain.parquet", testdata); + let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet"); let file = File::open(path).await.unwrap(); let builder = ParquetRecordBatchStreamBuilder::new_with_options(file, options) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 2c014068dff8..fa7f7070c3bc 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -365,13 +365,11 @@ fn batches_to_paths(batches: &[RecordBatch]) -> Result> { fn to_timestamp_millis(v: i64) -> Result> { match Utc.timestamp_millis_opt(v) { chrono::LocalResult::None => Err(DataFusionError::Execution(format!( - "Can not convert {} to UTC millisecond timestamp", - v + "Can not convert {v} to UTC millisecond timestamp" ))), chrono::LocalResult::Single(v) => Ok(v), chrono::LocalResult::Ambiguous(_, _) => Err(DataFusionError::Execution(format!( - "Ambiguous timestamp when converting {} to UTC millisecond timestamp", - v + "Ambiguous timestamp when converting {v} to UTC millisecond timestamp" ))), } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 99d009b34aae..954e7bf6b098 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -109,7 +109,7 @@ impl ListingTableConfig { } fn infer_format(path: &str) -> Result<(Arc, String)> { - let err_msg = format!("Unable to infer file type from path: {}", path); + let err_msg = format!("Unable to infer file type from path: {path}"); let mut exts = path.rsplit('.'); @@ -529,12 +529,12 @@ impl ListingTable { } else { Err(DataFusionError::Plan( - format!("Only support single column references in output_ordering, got {:?}", expr) + format!("Only support single column references in output_ordering, got {expr:?}") )) } } else { Err(DataFusionError::Plan( - format!("Expected Expr::Sort in output_ordering, but got {:?}", expr) + format!("Expected Expr::Sort in output_ordering, but got {expr:?}") )) } }) @@ -910,8 +910,7 @@ mod tests { } (expected_result, ordering_result) => { panic!( - "expected: {:#?}\n\nactual:{:#?}", - expected_result, ordering_result + "expected: {expected_result:#?}\n\nactual:{ordering_result:#?}" ); } } @@ -1193,7 +1192,7 @@ mod tests { name: &str, ) -> Result> { let testdata = crate::test_util::parquet_test_data(); - let filename = format!("{}/{}", testdata, name); + let filename = format!("{testdata}/{name}"); let table_path = ListingTableUrl::parse(filename).unwrap(); let config = ListingTableConfig::new(table_path) diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 0cb08ba31b0c..798359208ffc 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -104,7 +104,7 @@ impl ListingTableUrl { } else { Url::from_file_path(path) } - .map_err(|_| DataFusionError::Internal(format!("Can not open path: {}", s)))?; + .map_err(|_| DataFusionError::Internal(format!("Can not open path: {s}")))?; // TODO: Currently we do not have an IO-related error variant that accepts () // or a string. Once we have such a variant, change the error type above. @@ -283,8 +283,7 @@ mod tests { assert_eq!( split_glob_expression(input), expected, - "testing split_glob_expression with {}", - input + "testing split_glob_expression with {input}" ); } diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 52b4abf186e4..0b7cd6d5b185 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -257,10 +257,10 @@ mod tests { Err(DataFusionError::ArrowError(ArrowError::SchemaError(e))) => { assert_eq!( "\"project index 4 out of bounds, max field 3\"", - format!("{:?}", e) + format!("{e:?}") ) } - res => panic!("Scan should failed on invalid projection, got {:?}", res), + res => panic!("Scan should failed on invalid projection, got {res:?}"), }; Ok(()) @@ -290,10 +290,9 @@ mod tests { )?; match MemTable::try_new(schema2, vec![vec![batch]]) { - Err(DataFusionError::Plan(e)) => assert_eq!( - "\"Mismatch between schema and batches\"", - format!("{:?}", e) - ), + Err(DataFusionError::Plan(e)) => { + assert_eq!("\"Mismatch between schema and batches\"", format!("{e:?}")) + } _ => panic!("MemTable::new should have failed due to schema mismatch"), } @@ -322,10 +321,9 @@ mod tests { )?; match MemTable::try_new(schema2, vec![vec![batch]]) { - Err(DataFusionError::Plan(e)) => assert_eq!( - "\"Mismatch between schema and batches\"", - format!("{:?}", e) - ), + Err(DataFusionError::Plan(e)) => { + assert_eq!("\"Mismatch between schema and batches\"", format!("{e:?}")) + } _ => panic!("MemTable::new should have failed due to schema mismatch"), } diff --git a/datafusion/core/src/datasource/object_store.rs b/datafusion/core/src/datasource/object_store.rs index b6301f96bf3a..ef48224506b5 100644 --- a/datafusion/core/src/datasource/object_store.rs +++ b/datafusion/core/src/datasource/object_store.rs @@ -41,8 +41,7 @@ impl ObjectStoreUrl { let remaining = &parsed[url::Position::BeforePath..]; if !remaining.is_empty() && remaining != "/" { return Err(DataFusionError::Execution(format!( - "ObjectStoreUrl must only contain scheme and authority, got: {}", - remaining + "ObjectStoreUrl must only contain scheme and authority, got: {remaining}" ))); } @@ -208,8 +207,7 @@ impl ObjectStoreRegistry { Ok(store) } None => Err(DataFusionError::Internal(format!( - "No suitable object store found for {}", - url + "No suitable object store found for {url}" ))), }, } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 323e50ebd8cd..f5814778b943 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -309,8 +309,7 @@ impl SessionContext { self.return_empty_dataframe() } (false, false, Ok(_)) => Err(DataFusionError::Execution(format!( - "Table '{}' already exists", - name + "Table '{name}' already exists" ))), } } @@ -340,8 +339,7 @@ impl SessionContext { self.return_empty_dataframe() } (false, Ok(_)) => Err(DataFusionError::Execution(format!( - "Table '{}' already exists", - name + "Table '{name}' already exists" ))), } } @@ -354,8 +352,7 @@ impl SessionContext { (Ok(true), _) => self.return_empty_dataframe(), (_, true) => self.return_empty_dataframe(), (_, _) => Err(DataFusionError::Execution(format!( - "Table '{}' doesn't exist.", - name + "Table '{name}' doesn't exist." ))), } } @@ -368,8 +365,7 @@ impl SessionContext { (Ok(true), _) => self.return_empty_dataframe(), (_, true) => self.return_empty_dataframe(), (_, _) => Err(DataFusionError::Execution(format!( - "View '{}' doesn't exist.", - name + "View '{name}' doesn't exist." ))), } } @@ -382,8 +378,7 @@ impl SessionContext { let old_value = config_options.get(&variable).ok_or_else(|| { DataFusionError::Execution(format!( - "Can not SET variable: Unknown Variable {}", - variable + "Can not SET variable: Unknown Variable {variable}" )) })?; @@ -391,8 +386,7 @@ impl SessionContext { ScalarValue::Boolean(_) => { let new_value = value.parse::().map_err(|_| { DataFusionError::Execution(format!( - "Failed to parse {} as bool", - value, + "Failed to parse {value} as bool", )) })?; config_options.set_bool(&variable, new_value); @@ -401,8 +395,7 @@ impl SessionContext { ScalarValue::UInt64(_) => { let new_value = value.parse::().map_err(|_| { DataFusionError::Execution(format!( - "Failed to parse {} as u64", - value, + "Failed to parse {value} as u64", )) })?; config_options.set_u64(&variable, new_value); @@ -411,8 +404,7 @@ impl SessionContext { ScalarValue::Utf8(_) => { let new_value = value.parse::().map_err(|_| { DataFusionError::Execution(format!( - "Failed to parse {} as String", - value, + "Failed to parse {value} as String", )) })?; config_options.set_string(&variable, new_value); @@ -441,14 +433,12 @@ impl SessionContext { 1 => Ok((DEFAULT_CATALOG, schema_name.as_str())), 2 => Ok((tokens[0], tokens[1])), _ => Err(DataFusionError::Execution(format!( - "Unable to parse catalog from {}", - schema_name + "Unable to parse catalog from {schema_name}" ))), }?; let catalog = self.catalog(catalog).ok_or_else(|| { DataFusionError::Execution(format!( - "Missing '{}' catalog", - DEFAULT_CATALOG + "Missing '{DEFAULT_CATALOG}' catalog" )) })?; @@ -462,8 +452,7 @@ impl SessionContext { self.return_empty_dataframe() } (false, Some(_)) => Err(DataFusionError::Execution(format!( - "Schema '{}' already exists", - schema_name + "Schema '{schema_name}' already exists" ))), } } @@ -485,8 +474,7 @@ impl SessionContext { self.return_empty_dataframe() } (false, Some(_)) => Err(DataFusionError::Execution(format!( - "Catalog '{}' already exists", - catalog_name + "Catalog '{catalog_name}' already exists" ))), } } @@ -1388,7 +1376,7 @@ impl SessionConfig { let mut map = HashMap::new(); // copy configs from config_options for (k, v) in self.config_options.options() { - map.insert(k.to_string(), format!("{}", v)); + map.insert(k.to_string(), format!("{v}")); } map.insert( TARGET_PARTITIONS.to_owned(), @@ -1933,8 +1921,7 @@ impl FunctionRegistry for SessionState { result.cloned().ok_or_else(|| { DataFusionError::Plan(format!( - "There is no UDF named \"{}\" in the registry", - name + "There is no UDF named \"{name}\" in the registry" )) }) } @@ -1944,8 +1931,7 @@ impl FunctionRegistry for SessionState { result.cloned().ok_or_else(|| { DataFusionError::Plan(format!( - "There is no UDAF named \"{}\" in the registry", - name + "There is no UDAF named \"{name}\" in the registry" )) }) } @@ -2082,8 +2068,7 @@ impl FunctionRegistry for TaskContext { result.cloned().ok_or_else(|| { DataFusionError::Internal(format!( - "There is no UDF named \"{}\" in the TaskContext", - name + "There is no UDF named \"{name}\" in the TaskContext" )) }) } @@ -2093,8 +2078,7 @@ impl FunctionRegistry for TaskContext { result.cloned().ok_or_else(|| { DataFusionError::Internal(format!( - "There is no UDAF named \"{}\" in the TaskContext", - name + "There is no UDAF named \"{name}\" in the TaskContext" )) }) } @@ -2465,7 +2449,7 @@ mod tests { for table_ref in &["my_catalog.my_schema.test", "my_schema.test", "test"] { let result = plan_and_collect( &ctx, - &format!("SELECT COUNT(*) AS count FROM {}", table_ref), + &format!("SELECT COUNT(*) AS count FROM {table_ref}"), ) .await .unwrap(); @@ -2715,7 +2699,7 @@ mod tests { // generate a partitioned file for partition in 0..partition_count { - let filename = format!("partition-{}.{}", partition, file_extension); + let filename = format!("partition-{partition}.{file_extension}"); let file_path = tmp_dir.path().join(filename); let mut file = File::create(file_path)?; diff --git a/datafusion/core/src/execution/disk_manager.rs b/datafusion/core/src/execution/disk_manager.rs index b8e5fac3bc25..2749d8cbc9ba 100644 --- a/datafusion/core/src/execution/disk_manager.rs +++ b/datafusion/core/src/execution/disk_manager.rs @@ -110,8 +110,7 @@ impl DiskManager { let mut guard = self.local_dirs.lock(); let local_dirs = guard.as_mut().ok_or_else(|| { DataFusionError::ResourcesExhausted(format!( - "Memory Exhausted while {} (DiskManager is disabled)", - request_description + "Memory Exhausted while {request_description} (DiskManager is disabled)" )) })?; @@ -228,6 +227,6 @@ mod tests { .any(|candidate_path| *file_path == candidate_path) }); - assert!(found, "Can't find {:?} in dirs: {:?}", file_path, dirs); + assert!(found, "Can't find {file_path:?} in dirs: {dirs:?}"); } } diff --git a/datafusion/core/src/execution/memory_pool/mod.rs b/datafusion/core/src/execution/memory_pool/mod.rs index 6369cda4d149..8b519d52954b 100644 --- a/datafusion/core/src/execution/memory_pool/mod.rs +++ b/datafusion/core/src/execution/memory_pool/mod.rs @@ -184,7 +184,7 @@ pub fn human_readable_size(size: usize) -> String { (size as f64, "B") } }; - format!("{:.1} {}", value, unit) + format!("{value:.1} {unit}") } #[cfg(test)] diff --git a/datafusion/core/src/physical_optimizer/enforcement.rs b/datafusion/core/src/physical_optimizer/enforcement.rs index f017d0e0aaf8..7c55a81aa75c 100644 --- a/datafusion/core/src/physical_optimizer/enforcement.rs +++ b/datafusion/core/src/physical_optimizer/enforcement.rs @@ -1201,7 +1201,7 @@ mod tests { for join_type in join_types { let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); let join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"b1\", index: 1 }})]", join_type); + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"b1\", index: 1 }})]"); match join_type { JoinType::Inner @@ -1222,7 +1222,7 @@ mod tests { &join_type, ); let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"c\", index: 2 }})]", join_type); + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"c\", index: 2 }})]"); let expected = match join_type { // Should include 3 RepartitionExecs @@ -1274,9 +1274,9 @@ mod tests { hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); let top_join_plan = match join_type { JoinType::RightSemi | JoinType::RightAnti => - format!("HashJoinExec: mode=Partitioned, join_type={}, on=[(Column {{ name: \"b1\", index: 1 }}, Column {{ name: \"c\", index: 2 }})]", join_type), + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"b1\", index: 1 }}, Column {{ name: \"c\", index: 2 }})]"), _ => - format!("HashJoinExec: mode=Partitioned, join_type={}, on=[(Column {{ name: \"b1\", index: 6 }}, Column {{ name: \"c\", index: 2 }})]", join_type), + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"b1\", index: 6 }}, Column {{ name: \"c\", index: 2 }})]"), }; let expected = match join_type { @@ -1909,7 +1909,7 @@ mod tests { let join = sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); let join_plan = - format!("SortMergeJoin: join_type={}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"b1\", index: 1 }})]", join_type); + format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"b1\", index: 1 }})]"); // Top join on (a == c) let top_join_on = vec![( @@ -1923,7 +1923,7 @@ mod tests { &join_type, ); let top_join_plan = - format!("SortMergeJoin: join_type={}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"c\", index: 2 }})]", join_type); + format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"c\", index: 2 }})]"); let expected = match join_type { // Should include 3 RepartitionExecs 3 SortExecs @@ -1977,7 +1977,7 @@ mod tests { &join_type, ); let top_join_plan = - format!("SortMergeJoin: join_type={}, on=[(Column {{ name: \"b1\", index: 6 }}, Column {{ name: \"c\", index: 2 }})]", join_type); + format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"b1\", index: 6 }}, Column {{ name: \"c\", index: 2 }})]"); let expected = match join_type { // Should include 3 RepartitionExecs and 3 SortExecs diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index c479e2c9af69..b378e3beb60c 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -553,7 +553,7 @@ mod tests { .expect("A proj is required to swap columns back to their original order"); assert_eq!(swapping_projection.expr().len(), 2); - println!("swapping_projection {:?}", swapping_projection); + println!("swapping_projection {swapping_projection:?}"); let (col, name) = &swapping_projection.expr()[0]; assert_eq!(name, "small_col"); assert_col_expr(col, "small_col", 1); diff --git a/datafusion/core/src/physical_optimizer/optimize_sorts.rs b/datafusion/core/src/physical_optimizer/optimize_sorts.rs index ed827c14e136..a47026cc773d 100644 --- a/datafusion/core/src/physical_optimizer/optimize_sorts.rs +++ b/datafusion/core/src/physical_optimizer/optimize_sorts.rs @@ -585,8 +585,7 @@ mod tests { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let optimized_physical_plan = OptimizeSorts::new().optimize(physical_plan, state.config_options())?; @@ -599,8 +598,7 @@ mod tests { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); Ok(()) } @@ -686,8 +684,7 @@ mod tests { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let optimized_physical_plan = OptimizeSorts::new().optimize(physical_plan, state.config_options())?; @@ -706,8 +703,7 @@ mod tests { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); Ok(()) } @@ -732,8 +728,7 @@ mod tests { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let optimized_physical_plan = OptimizeSorts::new().optimize(physical_plan, state.config_options())?; @@ -751,8 +746,7 @@ mod tests { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); Ok(()) } @@ -799,8 +793,7 @@ mod tests { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let optimized_physical_plan = OptimizeSorts::new().optimize(physical_plan, state.config_options())?; @@ -818,8 +811,7 @@ mod tests { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); Ok(()) } @@ -861,8 +853,7 @@ mod tests { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let optimized_physical_plan = OptimizeSorts::new().optimize(physical_plan, state.config_options())?; @@ -879,8 +870,7 @@ mod tests { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); Ok(()) } diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index 92cb45d56c49..ab74ad37e03e 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -163,8 +163,7 @@ fn swap(hash_join: &HashJoinExec) -> Result> { _, JoinType::Right | JoinType::RightSemi | JoinType::RightAnti | JoinType::Full, ) => Err(DataFusionError::Internal(format!( - "{} join cannot be swapped for unbounded input.", - join_type + "{join_type} join cannot be swapped for unbounded input." ))), (PartitionMode::Partitioned, _) => { swap_hash_join(hash_join, PartitionMode::Partitioned) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 2645b15ee09d..e755a2de276b 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -203,8 +203,7 @@ impl PruningPredicate { other => { Err(DataFusionError::Internal(format!( "Unexpected result of pruning predicate evaluation. Expected Boolean array \ - or scalar but got {:?}", - other + or scalar but got {other:?}" ))) } } @@ -432,7 +431,7 @@ fn build_statistics_record_batch( ); RecordBatch::try_new_with_options(schema, arrays, &options).map_err(|err| { - DataFusionError::Plan(format!("Can not create statistics record batch: {}", err)) + DataFusionError::Plan(format!("Can not create statistics record batch: {err}")) }) } @@ -578,15 +577,13 @@ fn rewrite_expr_to_prunable( Expr::Not(Box::new(scalar_expr.clone())), )), _ => Err(DataFusionError::Plan(format!( - "Not with complex expression {:?} is not supported", - column_expr + "Not with complex expression {column_expr:?} is not supported" ))), }; } _ => Err(DataFusionError::Plan(format!( - "column expression {:?} is not supported", - column_expr + "column expression {column_expr:?} is not supported" ))), } } @@ -623,8 +620,7 @@ fn verify_support_type_for_prune(from_type: &DataType, to_type: &DataType) -> Re Ok(()) } else { Err(DataFusionError::Plan(format!( - "Try Cast/Cast with from type {} to type {} is not supported", - from_type, to_type + "Try Cast/Cast with from type {from_type} to type {to_type} is not supported" ))) } } @@ -658,8 +654,7 @@ fn rewrite_column_expr( fn reverse_operator(op: Operator) -> Result { op.swap().ok_or_else(|| { DataFusionError::Internal(format!( - "Could not reverse operator {} while building pruning predicate", - op + "Could not reverse operator {op} while building pruning predicate" )) }) } @@ -1295,13 +1290,13 @@ mod tests { let expr = col("c1").eq(lit(1)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(1).eq(col("c1")); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1315,13 +1310,13 @@ mod tests { let expr = col("c1").not_eq(lit(1)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(1).not_eq(col("c1")); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1335,13 +1330,13 @@ mod tests { let expr = col("c1").gt(lit(1)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(1).lt(col("c1")); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1355,12 +1350,12 @@ mod tests { let expr = col("c1").gt_eq(lit(1)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(1).lt_eq(col("c1")); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1374,13 +1369,13 @@ mod tests { let expr = col("c1").lt(lit(1)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(1).gt(col("c1")); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1394,12 +1389,12 @@ mod tests { let expr = col("c1").lt_eq(lit(1)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(1).gt_eq(col("c1")); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1416,7 +1411,7 @@ mod tests { let expected_expr = "c1_min < Int32(1)"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1432,7 +1427,7 @@ mod tests { let expected_expr = "Boolean(true)"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1445,7 +1440,7 @@ mod tests { let expr = col("c1").not(); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1458,7 +1453,7 @@ mod tests { let expr = col("c1").not(); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1471,7 +1466,7 @@ mod tests { let expr = col("c1"); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1486,7 +1481,7 @@ mod tests { let expr = col("c1").lt(lit(true)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1505,7 +1500,7 @@ mod tests { let expected_expr = "c1_min < Int32(1) AND (c2_min <= Int32(2) AND Int32(2) <= c2_max OR c2_min <= Int32(3) AND Int32(3) <= c2_max)"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut required_columns)?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // c1 < 1 should add c1_min let c1_min_field = Field::new("c1_min", DataType::Int32, false); assert_eq!( @@ -1544,7 +1539,7 @@ mod tests { let expected_expr = "c1_min <= Int32(1) AND Int32(1) <= c1_max OR c1_min <= Int32(2) AND Int32(2) <= c1_max OR c1_min <= Int32(3) AND Int32(3) <= c1_max"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1564,7 +1559,7 @@ mod tests { let expected_expr = "Boolean(true)"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1586,7 +1581,7 @@ mod tests { AND (c1_min != Int32(3) OR Int32(3) != c1_max)"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1601,13 +1596,13 @@ mod tests { let expr = cast(col("c1"), DataType::Int64).eq(lit(ScalarValue::Int64(Some(1)))); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(ScalarValue::Int64(Some(1))).eq(cast(col("c1"), DataType::Int64)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); let expected_expr = "TRY_CAST(c1_max AS Int64) > Int64(1)"; @@ -1616,14 +1611,14 @@ mod tests { try_cast(col("c1"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(1)))); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); // test column on the right let expr = lit(ScalarValue::Int64(Some(1))).lt(try_cast(col("c1"), DataType::Int64)); let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } @@ -1644,7 +1639,7 @@ mod tests { let expected_expr = "CAST(c1_min AS Int64) <= Int64(1) AND Int64(1) <= CAST(c1_max AS Int64) OR CAST(c1_min AS Int64) <= Int64(2) AND Int64(2) <= CAST(c1_max AS Int64) OR CAST(c1_min AS Int64) <= Int64(3) AND Int64(3) <= CAST(c1_max AS Int64)"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); let expr = Expr::InList { expr: Box::new(cast(col("c1"), DataType::Int64)), @@ -1661,7 +1656,7 @@ mod tests { AND (CAST(c1_min AS Int64) != Int64(3) OR Int64(3) != CAST(c1_max AS Int64))"; let predicate_expr = build_predicate_expression(&expr, &schema, &mut RequiredStatColumns::new())?; - assert_eq!(format!("{:?}", predicate_expr), expected_expr); + assert_eq!(format!("{predicate_expr:?}"), expected_expr); Ok(()) } diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index ce6e252c75a9..1404dfa20c30 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -31,7 +31,7 @@ async fn register_current_csv( let schema = crate::test_util::aggr_test_schema(); ctx.register_csv( table_name, - &format!("{}/csv/aggregate_test_100.csv", testdata), + &format!("{testdata}/csv/aggregate_test_100.csv"), CsvReadOptions::new() .schema(&schema) .mark_infinite(infinite), diff --git a/datafusion/core/src/physical_plan/aggregates/hash.rs b/datafusion/core/src/physical_plan/aggregates/hash.rs index 64b21ecf9a6b..7c4327014655 100644 --- a/datafusion/core/src/physical_plan/aggregates/hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/hash.rs @@ -125,7 +125,7 @@ impl GroupedHashAggregateStream { timer.done(); let reservation = - MemoryConsumer::new(format!("GroupedHashAggregateStream[{}]", partition)) + MemoryConsumer::new(format!("GroupedHashAggregateStream[{partition}]")) .register(context.memory_pool()); let inner = GroupedHashAggregateStreamInner { diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index f5591f2c3bb0..cb24193e3e93 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -449,7 +449,7 @@ impl ExecutionPlan for AggregateExec { .map(|(e, alias)| { let e = e.to_string(); if &e != alias { - format!("{} as {}", e, alias) + format!("{e} as {alias}") } else { e } @@ -468,7 +468,7 @@ impl ExecutionPlan for AggregateExec { let (e, alias) = &self.group_by.null_expr[idx]; let e = e.to_string(); if &e != alias { - format!("{} as {}", e, alias) + format!("{e} as {alias}") } else { e } @@ -476,7 +476,7 @@ impl ExecutionPlan for AggregateExec { let (e, alias) = &self.group_by.expr[idx]; let e = e.to_string(); if &e != alias { - format!("{} as {}", e, alias) + format!("{e} as {alias}") } else { e } @@ -484,7 +484,7 @@ impl ExecutionPlan for AggregateExec { }) .collect::>() .join(", "); - format!("({})", terms) + format!("({terms})") }) .collect() }; @@ -1032,8 +1032,7 @@ mod tests { _: Vec>, ) -> Result> { Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self + "Children cannot be replaced in {self:?}" ))) } @@ -1215,8 +1214,7 @@ mod tests { if let Some(err) = err.downcast_ref::() { assert!( matches!(err, DataFusionError::ResourcesExhausted(_)), - "Wrong inner error type: {}", - err, + "Wrong inner error type: {err}", ); } else { panic!("Wrong arrow error type: {err}") diff --git a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs index 8a312abafd9b..4d9de842f6f0 100644 --- a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs +++ b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs @@ -72,7 +72,7 @@ impl AggregateStream { let aggregate_expressions = aggregate_expressions(&aggr_expr, &mode, 0)?; let accumulators = create_accumulators(&aggr_expr)?; - let reservation = MemoryConsumer::new(format!("AggregateStream[{}]", partition)) + let reservation = MemoryConsumer::new(format!("AggregateStream[{partition}]")) .register(context.memory_pool()); let inner = AggregateStreamInner { diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index e769397871ef..94654d502f79 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -140,7 +140,7 @@ impl GroupedHashAggregateStreamV2 { let aggr_layout = Arc::new(RowLayout::new(&aggr_schema, RowType::WordAligned)); let reservation = - MemoryConsumer::new(format!("GroupedHashAggregateStreamV2[{}]", partition)) + MemoryConsumer::new(format!("GroupedHashAggregateStreamV2[{partition}]")) .register(context.memory_pool()); let aggr_state = AggregationState { diff --git a/datafusion/core/src/physical_plan/analyze.rs b/datafusion/core/src/physical_plan/analyze.rs index 7f1cfe57a70a..2e7285aac597 100644 --- a/datafusion/core/src/physical_plan/analyze.rs +++ b/datafusion/core/src/physical_plan/analyze.rs @@ -117,8 +117,7 @@ impl ExecutionPlan for AnalyzeExec { ) -> Result { if 0 != partition { return Err(DataFusionError::Internal(format!( - "AnalyzeExec invalid partition. Expected 0, got {}", - partition + "AnalyzeExec invalid partition. Expected 0, got {partition}" ))); } @@ -126,8 +125,7 @@ impl ExecutionPlan for AnalyzeExec { let input_partitions = self.input.output_partitioning().partition_count(); if input_partitions != 1 { return Err(DataFusionError::Internal(format!( - "AnalyzeExec invalid number of input partitions. Expected 1, got {}", - input_partitions + "AnalyzeExec invalid number of input partitions. Expected 1, got {input_partitions}" ))); } diff --git a/datafusion/core/src/physical_plan/coalesce_partitions.rs b/datafusion/core/src/physical_plan/coalesce_partitions.rs index d765c275ab54..1b847372eed6 100644 --- a/datafusion/core/src/physical_plan/coalesce_partitions.rs +++ b/datafusion/core/src/physical_plan/coalesce_partitions.rs @@ -114,8 +114,7 @@ impl ExecutionPlan for CoalescePartitionsExec { // CoalescePartitionsExec produces a single partition if 0 != partition { return Err(DataFusionError::Internal(format!( - "CoalescePartitionsExec invalid partition {}", - partition + "CoalescePartitionsExec invalid partition {partition}" ))); } diff --git a/datafusion/core/src/physical_plan/common.rs b/datafusion/core/src/physical_plan/common.rs index 3cdc268cb8d4..7ea8dfe35bc6 100644 --- a/datafusion/core/src/physical_plan/common.rs +++ b/datafusion/core/src/physical_plan/common.rs @@ -101,9 +101,7 @@ pub fn build_checked_file_list(dir: &str, ext: &str) -> Result> { build_file_list_recurse(dir, &mut filenames, ext)?; if filenames.is_empty() { return Err(DataFusionError::Plan(format!( - "No files found at {path} with file extension {file_extension}", - path = dir, - file_extension = ext + "No files found at {dir} with file extension {ext}" ))); } Ok(filenames) @@ -378,8 +376,7 @@ impl IPCWriter { pub fn new(path: &Path, schema: &Schema) -> Result { let file = File::create(path).map_err(|e| { DataFusionError::Execution(format!( - "Failed to create partition file at {:?}: {:?}", - path, e + "Failed to create partition file at {path:?}: {e:?}" )) })?; Ok(Self { @@ -399,8 +396,7 @@ impl IPCWriter { ) -> Result { let file = File::create(path).map_err(|e| { DataFusionError::Execution(format!( - "Failed to create partition file at {:?}: {:?}", - path, e + "Failed to create partition file at {path:?}: {e:?}" )) })?; Ok(Self { diff --git a/datafusion/core/src/physical_plan/display.rs b/datafusion/core/src/physical_plan/display.rs index a8093d8a7f0d..24613484fd02 100644 --- a/datafusion/core/src/physical_plan/display.rs +++ b/datafusion/core/src/physical_plan/display.rs @@ -171,14 +171,14 @@ impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { .sorted_for_display() .timestamps_removed(); - write!(self.f, ", metrics=[{}]", metrics)?; + write!(self.f, ", metrics=[{metrics}]")?; } else { write!(self.f, ", metrics=[]")?; } } ShowMetrics::Full => { if let Some(metrics) = plan.metrics() { - write!(self.f, ", metrics=[{}]", metrics)?; + write!(self.f, ", metrics=[{metrics}]")?; } else { write!(self.f, ", metrics=[]")?; } diff --git a/datafusion/core/src/physical_plan/empty.rs b/datafusion/core/src/physical_plan/empty.rs index b71f6739b65b..c3bcc5d2aa83 100644 --- a/datafusion/core/src/physical_plan/empty.rs +++ b/datafusion/core/src/physical_plan/empty.rs @@ -76,7 +76,7 @@ impl EmptyExec { (0..n_field) .into_iter() .map(|i| { - Field::new(format!("placeholder_{}", i), DataType::Null, true) + Field::new(format!("placeholder_{i}"), DataType::Null, true) }) .collect(), )), diff --git a/datafusion/core/src/physical_plan/explain.rs b/datafusion/core/src/physical_plan/explain.rs index 077ed8dcc461..93fcfe45dadf 100644 --- a/datafusion/core/src/physical_plan/explain.rs +++ b/datafusion/core/src/physical_plan/explain.rs @@ -112,8 +112,7 @@ impl ExecutionPlan for ExplainExec { debug!("Start ExplainExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); if 0 != partition { return Err(DataFusionError::Internal(format!( - "ExplainExec invalid partition {}", - partition + "ExplainExec invalid partition {partition}" ))); } diff --git a/datafusion/core/src/physical_plan/file_format/csv.rs b/datafusion/core/src/physical_plan/file_format/csv.rs index 3afd844cfe7c..26ac8cfcda6d 100644 --- a/datafusion/core/src/physical_plan/file_format/csv.rs +++ b/datafusion/core/src/physical_plan/file_format/csv.rs @@ -253,7 +253,7 @@ pub async fn plan_to_csv( let mut tasks = vec![]; for i in 0..plan.output_partitioning().partition_count() { let plan = plan.clone(); - let filename = format!("part-{}.csv", i); + let filename = format!("part-{i}.csv"); let path = fs_path.join(filename); let file = fs::File::create(path)?; let mut writer = csv::Writer::new(file); @@ -272,13 +272,12 @@ pub async fn plan_to_csv( .await .into_iter() .try_for_each(|result| { - result.map_err(|e| DataFusionError::Execution(format!("{}", e)))? + result.map_err(|e| DataFusionError::Execution(format!("{e}")))? })?; Ok(()) } Err(e) => Err(DataFusionError::Execution(format!( - "Could not create directory {}: {:?}", - path, e + "Could not create directory {path}: {e:?}" ))), } } @@ -546,7 +545,7 @@ mod tests { // generate a partitioned file for partition in 0..partition_count { - let filename = format!("partition-{}.{}", partition, file_extension); + let filename = format!("partition-{partition}.{file_extension}"); let file_path = tmp_dir.path().join(filename); let mut file = File::create(file_path)?; @@ -629,7 +628,7 @@ mod tests { .write_csv(&out_dir) .await .expect_err("should fail because input file does not match inferred schema"); - assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{}", e)); + assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}")); Ok(()) } @@ -667,7 +666,7 @@ mod tests { let csv_read_option = CsvReadOptions::new().schema(&schema); ctx.register_csv( "part0", - &format!("{}/part-0.csv", out_dir), + &format!("{out_dir}/part-0.csv"), csv_read_option.clone(), ) .await?; diff --git a/datafusion/core/src/physical_plan/file_format/json.rs b/datafusion/core/src/physical_plan/file_format/json.rs index 3edb74025acb..8184d3f1ac45 100644 --- a/datafusion/core/src/physical_plan/file_format/json.rs +++ b/datafusion/core/src/physical_plan/file_format/json.rs @@ -215,7 +215,7 @@ pub async fn plan_to_json( let mut tasks = vec![]; for i in 0..plan.output_partitioning().partition_count() { let plan = plan.clone(); - let filename = format!("part-{}.json", i); + let filename = format!("part-{i}.json"); let path = fs_path.join(filename); let file = fs::File::create(path)?; let mut writer = json::LineDelimitedWriter::new(file); @@ -234,13 +234,12 @@ pub async fn plan_to_json( .await .into_iter() .try_for_each(|result| { - result.map_err(|e| DataFusionError::Execution(format!("{}", e)))? + result.map_err(|e| DataFusionError::Execution(format!("{e}")))? })?; Ok(()) } Err(e) => Err(DataFusionError::Execution(format!( - "Could not create directory {}: {:?}", - path, e + "Could not create directory {path}: {e:?}" ))), } } @@ -536,7 +535,7 @@ mod tests { let ctx = SessionContext::with_config(SessionConfig::new().with_target_partitions(8)); - let path = format!("{}/1.json", TEST_DATA_BASE); + let path = format!("{TEST_DATA_BASE}/1.json"); // register json file with the execution context ctx.register_json("test", path.as_str(), NdJsonReadOptions::default()) @@ -554,7 +553,7 @@ mod tests { let json_read_option = NdJsonReadOptions::default(); ctx.register_json( "part0", - &format!("{}/part-0.json", out_dir), + &format!("{out_dir}/part-0.json"), json_read_option.clone(), ) .await?; @@ -612,7 +611,7 @@ mod tests { .write_json(&out_dir) .await .expect_err("should fail because input file does not match inferred schema"); - assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{}", e)); + assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}")); Ok(()) } } diff --git a/datafusion/core/src/physical_plan/file_format/parquet.rs b/datafusion/core/src/physical_plan/file_format/parquet.rs index f6e7d2ded747..2bdddcddc162 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet.rs @@ -334,7 +334,7 @@ impl ExecutionPlan for ParquetExec { let predicate_string = self .predicate .as_ref() - .map(|p| format!(", predicate={}", p)) + .map(|p| format!(", predicate={p}")) .unwrap_or_default(); let pruning_predicate_string = self @@ -379,7 +379,7 @@ fn make_output_ordering_string(ordering: &[PhysicalSortExpr]) -> String { if i > 0 { write!(&mut w, ", ").unwrap() } - write!(&mut w, "{}", e).unwrap() + write!(&mut w, "{e}").unwrap() } write!(&mut w, "]").unwrap(); w @@ -556,7 +556,7 @@ impl AsyncFileReader for ParquetFileReader { self.store .get_range(&self.meta.location, range) .map_err(|e| { - ParquetError::General(format!("AsyncChunkReader::get_bytes error: {}", e)) + ParquetError::General(format!("AsyncChunkReader::get_bytes error: {e}")) }) .boxed() } @@ -577,8 +577,7 @@ impl AsyncFileReader for ParquetFileReader { .await .map_err(|e| { ParquetError::General(format!( - "AsyncChunkReader::get_byte_ranges error: {}", - e + "AsyncChunkReader::get_byte_ranges error: {e}" )) }) } @@ -597,8 +596,7 @@ impl AsyncFileReader for ParquetFileReader { .await .map_err(|e| { ParquetError::General(format!( - "AsyncChunkReader::get_metadata error: {}", - e + "AsyncChunkReader::get_metadata error: {e}" )) })?; Ok(Arc::new(metadata)) @@ -644,7 +642,7 @@ pub async fn plan_to_parquet( let mut tasks = vec![]; for i in 0..plan.output_partitioning().partition_count() { let plan = plan.clone(); - let filename = format!("part-{}.parquet", i); + let filename = format!("part-{i}.parquet"); let path = fs_path.join(filename); let file = fs::File::create(path)?; let mut writer = @@ -666,13 +664,12 @@ pub async fn plan_to_parquet( .await .into_iter() .try_for_each(|result| { - result.map_err(|e| DataFusionError::Execution(format!("{}", e)))? + result.map_err(|e| DataFusionError::Execution(format!("{e}")))? })?; Ok(()) } Err(e) => Err(DataFusionError::Execution(format!( - "Could not create directory {}: {:?}", - path, e + "Could not create directory {path}: {e:?}" ))), } } @@ -873,7 +870,7 @@ mod tests { .write_parquet(&out_dir, None) .await .expect_err("should fail because input file does not match inferred schema"); - assert_eq!("Parquet error: Arrow: underlying Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{}", e)); + assert_eq!("Parquet error: Arrow: underlying Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}")); Ok(()) } @@ -1373,7 +1370,7 @@ mod tests { let state = session_ctx.state(); let testdata = crate::test_util::parquet_test_data(); - let filename = format!("{}/alltypes_plain.parquet", testdata); + let filename = format!("{testdata}/alltypes_plain.parquet"); let meta = local_unpartitioned_file(filename); @@ -1406,7 +1403,7 @@ mod tests { let store = state.runtime_env().object_store(&object_store_url).unwrap(); let testdata = crate::test_util::parquet_test_data(); - let filename = format!("{}/alltypes_plain.parquet", testdata); + let filename = format!("{testdata}/alltypes_plain.parquet"); let meta = local_unpartitioned_file(filename); @@ -1538,8 +1535,7 @@ mod tests { assert_eq!(get_value(&metrics, "page_index_rows_filtered"), 3); assert!( get_value(&metrics, "page_index_eval_time") > 0, - "no eval time in metrics: {:#?}", - metrics + "no eval time in metrics: {metrics:#?}" ); } @@ -1577,8 +1573,7 @@ mod tests { assert_eq!(get_value(&metrics, "pushdown_rows_filtered"), 5); assert!( get_value(&metrics, "pushdown_eval_time") > 0, - "no eval time in metrics: {:#?}", - metrics + "no eval time in metrics: {metrics:#?}" ); } @@ -1646,8 +1641,7 @@ mod tests { let pruning_predicate = &rt.parquet_exec.pruning_predicate; assert!( pruning_predicate.is_none(), - "Still had pruning predicate: {:?}", - pruning_predicate + "Still had pruning predicate: {pruning_predicate:?}" ); // but does still has a pushdown down predicate @@ -1667,8 +1661,7 @@ mod tests { Some(v) => v.as_usize(), _ => { panic!( - "Expected metric not found. Looking for '{}' in\n\n{:#?}", - metric_name, metrics + "Expected metric not found. Looking for '{metric_name}' in\n\n{metrics:#?}" ); } } @@ -1688,7 +1681,7 @@ mod tests { // generate a partitioned file for partition in 0..partition_count { - let filename = format!("partition-{}.{}", partition, file_extension); + let filename = format!("partition-{partition}.{file_extension}"); let file_path = tmp_dir.path().join(filename); let mut file = File::create(file_path)?; @@ -1730,25 +1723,25 @@ mod tests { // register each partition as well as the top level dir ctx.register_parquet( "part0", - &format!("{}/part-0.parquet", out_dir), + &format!("{out_dir}/part-0.parquet"), ParquetReadOptions::default(), ) .await?; ctx.register_parquet( "part1", - &format!("{}/part-1.parquet", out_dir), + &format!("{out_dir}/part-1.parquet"), ParquetReadOptions::default(), ) .await?; ctx.register_parquet( "part2", - &format!("{}/part-2.parquet", out_dir), + &format!("{out_dir}/part-2.parquet"), ParquetReadOptions::default(), ) .await?; ctx.register_parquet( "part3", - &format!("{}/part-3.parquet", out_dir), + &format!("{out_dir}/part-3.parquet"), ParquetReadOptions::default(), ) .await?; diff --git a/datafusion/core/src/physical_plan/file_format/parquet/page_filter.rs b/datafusion/core/src/physical_plan/file_format/parquet/page_filter.rs index b2bfcf72e1d3..57115fbf9842 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet/page_filter.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet/page_filter.rs @@ -168,8 +168,7 @@ impl PagePruningPredicate { ) .map_err(|e| { ArrowError::ParquetError(format!( - "Fail in prune_pages_in_one_row_group: {}", - e + "Fail in prune_pages_in_one_row_group: {e}" )) }), ); diff --git a/datafusion/core/src/physical_plan/file_format/parquet/row_filter.rs b/datafusion/core/src/physical_plan/file_format/parquet/row_filter.rs index a41ea8a05d6b..92c0ddc8724e 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet/row_filter.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet/row_filter.rs @@ -143,8 +143,7 @@ impl ArrowPredicate for DatafusionArrowPredicate { Ok(bool_arr) } Err(e) => Err(ArrowError::ComputeError(format!( - "Error evaluating filter predicate: {:?}", - e + "Error evaluating filter predicate: {e:?}" ))), } } @@ -402,7 +401,7 @@ mod test { #[should_panic(expected = "building candidate failed")] fn test_filter_candidate_builder_ignore_projected_columns() { let testdata = crate::test_util::parquet_test_data(); - let file = std::fs::File::open(format!("{}/alltypes_plain.parquet", testdata)) + let file = std::fs::File::open(format!("{testdata}/alltypes_plain.parquet")) .expect("opening file"); let reader = SerializedFileReader::new(file).expect("creating reader"); @@ -426,7 +425,7 @@ mod test { #[test] fn test_filter_candidate_builder_ignore_complex_types() { let testdata = crate::test_util::parquet_test_data(); - let file = std::fs::File::open(format!("{}/list_columns.parquet", testdata)) + let file = std::fs::File::open(format!("{testdata}/list_columns.parquet")) .expect("opening file"); let reader = SerializedFileReader::new(file).expect("creating reader"); @@ -450,7 +449,7 @@ mod test { #[test] fn test_filter_candidate_builder_rewrite_missing_column() { let testdata = crate::test_util::parquet_test_data(); - let file = std::fs::File::open(format!("{}/alltypes_plain.parquet", testdata)) + let file = std::fs::File::open(format!("{testdata}/alltypes_plain.parquet")) .expect("opening file"); let reader = SerializedFileReader::new(file).expect("creating reader"); diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs index 758e31a2877f..35c20c4cc6cb 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/core/src/physical_plan/filter.rs @@ -70,8 +70,7 @@ impl FilterExec { metrics: ExecutionPlanMetricsSet::new(), }), other => Err(DataFusionError::Plan(format!( - "Filter predicate must return boolean values, not {:?}", - other + "Filter predicate must return boolean values, not {other:?}" ))), } } diff --git a/datafusion/core/src/physical_plan/joins/hash_join.rs b/datafusion/core/src/physical_plan/joins/hash_join.rs index abb5b5e2904b..16f143560173 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join.rs @@ -1131,8 +1131,7 @@ fn equal_rows( other => { // This is internal because we should have caught this before. err = Some(Err(DataFusionError::Internal(format!( - "Unsupported data type in hasher: {}", - other + "Unsupported data type in hasher: {other}" )))); false } @@ -3008,8 +3007,7 @@ mod tests { .to_string(); assert!( result_string.contains("bad data error"), - "actual: {}", - result_string + "actual: {result_string}" ); } } diff --git a/datafusion/core/src/physical_plan/joins/utils.rs b/datafusion/core/src/physical_plan/joins/utils.rs index 11b9846940ac..3f568f98ec9d 100644 --- a/datafusion/core/src/physical_plan/joins/utils.rs +++ b/datafusion/core/src/physical_plan/joins/utils.rs @@ -85,9 +85,7 @@ fn check_join_set_is_valid( if !left_missing.is_empty() | !right_missing.is_empty() { return Err(DataFusionError::Plan(format!( - "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {:?}\nMissing on the right: {:?}", - left_missing, - right_missing, + "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {left_missing:?}\nMissing on the right: {right_missing:?}", ))); }; diff --git a/datafusion/core/src/physical_plan/limit.rs b/datafusion/core/src/physical_plan/limit.rs index b25f81087fd4..3a3a4a20b167 100644 --- a/datafusion/core/src/physical_plan/limit.rs +++ b/datafusion/core/src/physical_plan/limit.rs @@ -148,8 +148,7 @@ impl ExecutionPlan for GlobalLimitExec { // GlobalLimitExec has a single output partition if 0 != partition { return Err(DataFusionError::Internal(format!( - "GlobalLimitExec invalid partition {}", - partition + "GlobalLimitExec invalid partition {partition}" ))); } diff --git a/datafusion/core/src/physical_plan/memory.rs b/datafusion/core/src/physical_plan/memory.rs index 7753a5ba7765..a15b59552294 100644 --- a/datafusion/core/src/physical_plan/memory.rs +++ b/datafusion/core/src/physical_plan/memory.rs @@ -86,8 +86,7 @@ impl ExecutionPlan for MemoryExec { _: Vec>, ) -> Result> { Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self + "Children cannot be replaced in {self:?}" ))) } diff --git a/datafusion/core/src/physical_plan/metrics/mod.rs b/datafusion/core/src/physical_plan/metrics/mod.rs index db8723217e52..ca32cc96073e 100644 --- a/datafusion/core/src/physical_plan/metrics/mod.rs +++ b/datafusion/core/src/physical_plan/metrics/mod.rs @@ -103,7 +103,7 @@ impl Display for Metric { is_first = false; } - write!(f, "{}", i)?; + write!(f, "{i}")?; } write!(f, "}}")?; @@ -322,7 +322,7 @@ impl Display for MetricsSet { is_first = false; } - write!(f, "{}", i)?; + write!(f, "{i}")?; } Ok(()) } diff --git a/datafusion/core/src/physical_plan/metrics/tracker.rs b/datafusion/core/src/physical_plan/metrics/tracker.rs index c61398c65810..435cf19d8df8 100644 --- a/datafusion/core/src/physical_plan/metrics/tracker.rs +++ b/datafusion/core/src/physical_plan/metrics/tracker.rs @@ -42,9 +42,8 @@ impl MemTrackingMetrics { pool: &Arc, partition: usize, ) -> Self { - let reservation = - MemoryConsumer::new(format!("MemTrackingMetrics[{}]", partition)) - .register(pool); + let reservation = MemoryConsumer::new(format!("MemTrackingMetrics[{partition}]")) + .register(pool); Self { reservation, diff --git a/datafusion/core/src/physical_plan/metrics/value.rs b/datafusion/core/src/physical_plan/metrics/value.rs index 5c3aeb4dcdca..4df4e7567536 100644 --- a/datafusion/core/src/physical_plan/metrics/value.rs +++ b/datafusion/core/src/physical_plan/metrics/value.rs @@ -164,7 +164,7 @@ impl PartialEq for Time { impl Display for Time { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { let duration = std::time::Duration::from_nanos(self.value() as u64); - write!(f, "{:?}", duration) + write!(f, "{duration:?}") } } @@ -294,7 +294,7 @@ impl Display for Timestamp { match self.value() { None => write!(f, "NONE"), Some(v) => { - write!(f, "{}", v) + write!(f, "{v}") } } } @@ -542,22 +542,22 @@ impl std::fmt::Display for MetricValue { | Self::SpillCount(count) | Self::SpilledBytes(count) | Self::Count { count, .. } => { - write!(f, "{}", count) + write!(f, "{count}") } Self::CurrentMemoryUsage(gauge) | Self::Gauge { gauge, .. } => { - write!(f, "{}", gauge) + write!(f, "{gauge}") } Self::ElapsedCompute(time) | Self::Time { time, .. } => { // distinguish between no time recorded and very small // amount of time recorded if time.value() > 0 { - write!(f, "{}", time) + write!(f, "{time}") } else { write!(f, "NOT RECORDED") } } Self::StartTimestamp(timestamp) | Self::EndTimestamp(timestamp) => { - write!(f, "{}", timestamp) + write!(f, "{timestamp}") } } } @@ -581,12 +581,12 @@ mod tests { ]; for value in &values { - assert_eq!("0", value.to_string(), "value {:?}", value); + assert_eq!("0", value.to_string(), "value {value:?}"); } count.add(42); for value in &values { - assert_eq!("42", value.to_string(), "value {:?}", value); + assert_eq!("42", value.to_string(), "value {value:?}"); } } @@ -603,12 +603,12 @@ mod tests { // if time is not set, it should not be reported as zero for value in &values { - assert_eq!("NOT RECORDED", value.to_string(), "value {:?}", value); + assert_eq!("NOT RECORDED", value.to_string(), "value {value:?}"); } time.add_duration(Duration::from_nanos(1042)); for value in &values { - assert_eq!("1.042µs", value.to_string(), "value {:?}", value); + assert_eq!("1.042µs", value.to_string(), "value {value:?}"); } } @@ -622,7 +622,7 @@ mod tests { // if time is not set, it should not be reported as zero for value in &values { - assert_eq!("NONE", value.to_string(), "value {:?}", value); + assert_eq!("NONE", value.to_string(), "value {value:?}"); } timestamp.set(Utc.timestamp_nanos(1431648000000000)); @@ -630,8 +630,7 @@ mod tests { assert_eq!( "1970-01-17 13:40:48 UTC", value.to_string(), - "value {:?}", - value + "value {value:?}" ); } } diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index 5b001f01678e..124f16e330c5 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -113,22 +113,22 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { } Expr::Alias(_, name) => Ok(name.clone()), Expr::ScalarVariable(_, variable_names) => Ok(variable_names.join(".")), - Expr::Literal(value) => Ok(format!("{:?}", value)), + Expr::Literal(value) => Ok(format!("{value:?}")), Expr::BinaryExpr(BinaryExpr { left, op, right }) => { let left = create_physical_name(left, false)?; let right = create_physical_name(right, false)?; - Ok(format!("{} {} {}", left, op, right)) + Ok(format!("{left} {op} {right}")) } Expr::Case(case) => { let mut name = "CASE ".to_string(); if let Some(e) = &case.expr { - let _ = write!(name, "{:?} ", e); + let _ = write!(name, "{e:?} "); } for (w, t) in &case.when_then_expr { - let _ = write!(name, "WHEN {:?} THEN {:?} ", w, t); + let _ = write!(name, "WHEN {w:?} THEN {t:?} "); } if let Some(e) = &case.else_expr { - let _ = write!(name, "ELSE {:?} ", e); + let _ = write!(name, "ELSE {e:?} "); } name += "END"; Ok(name) @@ -143,47 +143,47 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { } Expr::Not(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("NOT {}", expr)) + Ok(format!("NOT {expr}")) } Expr::Negative(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("(- {})", expr)) + Ok(format!("(- {expr})")) } Expr::IsNull(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS NULL", expr)) + Ok(format!("{expr} IS NULL")) } Expr::IsNotNull(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS NOT NULL", expr)) + Ok(format!("{expr} IS NOT NULL")) } Expr::IsTrue(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS TRUE", expr)) + Ok(format!("{expr} IS TRUE")) } Expr::IsFalse(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS FALSE", expr)) + Ok(format!("{expr} IS FALSE")) } Expr::IsUnknown(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS UNKNOWN", expr)) + Ok(format!("{expr} IS UNKNOWN")) } Expr::IsNotTrue(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS NOT TRUE", expr)) + Ok(format!("{expr} IS NOT TRUE")) } Expr::IsNotFalse(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS NOT FALSE", expr)) + Ok(format!("{expr} IS NOT FALSE")) } Expr::IsNotUnknown(expr) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{} IS NOT UNKNOWN", expr)) + Ok(format!("{expr} IS NOT UNKNOWN")) } Expr::GetIndexedField(GetIndexedField { key, expr }) => { let expr = create_physical_name(expr, false)?; - Ok(format!("{}[{}]", expr, key)) + Ok(format!("{expr}[{key}]")) } Expr::ScalarFunction { fun, args, .. } => { create_function_physical_name(&fun.to_string(), false, args) @@ -237,7 +237,7 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { .map(|e| create_physical_name(e, false)) .collect::>>()? .join(", "); - strings.push(format!("({})", exprs_str)); + strings.push(format!("({exprs_str})")); } Ok(format!("GROUPING SETS ({})", strings.join(", "))) } @@ -251,9 +251,9 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { let expr = create_physical_name(expr, false)?; let list = list.iter().map(|expr| create_physical_name(expr, false)); if *negated { - Ok(format!("{} NOT IN ({:?})", expr, list)) + Ok(format!("{expr} NOT IN ({list:?})")) } else { - Ok(format!("{} IN ({:?})", expr, list)) + Ok(format!("{expr} IN ({list:?})")) } } Expr::Exists { .. } => Err(DataFusionError::NotImplemented( @@ -275,9 +275,9 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { let low = create_physical_name(low, false)?; let high = create_physical_name(high, false)?; if *negated { - Ok(format!("{} NOT BETWEEN {} AND {}", expr, low, high)) + Ok(format!("{expr} NOT BETWEEN {low} AND {high}")) } else { - Ok(format!("{} BETWEEN {} AND {}", expr, low, high)) + Ok(format!("{expr} BETWEEN {low} AND {high}")) } } Expr::Like(Like { @@ -289,14 +289,14 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { let expr = create_physical_name(expr, false)?; let pattern = create_physical_name(pattern, false)?; let escape = if let Some(char) = escape_char { - format!("CHAR '{}'", char) + format!("CHAR '{char}'") } else { "".to_string() }; if *negated { - Ok(format!("{} NOT LIKE {}{}", expr, pattern, escape)) + Ok(format!("{expr} NOT LIKE {pattern}{escape}")) } else { - Ok(format!("{} LIKE {}{}", expr, pattern, escape)) + Ok(format!("{expr} LIKE {pattern}{escape}")) } } Expr::ILike(Like { @@ -308,14 +308,14 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { let expr = create_physical_name(expr, false)?; let pattern = create_physical_name(pattern, false)?; let escape = if let Some(char) = escape_char { - format!("CHAR '{}'", char) + format!("CHAR '{char}'") } else { "".to_string() }; if *negated { - Ok(format!("{} NOT ILIKE {}{}", expr, pattern, escape)) + Ok(format!("{expr} NOT ILIKE {pattern}{escape}")) } else { - Ok(format!("{} ILIKE {}{}", expr, pattern, escape)) + Ok(format!("{expr} ILIKE {pattern}{escape}")) } } Expr::SimilarTo(Like { @@ -327,14 +327,14 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { let expr = create_physical_name(expr, false)?; let pattern = create_physical_name(pattern, false)?; let escape = if let Some(char) = escape_char { - format!("CHAR '{}'", char) + format!("CHAR '{char}'") } else { "".to_string() }; if *negated { - Ok(format!("{} NOT SIMILAR TO {}{}", expr, pattern, escape)) + Ok(format!("{expr} NOT SIMILAR TO {pattern}{escape}")) } else { - Ok(format!("{} SIMILAR TO {}{}", expr, pattern, escape)) + Ok(format!("{expr} SIMILAR TO {pattern}{escape}")) } } Expr::Sort { .. } => Err(DataFusionError::Internal( @@ -1581,8 +1581,7 @@ pub fn create_window_expr_with_name( ) } other => Err(DataFusionError::Internal(format!( - "Invalid window expression '{:?}'", - other + "Invalid window expression '{other:?}'" ))), } } @@ -1658,8 +1657,7 @@ pub fn create_aggregate_expr_with_name( udaf::create_aggregate_expr(fun, &args, physical_input_schema, name) } other => Err(DataFusionError::Internal(format!( - "Invalid aggregate expression '{:?}'", - other + "Invalid aggregate expression '{other:?}'" ))), } } @@ -1876,7 +1874,7 @@ mod tests { // the cast from u8 to i64 for literal will be simplified, and get lit(int64(5)) // the cast here is implicit so has CastOptions with safe=true let expected = "BinaryExpr { left: Column { name: \"c7\", index: 2 }, op: Lt, right: Literal { value: Int64(5) } }"; - assert!(format!("{:?}", exec_plan).contains(expected)); + assert!(format!("{exec_plan:?}").contains(expected)); Ok(()) } @@ -1902,7 +1900,7 @@ mod tests { let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[false, false, false], [true, false, false], [false, true, false], [false, false, true], [true, true, false], [true, false, true], [false, true, true], [true, true, true]] })"#; - assert_eq!(format!("{:?}", cube), expected); + assert_eq!(format!("{cube:?}"), expected); Ok(()) } @@ -1929,7 +1927,7 @@ mod tests { let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[true, true, true], [false, true, true], [false, false, true], [false, false, false]] })"#; - assert_eq!(format!("{:?}", rollup), expected); + assert_eq!(format!("{rollup:?}"), expected); Ok(()) } @@ -1949,7 +1947,7 @@ mod tests { )?; let expected = expressions::not(expressions::col("a", &schema)?)?; - assert_eq!(format!("{:?}", expr), format!("{:?}", expected)); + assert_eq!(format!("{expr:?}"), format!("{expected:?}")); Ok(()) } @@ -1967,7 +1965,7 @@ mod tests { // c12 is f64, c7 is u8 -> cast c7 to f64 // the cast here is implicit so has CastOptions with safe=true let _expected = "predicate: BinaryExpr { left: TryCastExpr { expr: Column { name: \"c7\", index: 6 }, cast_type: Float64 }, op: Lt, right: Column { name: \"c12\", index: 11 } }"; - let plan_debug_str = format!("{:?}", plan); + let plan_debug_str = format!("{plan:?}"); assert!(plan_debug_str.contains("GlobalLimitExec")); assert!(plan_debug_str.contains("skip: 3")); Ok(()) @@ -1977,7 +1975,7 @@ mod tests { async fn test_with_zero_offset_plan() -> Result<()> { let logical_plan = test_csv_scan().await?.limit(0, None)?.build()?; let plan = plan(&logical_plan).await?; - assert!(format!("{:?}", plan).contains("limit: None")); + assert!(format!("{plan:?}").contains("limit: None")); Ok(()) } @@ -1990,12 +1988,12 @@ mod tests { .build()?; let plan = plan(&logical_plan).await?; - assert!(format!("{:?}", plan).contains("GlobalLimitExec")); - assert!(format!("{:?}", plan).contains("skip: 3, fetch: Some(5)")); + assert!(format!("{plan:?}").contains("GlobalLimitExec")); + assert!(format!("{plan:?}").contains("skip: 3, fetch: Some(5)")); // LocalLimitExec adjusts the `fetch` - assert!(format!("{:?}", plan).contains("LocalLimitExec")); - assert!(format!("{:?}", plan).contains("fetch: 8")); + assert!(format!("{plan:?}").contains("LocalLimitExec")); + assert!(format!("{plan:?}").contains("fetch: 8")); Ok(()) } @@ -2023,8 +2021,7 @@ mod tests { for case in cases { let logical_plan = test_csv_scan().await?.project(vec![case.clone()]); let message = format!( - "Expression {:?} expected to error due to impossible coercion", - case + "Expression {case:?} expected to error due to impossible coercion" ); assert!(logical_plan.is_err(), "{}", message); } @@ -2048,9 +2045,7 @@ mod tests { Ok(_) => panic!("Expected planning failure"), Err(e) => assert!( e.to_string().contains(expected_error), - "Error '{}' did not contain expected error '{}'", - e, - expected_error + "Error '{e}' did not contain expected error '{expected_error}'" ), } } @@ -2095,9 +2090,7 @@ mod tests { Ok(_) => panic!("Expected planning failure"), Err(e) => assert!( e.to_string().contains(expected_error), - "Error '{}' did not contain expected error '{}'", - e, - expected_error + "Error '{e}' did not contain expected error '{expected_error}'" ), } } @@ -2117,7 +2110,7 @@ mod tests { let expected = "expr: [(BinaryExpr { left: BinaryExpr { left: Column { name: \"c1\", index: 0 }, op: Eq, right: Literal { value: Utf8(\"1\") } }, op: Or, right: BinaryExpr { left: Column { name: \"c1\", index: 0 }, op: Eq, right: Literal { value: Utf8(\"a\") } } }"; - let actual = format!("{:?}", execution_plan); + let actual = format!("{execution_plan:?}"); assert!(actual.contains(expected), "{}", actual); Ok(()) @@ -2209,7 +2202,7 @@ mod tests { .build()?; let execution_plan = plan(&logical_plan).await?; - let formatted = format!("{:?}", execution_plan); + let formatted = format!("{execution_plan:?}"); // Make sure the plan contains a FinalPartitioned, which means it will not use the Final // mode in Aggregate (which is slower) @@ -2240,7 +2233,7 @@ mod tests { .build()?; let execution_plan = plan(&logical_plan).await?; - let formatted = format!("{:?}", execution_plan); + let formatted = format!("{execution_plan:?}"); // Make sure the plan contains a FinalPartitioned, which means it will not use the Final // mode in Aggregate (which is slower) @@ -2261,7 +2254,7 @@ mod tests { .build()?; let execution_plan = plan(&logical_plan).await?; - let formatted = format!("{:?}", execution_plan); + let formatted = format!("{execution_plan:?}"); // Make sure the plan contains a FinalPartitioned, which means it will not use the Final // mode in Aggregate (which is slower) @@ -2443,7 +2436,7 @@ mod tests { async fn test_csv_scan_with_name(name: &str) -> Result { let ctx = SessionContext::new(); let testdata = crate::test_util::arrow_test_data(); - let path = format!("{}/csv/aggregate_test_100.csv", testdata); + let path = format!("{testdata}/csv/aggregate_test_100.csv"); let options = CsvReadOptions::new().schema_infer_max_records(100); let logical_plan = match ctx.read_csv(path, options).await?.into_optimized_plan()? { @@ -2466,7 +2459,7 @@ mod tests { async fn test_csv_scan() -> Result { let ctx = SessionContext::new(); let testdata = crate::test_util::arrow_test_data(); - let path = format!("{}/csv/aggregate_test_100.csv", testdata); + let path = format!("{testdata}/csv/aggregate_test_100.csv"); let options = CsvReadOptions::new().schema_infer_max_records(100); Ok(LogicalPlanBuilder::from( ctx.read_csv(path, options).await?.into_optimized_plan()?, diff --git a/datafusion/core/src/physical_plan/projection.rs b/datafusion/core/src/physical_plan/projection.rs index bc8087c9819f..14431b88951b 100644 --- a/datafusion/core/src/physical_plan/projection.rs +++ b/datafusion/core/src/physical_plan/projection.rs @@ -248,7 +248,7 @@ impl ExecutionPlan for ProjectionExec { .map(|(e, alias)| { let e = e.to_string(); if &e != alias { - format!("{} as {}", e, alias) + format!("{e} as {alias}") } else { e } diff --git a/datafusion/core/src/physical_plan/repartition.rs b/datafusion/core/src/physical_plan/repartition.rs index 967281c1c293..ee2e976cecdb 100644 --- a/datafusion/core/src/physical_plan/repartition.rs +++ b/datafusion/core/src/physical_plan/repartition.rs @@ -103,8 +103,7 @@ impl BatchPartitioner { }, other => { return Err(DataFusionError::NotImplemented(format!( - "Unsupported repartitioning scheme {:?}", - other + "Unsupported repartitioning scheme {other:?}" ))) } }; @@ -776,8 +775,7 @@ mod tests { assert!( result_string .contains("Unsupported repartitioning scheme UnknownPartitioning(1)"), - "actual: {}", - result_string + "actual: {result_string}" ); } @@ -803,8 +801,7 @@ mod tests { .to_string(); assert!( result_string.contains("ErrorExec, unsurprisingly, errored in partition 0"), - "actual: {}", - result_string + "actual: {result_string}" ); } @@ -838,8 +835,7 @@ mod tests { .to_string(); assert!( result_string.contains("bad data error"), - "actual: {}", - result_string + "actual: {result_string}" ); } diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 89b7d414f6d9..b75fe0d80d6c 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -97,10 +97,9 @@ impl ExternalSorter { ) -> Self { let metrics = metrics_set.new_intermediate_baseline(partition_id); - let reservation = - MemoryConsumer::new(format!("ExternalSorter[{}]", partition_id)) - .with_can_spill(true) - .register(&runtime.memory_pool); + let reservation = MemoryConsumer::new(format!("ExternalSorter[{partition_id}]")) + .with_can_spill(true) + .register(&runtime.memory_pool); Self { schema, @@ -567,8 +566,7 @@ async fn spill_partial_sorted_stream( match handle.await { Ok(r) => r, Err(e) => Err(DataFusionError::Execution(format!( - "Error occurred while spilling {}", - e + "Error occurred while spilling {e}" ))), } } @@ -618,7 +616,7 @@ fn read_spill(sender: Sender>, path: &Path) -> Result<( for batch in reader { sender .blocking_send(batch) - .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e}")))?; } Ok(()) } @@ -1107,7 +1105,7 @@ mod tests { let metrics = sort_exec.metrics().unwrap(); let did_it_spill = metrics.spill_count().unwrap() > 0; - assert_eq!(did_it_spill, expect_spillage, "with fetch: {:?}", fetch); + assert_eq!(did_it_spill, expect_spillage, "with fetch: {fetch:?}"); } Ok(()) } diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index f069cc5b007c..a5800746bcdb 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -164,8 +164,7 @@ impl ExecutionPlan for SortPreservingMergeExec { ); if 0 != partition { return Err(DataFusionError::Internal(format!( - "SortPreservingMergeExec invalid partition {}", - partition + "SortPreservingMergeExec invalid partition {partition}" ))); } @@ -1009,8 +1008,7 @@ mod tests { assert_eq!( basic, partition, - "basic:\n\n{}\n\npartition:\n\n{}\n\n", - basic, partition + "basic:\n\n{basic}\n\npartition:\n\n{partition}\n\n" ); } @@ -1286,8 +1284,7 @@ mod tests { assert_eq!( basic, partition, - "basic:\n\n{}\n\npartition:\n\n{}\n\n", - basic, partition + "basic:\n\n{basic}\n\npartition:\n\n{partition}\n\n" ); } diff --git a/datafusion/core/src/physical_plan/streaming.rs b/datafusion/core/src/physical_plan/streaming.rs index 456c60e8a17a..cd9ff1938e0d 100644 --- a/datafusion/core/src/physical_plan/streaming.rs +++ b/datafusion/core/src/physical_plan/streaming.rs @@ -98,8 +98,7 @@ impl ExecutionPlan for StreamingTableExec { _children: Vec>, ) -> Result> { Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self + "Children cannot be replaced in {self:?}" ))) } diff --git a/datafusion/core/src/physical_plan/udaf.rs b/datafusion/core/src/physical_plan/udaf.rs index 2b9ab65e694e..6f55b31e4951 100644 --- a/datafusion/core/src/physical_plan/udaf.rs +++ b/datafusion/core/src/physical_plan/udaf.rs @@ -80,7 +80,7 @@ impl AggregateExpr for AggregateFunctionExpr { .enumerate() .map(|(i, data_type)| { Field::new( - format_state_name(&self.name, &format!("{}", i)), + format_state_name(&self.name, &format!("{i}")), data_type.clone(), true, ) diff --git a/datafusion/core/src/physical_plan/union.rs b/datafusion/core/src/physical_plan/union.rs index 3a92cd12385b..9663c1fad7de 100644 --- a/datafusion/core/src/physical_plan/union.rs +++ b/datafusion/core/src/physical_plan/union.rs @@ -231,8 +231,7 @@ impl ExecutionPlan for UnionExec { warn!("Error in Union: Partition {} not found", partition); Err(crate::error::DataFusionError::Execution(format!( - "Partition {} not found in Union", - partition + "Partition {partition} not found in Union" ))) } diff --git a/datafusion/core/src/physical_plan/values.rs b/datafusion/core/src/physical_plan/values.rs index 6ab4f7b82490..2ac2dd1ae060 100644 --- a/datafusion/core/src/physical_plan/values.rs +++ b/datafusion/core/src/physical_plan/values.rs @@ -73,8 +73,7 @@ impl ValuesExec { } Ok(ColumnarValue::Array(a)) => { Err(DataFusionError::Plan(format!( - "Cannot have array values {:?} in a values list", - a + "Cannot have array values {a:?} in a values list" ))) } Err(err) => Err(err), @@ -135,8 +134,7 @@ impl ExecutionPlan for ValuesExec { // GlobalLimitExec has a single output partition if 0 != partition { return Err(DataFusionError::Internal(format!( - "ValuesExec invalid partition {} (expected 0)", - partition + "ValuesExec invalid partition {partition} (expected 0)" ))); } diff --git a/datafusion/core/src/physical_plan/windows/mod.rs b/datafusion/core/src/physical_plan/windows/mod.rs index 4a261c0f3bc6..bbf6c91821f4 100644 --- a/datafusion/core/src/physical_plan/windows/mod.rs +++ b/datafusion/core/src/physical_plan/windows/mod.rs @@ -100,7 +100,7 @@ fn get_scalar_value_from_args( .as_any() .downcast_ref::() .ok_or_else(|| DataFusionError::NotImplemented( - format!("There is only support Literal types for field at idx: {} in Window Function", index), + format!("There is only support Literal types for field at idx: {index} in Window Function"), ))? .value() .clone(); @@ -165,7 +165,7 @@ fn create_built_in_window_expr( let n: i64 = n .clone() .try_into() - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; let n: u32 = n as u32; let data_type = args[0].data_type(input_schema)?; Arc::new(NthValue::nth(name, arg, data_type, n)?) diff --git a/datafusion/core/src/scheduler/mod.rs b/datafusion/core/src/scheduler/mod.rs index e9e8aa7755ae..d552784fc1ba 100644 --- a/datafusion/core/src/scheduler/mod.rs +++ b/datafusion/core/src/scheduler/mod.rs @@ -105,7 +105,7 @@ impl SchedulerBuilder { let builder = ThreadPoolBuilder::new() .num_threads(num_threads) .panic_handler(|p| error!("{}", format_worker_panic(p))) - .thread_name(|idx| format!("df-worker-{}", idx)); + .thread_name(|idx| format!("df-worker-{idx}")); Self { inner: builder } } @@ -181,7 +181,7 @@ fn format_worker_panic(panic: Box) -> String { "UNKNOWN" }; - format!("worker {} panicked with: {}", worker, message) + format!("worker {worker} panicked with: {message}") } /// Returns `true` if the current thread is a rayon worker thread @@ -373,8 +373,7 @@ mod tests { assert_eq!( expected, scheduled, - "\n\nexpected:\n\n{}\nactual:\n\n{}\n\n", - expected, scheduled + "\n\nexpected:\n\n{expected}\nactual:\n\n{scheduled}\n\n" ); } } diff --git a/datafusion/core/src/scheduler/pipeline/repartition.rs b/datafusion/core/src/scheduler/pipeline/repartition.rs index c35ab909fde8..7eeb3c31de10 100644 --- a/datafusion/core/src/scheduler/pipeline/repartition.rs +++ b/datafusion/core/src/scheduler/pipeline/repartition.rs @@ -82,8 +82,7 @@ impl Pipeline for RepartitionPipeline { let mut state = self.state.lock(); assert!( !state.partition_closed[partition], - "attempt to push to closed partition {} of RepartitionPipeline({:?})", - partition, state + "attempt to push to closed partition {partition} of RepartitionPipeline({state:?})" ); let state = &mut *state; @@ -99,8 +98,7 @@ impl Pipeline for RepartitionPipeline { let mut state = self.state.lock(); assert!( !state.partition_closed[partition], - "attempt to close already closed partition {} of RepartitionPipeline({:?})", - partition, state + "attempt to close already closed partition {partition} of RepartitionPipeline({state:?})" ); state.partition_closed[partition] = true; diff --git a/datafusion/core/src/test/exec.rs b/datafusion/core/src/test/exec.rs index f74b1eb311a8..956774b62cad 100644 --- a/datafusion/core/src/test/exec.rs +++ b/datafusion/core/src/test/exec.rs @@ -189,7 +189,7 @@ impl ExecutionPlan for MockExec { for batch in data { println!("Sending batch via delayed stream"); if let Err(e) = tx.send(batch).await { - println!("ERROR batch via delayed stream: {}", e); + println!("ERROR batch via delayed stream: {e}"); } } }); @@ -314,12 +314,12 @@ impl ExecutionPlan for BarrierExec { let data = self.data[partition].clone(); let b = self.barrier.clone(); let join_handle = tokio::task::spawn(async move { - println!("Partition {} waiting on barrier", partition); + println!("Partition {partition} waiting on barrier"); b.wait().await; for batch in data { - println!("Partition {} sending batch", partition); + println!("Partition {partition} sending batch"); if let Err(e) = tx.send(Ok(batch)).await { - println!("ERROR batch via barrier stream stream: {}", e); + println!("ERROR batch via barrier stream stream: {e}"); } } }); @@ -407,8 +407,7 @@ impl ExecutionPlan for ErrorExec { _context: Arc, ) -> Result { Err(DataFusionError::Internal(format!( - "ErrorExec, unsurprisingly, errored in partition {}", - partition + "ErrorExec, unsurprisingly, errored in partition {partition}" ))) } @@ -641,8 +640,7 @@ impl ExecutionPlan for BlockingExec { _: Vec>, ) -> Result> { Err(DataFusionError::Internal(format!( - "Children cannot be replaced in {:?}", - self + "Children cannot be replaced in {self:?}" ))) } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index cf7594c969ff..737893f51845 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -96,7 +96,7 @@ pub fn partitioned_file_groups( file_type: FileType, file_compression_type: FileCompressionType, ) -> Result>> { - let path = format!("{}/{}", path, filename); + let path = format!("{path}/{filename}"); let tmp_dir = TempDir::new()?.into_path(); diff --git a/datafusion/core/src/test_util.rs b/datafusion/core/src/test_util.rs index 8d41ec935004..2db6f3bc46ba 100644 --- a/datafusion/core/src/test_util.rs +++ b/datafusion/core/src/test_util.rs @@ -121,7 +121,7 @@ macro_rules! assert_batches_sorted_eq { pub fn arrow_test_data() -> String { match get_data_dir("ARROW_TEST_DATA", "../../testing/data") { Ok(pb) => pb.display().to_string(), - Err(err) => panic!("failed to get arrow data dir: {}", err), + Err(err) => panic!("failed to get arrow data dir: {err}"), } } @@ -143,7 +143,7 @@ pub fn arrow_test_data() -> String { pub fn parquet_test_data() -> String { match get_data_dir("PARQUET_TEST_DATA", "../../parquet-testing/data") { Ok(pb) => pb.display().to_string(), - Err(err) => panic!("failed to get parquet data dir: {}", err), + Err(err) => panic!("failed to get parquet data dir: {err}"), } } diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index b97473e14cef..ba17c56e4efa 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -230,11 +230,10 @@ async fn custom_source_dataframe() -> Result<()> { } let expected = format!( - "Projection: {}.c2\ - \n TableScan: {} projection=[c2]", - UNNAMED_TABLE, UNNAMED_TABLE + "Projection: {UNNAMED_TABLE}.c2\ + \n TableScan: {UNNAMED_TABLE} projection=[c2]" ); - assert_eq!(format!("{:?}", optimized_plan), expected); + assert_eq!(format!("{optimized_plan:?}"), expected); let physical_plan = state.create_physical_plan(&optimized_plan).await?; @@ -266,8 +265,7 @@ async fn optimizers_catch_all_statistics() { // when the optimization kicks in, the source is replaced by an EmptyExec assert!( contains_empty_exec(Arc::clone(&physical_plan)), - "Expected aggregate_statistics optimizations missing: {:?}", - physical_plan + "Expected aggregate_statistics optimizations missing: {physical_plan:?}" ); let expected = RecordBatch::try_new( @@ -288,7 +286,7 @@ async fn optimizers_catch_all_statistics() { let actual = collect(physical_plan, task_ctx).await.unwrap(); assert_eq!(actual.len(), 1); - assert_eq!(format!("{:?}", actual[0]), format!("{:?}", expected)); + assert_eq!(format!("{:?}", actual[0]), format!("{expected:?}")); } fn contains_empty_exec(plan: Arc) -> bool { diff --git a/datafusion/core/tests/dataframe.rs b/datafusion/core/tests/dataframe.rs index b57cd295522f..00ba4524d02c 100644 --- a/datafusion/core/tests/dataframe.rs +++ b/datafusion/core/tests/dataframe.rs @@ -383,7 +383,7 @@ async fn aggregates_table(ctx: &SessionContext) -> Result { let testdata = datafusion::test_util::arrow_test_data(); ctx.read_csv( - format!("{}/csv/aggregate_test_100.csv", testdata), + format!("{testdata}/csv/aggregate_test_100.csv"), CsvReadOptions::default(), ) .await diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 677347594c82..ad5c0d09db6e 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -111,7 +111,7 @@ mod unix_test { let test_data = arrow_test_data(); ctx.register_csv( "right", - &format!("{}/csv/aggregate_test_100.csv", test_data), + &format!("{test_data}/csv/aggregate_test_100.csv"), CsvReadOptions::new().schema(schema.as_ref()), ) .await?; @@ -194,7 +194,7 @@ mod unix_test { if cnt % TEST_BATCH_SIZE == 0 { thread_tx.send(Operation::Write).unwrap(); } - let line = format!("{},{}\n", string_col, cnt).to_owned(); + let line = format!("{string_col},{cnt}\n").to_owned(); write_to_fifo(&first_file, &line, execution_start, broken_pipe_timeout) .unwrap(); } diff --git a/datafusion/core/tests/merge_fuzz.rs b/datafusion/core/tests/merge_fuzz.rs index 64738c3ffaa3..6411f31be0ce 100644 --- a/datafusion/core/tests/merge_fuzz.rs +++ b/datafusion/core/tests/merge_fuzz.rs @@ -140,7 +140,7 @@ async fn run_merge_test(input: Vec>) { let expected = partitions_to_sorted_vec(&input); let actual = batches_to_vec(&collected); - assert_eq!(expected, actual, "failure in @ batch_size {}", batch_size); + assert_eq!(expected, actual, "failure in @ batch_size {batch_size}"); } } diff --git a/datafusion/core/tests/order_spill_fuzz.rs b/datafusion/core/tests/order_spill_fuzz.rs index 923b44e2681b..42a7d58d2785 100644 --- a/datafusion/core/tests/order_spill_fuzz.rs +++ b/datafusion/core/tests/order_spill_fuzz.rs @@ -98,7 +98,7 @@ async fn run_sort(pool_size: usize, size_spill: Vec<(usize, bool)>) { 0, "The sort should have returned all memory used back to the memory pool" ); - assert_eq!(expected, actual, "failure in @ pool_size {}", pool_size); + assert_eq!(expected, actual, "failure in @ pool_size {pool_size}"); } } diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index f40258219a32..30802805cee3 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -221,7 +221,7 @@ impl AsyncFileReader for ParquetFileReader { self.store .get_range(&self.meta.location, range) .map_err(|e| { - ParquetError::General(format!("AsyncChunkReader::get_bytes error: {}", e)) + ParquetError::General(format!("AsyncChunkReader::get_bytes error: {e}")) }) .boxed() } @@ -238,8 +238,7 @@ impl AsyncFileReader for ParquetFileReader { .await .map_err(|e| { ParquetError::General(format!( - "AsyncChunkReader::get_metadata error: {}", - e + "AsyncChunkReader::get_metadata error: {e}" )) })?; Ok(Arc::new(metadata)) diff --git a/datafusion/core/tests/parquet/filter_pushdown.rs b/datafusion/core/tests/parquet/filter_pushdown.rs index 101a8e14170a..f7f22417297f 100644 --- a/datafusion/core/tests/parquet/filter_pushdown.rs +++ b/datafusion/core/tests/parquet/filter_pushdown.rs @@ -54,7 +54,7 @@ fn generate_file(tempdir: &TempDir, props: WriterProperties) -> TestParquetFile let file = tempdir.path().join("data.parquet"); let start = Instant::now(); - println!("Writing test data to {:?}", file); + println!("Writing test data to {file:?}"); let test_parquet_file = TestParquetFile::try_new(file, props, generator).unwrap(); println!( "Completed generating test data in {:?}", @@ -513,7 +513,7 @@ impl<'a> TestCase<'a> { filter: &Expr, ) -> RecordBatch { println!(" scan options: {scan_options:?}"); - println!(" reading with filter {:?}", filter); + println!(" reading with filter {filter:?}"); let ctx = SessionContext::with_config(scan_options.config()); let exec = self .test_parquet_file @@ -548,7 +548,7 @@ impl<'a> TestCase<'a> { }; let pushdown_rows_filtered = get_value(&metrics, "pushdown_rows_filtered"); - println!(" pushdown_rows_filtered: {}", pushdown_rows_filtered); + println!(" pushdown_rows_filtered: {pushdown_rows_filtered}"); match pushdown_expected { PushdownExpected::None => { @@ -564,7 +564,7 @@ impl<'a> TestCase<'a> { }; let page_index_rows_filtered = get_value(&metrics, "page_index_rows_filtered"); - println!(" page_index_rows_filtered: {}", page_index_rows_filtered); + println!(" page_index_rows_filtered: {page_index_rows_filtered}"); let page_index_filtering_expected = if scan_options.enable_page_index { self.page_index_filtering_expected @@ -595,8 +595,7 @@ fn get_value(metrics: &MetricsSet, metric_name: &str) -> usize { Some(v) => v.as_usize(), _ => { panic!( - "Expected metric not found. Looking for '{}' in\n\n{:#?}", - metric_name, metrics + "Expected metric not found. Looking for '{metric_name}' in\n\n{metrics:#?}" ); } } diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index d3bf4909c7f6..93b276210354 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -167,7 +167,7 @@ impl ContextWithParquet { /// runs a query like "SELECT * from t WHERE and returns /// the number of output rows and normalized execution metrics async fn query_with_expr(&mut self, expr: Expr) -> TestOutput { - let sql = format!("EXPR only: {:?}", expr); + let sql = format!("EXPR only: {expr:?}"); let logical_plan = LogicalPlanBuilder::scan( "t", provider_as_source(self.provider.clone()), @@ -184,7 +184,7 @@ impl ContextWithParquet { /// Runs the specified SQL query and returns the number of output /// rows and normalized execution metrics async fn query(&mut self, sql: &str) -> TestOutput { - println!("Planning sql {}", sql); + println!("Planning sql {sql}"); let logical_plan = self .ctx .sql(sql) @@ -309,7 +309,7 @@ fn make_timestamp_batch(offset: Duration) -> RecordBatch { let names = ts_nanos .iter() .enumerate() - .map(|(i, _)| format!("Row {} + {}", i, offset)) + .map(|(i, _)| format!("Row {i} + {offset}")) .collect::>(); let arr_nanos = TimestampNanosecondArray::from(ts_nanos); @@ -400,7 +400,7 @@ fn make_date_batch(offset: Duration) -> RecordBatch { let names = date_strings .iter() .enumerate() - .map(|(i, val)| format!("Row {} + {}: {:?}", i, offset, val)) + .map(|(i, val)| format!("Row {i} + {offset}: {val:?}")) .collect::>(); // Copied from `cast.rs` cast kernel due to lack of temporal kernels diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 851c15e8cd90..61e74e80d047 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -36,7 +36,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { let store = state.runtime_env().object_store(&object_store_url).unwrap(); let testdata = datafusion::test_util::parquet_test_data(); - let filename = format!("{}/alltypes_tiny_pages.parquet", testdata); + let filename = format!("{testdata}/alltypes_tiny_pages.parquet"); let location = Path::from_filesystem_path(filename.as_str()).unwrap(); let metadata = std::fs::metadata(filename).expect("Local file metadata"); diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index f04ed032b12c..2d257d49a389 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -132,7 +132,7 @@ async fn parquet_distinct_partition_col() -> Result<()> { min_limit -= 1; - let sql_cross_partition_boundary = format!("SELECT month FROM t limit {}", max_limit); + let sql_cross_partition_boundary = format!("SELECT month FROM t limit {max_limit}"); let resulting_limit: i64 = ctx .sql(sql_cross_partition_boundary.as_str()) .await? @@ -144,8 +144,7 @@ async fn parquet_distinct_partition_col() -> Result<()> { assert_eq!(max_limit, resulting_limit); - let sql_within_partition_boundary = - format!("SELECT month from t limit {}", min_limit); + let sql_within_partition_boundary = format!("SELECT month from t limit {min_limit}"); let resulting_limit: i64 = ctx .sql(sql_within_partition_boundary.as_str()) .await? @@ -160,7 +159,7 @@ async fn parquet_distinct_partition_col() -> Result<()> { let s = ScalarValue::try_from_array(results[0].column(1), 0)?; let month = match extract_as_utf(&s) { Some(month) => month, - s => panic!("Expected month as Dict(_, Utf8) found {:?}", s), + s => panic!("Expected month as Dict(_, Utf8) found {s:?}"), }; let sql_on_partition_boundary = format!( @@ -516,7 +515,7 @@ fn register_partitioned_aggregate_csv( table_path: &str, ) { let testdata = arrow_test_data(); - let csv_file_path = format!("{}/csv/aggregate_test_100.csv", testdata); + let csv_file_path = format!("{testdata}/csv/aggregate_test_100.csv"); let file_schema = test_util::aggr_test_schema(); ctx.runtime_env().register_object_store( "mirror", @@ -550,7 +549,7 @@ async fn register_partitioned_alltypes_parquet( source_file: &str, ) { let testdata = parquet_test_data(); - let parquet_file_path = format!("{}/{}", testdata, source_file); + let parquet_file_path = format!("{testdata}/{source_file}"); ctx.runtime_env().register_object_store( "mirror", "", @@ -597,7 +596,7 @@ pub struct MirroringObjectStore { impl std::fmt::Display for MirroringObjectStore { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{:?}", self) + write!(f, "{self:?}") } } diff --git a/datafusion/core/tests/provider_filter_pushdown.rs b/datafusion/core/tests/provider_filter_pushdown.rs index 13160fd52e1a..36b0789829a2 100644 --- a/datafusion/core/tests/provider_filter_pushdown.rs +++ b/datafusion/core/tests/provider_filter_pushdown.rs @@ -163,22 +163,19 @@ impl TableProvider for CustomProvider { ScalarValue::Int64(Some(v)) => *v, other_value => { return Err(DataFusionError::NotImplemented(format!( - "Do not support value {:?}", - other_value + "Do not support value {other_value:?}" ))); } }, other_expr => { return Err(DataFusionError::NotImplemented(format!( - "Do not support expr {:?}", - other_expr + "Do not support expr {other_expr:?}" ))); } }, other_expr => { return Err(DataFusionError::NotImplemented(format!( - "Do not support expr {:?}", - other_expr + "Do not support expr {other_expr:?}" ))); } }; @@ -222,7 +219,7 @@ async fn assert_provider_row_count(value: i64, expected_count: i64) -> Result<() ctx.register_table("data", Arc::new(provider))?; let sql_results = ctx - .sql(&format!("select count(*) from data where flag = {}", value)) + .sql(&format!("select count(*) from data where flag = {value}")) .await? .collect() .await?; diff --git a/datafusion/core/tests/row.rs b/datafusion/core/tests/row.rs index 0652961b0497..c04deb92b92a 100644 --- a/datafusion/core/tests/row.rs +++ b/datafusion/core/tests/row.rs @@ -82,7 +82,7 @@ async fn get_exec( limit: Option, ) -> Result> { let testdata = datafusion::test_util::parquet_test_data(); - let filename = format!("{}/{}", testdata, file_name); + let filename = format!("{testdata}/{file_name}"); let path = Path::from_filesystem_path(filename).unwrap(); diff --git a/datafusion/core/tests/sort_key_cursor.rs b/datafusion/core/tests/sort_key_cursor.rs index 870d47b8260b..7d03ffc87bf5 100644 --- a/datafusion/core/tests/sort_key_cursor.rs +++ b/datafusion/core/tests/sort_key_cursor.rs @@ -209,8 +209,7 @@ fn assert_indexes( assert_eq!( expected_lines, actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines + "\n\nexpected:\n\n{expected_lines:#?}\nactual:\n\n{actual_lines:#?}\n\n" ); } diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index 77e9558f5878..89077ae192d8 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -1000,7 +1000,7 @@ async fn run_count_distinct_integers_aggregated_scenario( ])); for (i, partition) in partitions.iter().enumerate() { - let filename = format!("partition-{}.csv", i); + let filename = format!("partition-{i}.csv"); let file_path = tmp_dir.path().join(filename); let mut file = File::create(file_path)?; for row in partition { diff --git a/datafusion/core/tests/sql/errors.rs b/datafusion/core/tests/sql/errors.rs index 1a02c3442a1c..ed15631a681d 100644 --- a/datafusion/core/tests/sql/errors.rs +++ b/datafusion/core/tests/sql/errors.rs @@ -123,12 +123,11 @@ async fn invalid_qualified_table_references() -> Result<()> { "nonexistentcatalog.public.aggregate_test_100", "way.too.many.namespaces.as.ident.prefixes.aggregate_test_100", ] { - let sql = format!("SELECT COUNT(*) FROM {}", table_ref); + let sql = format!("SELECT COUNT(*) FROM {table_ref}"); let result = ctx.sql(&sql).await; assert!( matches!(result, Err(DataFusionError::Plan(_))), - "result was: {:?}", - result + "result was: {result:?}" ); } Ok(()) diff --git a/datafusion/core/tests/sql/explain.rs b/datafusion/core/tests/sql/explain.rs index 4774f95465d0..041c557f456b 100644 --- a/datafusion/core/tests/sql/explain.rs +++ b/datafusion/core/tests/sql/explain.rs @@ -34,7 +34,7 @@ fn optimize_explain() { if let LogicalPlan::Explain(e) = &plan { assert_eq!(e.stringified_plans.len(), 1); } else { - panic!("plan was not an explain: {:?}", plan); + panic!("plan was not an explain: {plan:?}"); } let ctx = SessionContext::new(); @@ -57,6 +57,6 @@ fn optimize_explain() { assert!(opt, "plans: {:#?}", e.stringified_plans); } else { - panic!("plan was not an explain: {:?}", plan); + panic!("plan was not an explain: {plan:?}"); } } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index d3848c10aa7b..8574aa9b9585 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -41,7 +41,7 @@ async fn explain_analyze_baseline_metrics() { UNION ALL \ SELECT lead(c1, 1) OVER () as cnt FROM (select 1 as c1) AS b \ LIMIT 3"; - println!("running query: {}", sql); + println!("running query: {sql}"); let dataframe = ctx.sql(sql).await.unwrap(); let physical_plan = dataframe.create_physical_plan().await.unwrap(); let task_ctx = ctx.task_ctx(); @@ -49,7 +49,7 @@ async fn explain_analyze_baseline_metrics() { let formatted = arrow::util::pretty::pretty_format_batches(&results) .unwrap() .to_string(); - println!("Query Output:\n\n{}", formatted); + println!("Query Output:\n\n{formatted}"); assert_metrics!( &formatted, @@ -181,13 +181,13 @@ async fn csv_explain_plans() { // Logical plan // Create plan - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let logical_schema = dataframe.schema(); let plan = dataframe.logical_plan(); // - println!("SQL: {}", sql); + println!("SQL: {sql}"); // // Verify schema let expected = vec![ @@ -200,8 +200,7 @@ async fn csv_explain_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // Verify the text format of the plan @@ -215,8 +214,7 @@ async fn csv_explain_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // verify the grahviz format of the plan @@ -252,13 +250,12 @@ async fn csv_explain_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // Optimized logical plan let state = ctx.state(); - let msg = format!("Optimizing logical plan for '{}': {:?}", sql, plan); + let msg = format!("Optimizing logical plan for '{sql}': {plan:?}"); let plan = state.optimize(plan).expect(&msg); let optimized_logical_schema = plan.schema(); // Both schema has to be the same @@ -275,8 +272,7 @@ async fn csv_explain_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // Verify the text format of the plan @@ -290,8 +286,7 @@ async fn csv_explain_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // verify the grahviz format of the plan @@ -327,17 +322,16 @@ async fn csv_explain_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // Physical plan // Create plan - let msg = format!("Creating physical plan for '{}': {:?}", sql, plan); + let msg = format!("Creating physical plan for '{sql}': {plan:?}"); let plan = state.create_physical_plan(&plan).await.expect(&msg); // // Execute plan - let msg = format!("Executing physical plan for '{}': {:?}", sql, plan); + let msg = format!("Executing physical plan for '{sql}': {plan:?}"); let results = collect(plan, state.task_ctx()).await.expect(&msg); let actual = result_vec(&results); // flatten to a single string @@ -405,11 +399,11 @@ async fn csv_explain_verbose_plans() { // Logical plan // Create plan - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let logical_schema = dataframe.schema().clone(); // - println!("SQL: {}", sql); + println!("SQL: {sql}"); // // Verify schema @@ -423,8 +417,7 @@ async fn csv_explain_verbose_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // Verify the text format of the plan @@ -438,8 +431,7 @@ async fn csv_explain_verbose_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // verify the grahviz format of the plan @@ -475,12 +467,11 @@ async fn csv_explain_verbose_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // Optimized logical plan - let msg = format!("Optimizing logical plan for '{}': {:?}", sql, dataframe); + let msg = format!("Optimizing logical plan for '{sql}': {dataframe:?}"); let (state, plan) = dataframe.into_parts(); let plan = state.optimize(&plan).expect(&msg); let optimized_logical_schema = plan.schema(); @@ -498,8 +489,7 @@ async fn csv_explain_verbose_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // Verify the text format of the plan @@ -513,8 +503,7 @@ async fn csv_explain_verbose_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // // verify the grahviz format of the plan @@ -550,17 +539,16 @@ async fn csv_explain_verbose_plans() { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // Physical plan // Create plan - let msg = format!("Creating physical plan for '{}': {:?}", sql, plan); + let msg = format!("Creating physical plan for '{sql}': {plan:?}"); let plan = state.create_physical_plan(&plan).await.expect(&msg); // // Execute plan - let msg = format!("Executing physical plan for '{}': {:?}", sql, plan); + let msg = format!("Executing physical plan for '{sql}': {plan:?}"); let task_ctx = ctx.task_ctx(); let results = collect(plan, task_ctx).await.expect(&msg); let actual = result_vec(&results); @@ -659,7 +647,7 @@ order by \n Filter: lineitem.l_returnflag = Utf8(\"R\")\ \n TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8(\"R\")]\ \n TableScan: nation projection=[n_nationkey, n_name]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -702,8 +690,7 @@ async fn test_physical_plan_display_indent() { .collect::>(); assert_eq!( expected, actual, - "expected:\n{:#?}\nactual:\n\n{:#?}\n", - expected, actual + "expected:\n{expected:#?}\nactual:\n\n{actual:#?}\n" ); } @@ -749,8 +736,7 @@ async fn test_physical_plan_display_indent_multi_children() { assert_eq!( expected, actual, - "expected:\n{:#?}\nactual:\n\n{:#?}\n", - expected, actual + "expected:\n{expected:#?}\nactual:\n\n{actual:#?}\n" ); } diff --git a/datafusion/core/tests/sql/expr.rs b/datafusion/core/tests/sql/expr.rs index 4f02c2d74075..c8248730e93f 100644 --- a/datafusion/core/tests/sql/expr.rs +++ b/datafusion/core/tests/sql/expr.rs @@ -1712,7 +1712,7 @@ async fn comparisons_with_null_lt() { ]; for sql in cases { - println!("Computing: {}", sql); + println!("Computing: {sql}"); let mut actual = execute_to_batches(&ctx, sql).await; assert_eq!(actual.len(), 1); @@ -1760,7 +1760,7 @@ async fn binary_mathematical_operator_with_null_lt() { ]; for sql in cases { - println!("Computing: {}", sql); + println!("Computing: {sql}"); let mut actual = execute_to_batches(&ctx, sql).await; assert_eq!(actual.len(), 1); diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index 5c4171484af5..4cc9628d15ae 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -1349,7 +1349,7 @@ async fn hash_join_with_date32() -> Result<()> { // inner join on hash supported data type (Date32) let sql = "select * from t1 join t2 on t1.c1 = t2.c1"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1363,8 +1363,7 @@ async fn hash_join_with_date32() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -1388,7 +1387,7 @@ async fn hash_join_with_date64() -> Result<()> { // left join on hash supported data type (Date64) let sql = "select * from t1 left join t2 on t1.c2 = t2.c2"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1402,8 +1401,7 @@ async fn hash_join_with_date64() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -1429,7 +1427,7 @@ async fn hash_join_with_decimal() -> Result<()> { // right join on hash supported data type (Decimal) let sql = "select * from t1 right join t2 on t1.c3 = t2.c3"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1443,8 +1441,7 @@ async fn hash_join_with_decimal() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -1470,7 +1467,7 @@ async fn hash_join_with_dictionary() -> Result<()> { // inner join on hash supported data type (Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))) let sql = "select * from t1 join t2 on t1.c4 = t2.c4"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1484,8 +1481,7 @@ async fn hash_join_with_dictionary() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -1511,7 +1507,7 @@ async fn reduce_left_join_1() -> Result<()> { // reduce to inner join let sql = "select * from t1 left join t2 on t1.t1_id = t2.t2_id where t2.t2_id < 100"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1527,8 +1523,7 @@ async fn reduce_left_join_1() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1555,7 +1550,7 @@ async fn reduce_left_join_2() -> Result<()> { // reduce to inner join let sql = "select * from t1 left join t2 on t1.t1_id = t2.t2_id where t2.t2_int < 10 or (t1.t1_int > 2 and t2.t2_name != 'w')"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; @@ -1576,8 +1571,7 @@ async fn reduce_left_join_2() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1604,7 +1598,7 @@ async fn reduce_left_join_3() -> Result<()> { // reduce subquery to inner join let sql = "select * from (select t1.* from t1 left join t2 on t1.t1_id = t2.t2_id where t2.t2_int < 3) t3 left join t2 on t3.t1_int = t2.t2_int where t3.t1_id < 100"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1624,8 +1618,7 @@ async fn reduce_left_join_3() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1650,7 +1643,7 @@ async fn reduce_right_join_1() -> Result<()> { // reduce to inner join let sql = "select * from t1 right join t2 on t1.t1_id = t2.t2_id where t1.t1_int is not null"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1665,8 +1658,7 @@ async fn reduce_right_join_1() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1693,7 +1685,7 @@ async fn reduce_right_join_2() -> Result<()> { // reduce to inner join let sql = "select * from t1 right join t2 on t1.t1_id = t2.t2_id where not(t1.t1_int = t2.t2_int)"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1708,8 +1700,7 @@ async fn reduce_right_join_2() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1736,7 +1727,7 @@ async fn reduce_full_join_to_right_join() -> Result<()> { // reduce to right join let sql = "select * from t1 full join t2 on t1.t1_id = t2.t2_id where t2.t2_name is not null"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1751,8 +1742,7 @@ async fn reduce_full_join_to_right_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1781,7 +1771,7 @@ async fn reduce_full_join_to_left_join() -> Result<()> { // reduce to left join let sql = "select * from t1 full join t2 on t1.t1_id = t2.t2_id where t1.t1_name != 'b'"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1796,8 +1786,7 @@ async fn reduce_full_join_to_left_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1823,7 +1812,7 @@ async fn reduce_full_join_to_inner_join() -> Result<()> { // reduce to inner join let sql = "select * from t1 full join t2 on t1.t1_id = t2.t2_id where t1.t1_name != 'b' and t2.t2_name = 'x'"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -1839,8 +1828,7 @@ async fn reduce_full_join_to_inner_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -1887,7 +1875,7 @@ async fn sort_merge_join_on_date32() -> Result<()> { // inner sort merge join on data type (Date32) let sql = "select * from t1 join t2 on t1.c1 = t2.c1"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = vec![ @@ -1908,8 +1896,7 @@ async fn sort_merge_join_on_date32() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -1933,7 +1920,7 @@ async fn sort_merge_join_on_decimal() -> Result<()> { // right join on data type (Decimal) let sql = "select * from t1 right join t2 on t1.c3 = t2.c3"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = vec![ @@ -1956,8 +1943,7 @@ async fn sort_merge_join_on_decimal() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -1989,7 +1975,7 @@ async fn left_semi_join() -> Result<()> { .unwrap(); let sql = "SELECT t1_id, t1_name FROM t1 WHERE t1_id IN (SELECT t2_id FROM t2) ORDER BY t1_id"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { @@ -2026,8 +2012,7 @@ async fn left_semi_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -2178,7 +2163,7 @@ async fn right_semi_join() -> Result<()> { .unwrap(); let sql = "SELECT t1_id, t1_name, t1_int FROM t1 WHERE EXISTS (SELECT * FROM t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { @@ -2212,8 +2197,7 @@ async fn right_semi_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -2263,7 +2247,7 @@ async fn reduce_cross_join_with_expr_join_key_all() -> Result<()> { // reduce to inner join let sql = "select * from t1 cross join t2 where t1.t1_id + 12 = t2.t2_id + 1"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -2278,8 +2262,7 @@ async fn reduce_cross_join_with_expr_join_key_all() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+---------+--------+-------+---------+--------+", @@ -2306,7 +2289,7 @@ async fn reduce_cross_join_with_cast_expr_join_key() -> Result<()> { let sql = "select t1.t1_id, t2.t2_id, t1.t1_name from t1 cross join t2 where t1.t1_id + 11 = cast(t2.t2_id as BIGINT)"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; let expected = vec![ @@ -2321,8 +2304,7 @@ async fn reduce_cross_join_with_cast_expr_join_key() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ "+-------+-------+---------+", @@ -2350,7 +2332,7 @@ async fn reduce_cross_join_with_wildcard_and_expr() -> Result<()> { let sql = "select *,t1.t1_id+11 from t1,t2 where t1.t1_id+11=t2.t2_id"; // assert logical plan - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan()?; @@ -2366,12 +2348,11 @@ async fn reduce_cross_join_with_wildcard_and_expr() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // assert physical plan - let msg = format!("Creating physical plan for '{}'", sql); + let msg = format!("Creating physical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { @@ -2410,8 +2391,7 @@ async fn reduce_cross_join_with_wildcard_and_expr() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // assert execution result @@ -2443,7 +2423,7 @@ async fn both_side_expr_key_inner_join() -> Result<()> { INNER JOIN t2 \ ON t1.t1_id + cast(12 as INT UNSIGNED) = t2.t2_id + cast(1 as INT UNSIGNED)"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; @@ -2483,8 +2463,7 @@ async fn both_side_expr_key_inner_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -2515,7 +2494,7 @@ async fn left_side_expr_key_inner_join() -> Result<()> { INNER JOIN t2 \ ON t1.t1_id + cast(11 as INT UNSIGNED) = t2.t2_id"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; @@ -2553,8 +2532,7 @@ async fn left_side_expr_key_inner_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -2585,7 +2563,7 @@ async fn right_side_expr_key_inner_join() -> Result<()> { INNER JOIN t2 \ ON t1.t1_id = t2.t2_id - cast(11 as INT UNSIGNED)"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; @@ -2621,8 +2599,7 @@ async fn right_side_expr_key_inner_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -2653,7 +2630,7 @@ async fn select_wildcard_with_expr_key_inner_join() -> Result<()> { INNER JOIN t2 \ ON t1.t1_id = t2.t2_id - cast(11 as INT UNSIGNED)"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; @@ -2689,8 +2666,7 @@ async fn select_wildcard_with_expr_key_inner_join() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -2717,7 +2693,7 @@ async fn join_with_type_coercion_for_equi_expr() -> Result<()> { let sql = "select t1.t1_id, t1.t1_name, t2.t2_id from t1 inner join t2 on t1.t1_id + 11 = t2.t2_id"; // assert logical plan - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan().unwrap(); @@ -2733,8 +2709,7 @@ async fn join_with_type_coercion_for_equi_expr() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -2760,7 +2735,7 @@ async fn join_only_with_filter() -> Result<()> { let sql = "select t1.t1_id, t1.t1_name, t2.t2_id from t1 inner join t2 on t1.t1_id * 4 < t2.t2_id"; // assert logical plan - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan().unwrap(); @@ -2776,8 +2751,7 @@ async fn join_only_with_filter() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ @@ -2804,7 +2778,7 @@ async fn type_coercion_join_with_filter_and_equi_expr() -> Result<()> { on t1.t1_id * 5 = t2.t2_id and t1.t1_id * 4 < t2.t2_id"; // assert logical plan - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan().unwrap(); @@ -2820,8 +2794,7 @@ async fn type_coercion_join_with_filter_and_equi_expr() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let expected = vec![ diff --git a/datafusion/core/tests/sql/json.rs b/datafusion/core/tests/sql/json.rs index b33c0aa46af7..b6b847382959 100644 --- a/datafusion/core/tests/sql/json.rs +++ b/datafusion/core/tests/sql/json.rs @@ -22,7 +22,7 @@ const TEST_DATA_BASE: &str = "tests/jsons"; #[tokio::test] async fn json_query() { let ctx = SessionContext::new(); - let path = format!("{}/2.json", TEST_DATA_BASE); + let path = format!("{TEST_DATA_BASE}/2.json"); ctx.register_json("t1", &path, NdJsonReadOptions::default()) .await .unwrap(); @@ -55,7 +55,7 @@ async fn json_query() { #[should_panic] async fn json_single_nan_schema() { let ctx = SessionContext::new(); - let path = format!("{}/3.json", TEST_DATA_BASE); + let path = format!("{TEST_DATA_BASE}/3.json"); ctx.register_json("single_nan", &path, NdJsonReadOptions::default()) .await .unwrap(); @@ -72,7 +72,7 @@ async fn json_single_nan_schema() { #[cfg_attr(tarpaulin, ignore)] async fn json_explain() { let ctx = SessionContext::new(); - let path = format!("{}/2.json", TEST_DATA_BASE); + let path = format!("{TEST_DATA_BASE}/2.json"); ctx.register_json("t1", &path, NdJsonReadOptions::default()) .await .unwrap(); diff --git a/datafusion/core/tests/sql/limit.rs b/datafusion/core/tests/sql/limit.rs index 646074209541..261e7f819020 100644 --- a/datafusion/core/tests/sql/limit.rs +++ b/datafusion/core/tests/sql/limit.rs @@ -175,11 +175,11 @@ async fn limit_multi_partitions() -> Result<()> { assert_eq!(num_rows, 15); for limit in 1..10 { - let query = format!("SELECT i FROM t limit {}", limit); + let query = format!("SELECT i FROM t limit {limit}"); let results = plan_and_collect(&ctx, &query).await.unwrap(); let num_rows: usize = results.into_iter().map(|b| b.num_rows()).sum(); - assert_eq!(num_rows, limit, "mismatch with query {}", query); + assert_eq!(num_rows, limit, "mismatch with query {query}"); } Ok(()) diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index c7822e5b4a3d..19c9838efa9d 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -132,7 +132,7 @@ where if l.is_nan() || r.is_nan() { assert!(l.is_nan() && r.is_nan()); } else if (l - r).abs() > 2.0 * f64::EPSILON { - panic!("{} != {}", l, r) + panic!("{l} != {r}") } }); } @@ -784,7 +784,7 @@ async fn register_tpch_csv(ctx: &SessionContext, table: &str) -> Result<()> { ctx.register_csv( table, - format!("tests/tpch-csv/{}.csv", table).as_str(), + format!("tests/tpch-csv/{table}.csv").as_str(), CsvReadOptions::new().schema(&schema), ) .await?; @@ -913,9 +913,8 @@ async fn register_aggregate_csv_by_sql(ctx: &SessionContext) { ) STORED AS CSV WITH HEADER ROW - LOCATION '{}/csv/aggregate_test_100.csv' - ", - testdata + LOCATION '{testdata}/csv/aggregate_test_100.csv' + " )) .await .expect("Creating dataframe for CREATE EXTERNAL TABLE"); @@ -1003,7 +1002,7 @@ async fn register_aggregate_csv(ctx: &SessionContext) -> Result<()> { let schema = test_util::aggr_test_schema(); ctx.register_csv( "aggregate_test_100", - &format!("{}/csv/aggregate_test_100.csv", testdata), + &format!("{testdata}/csv/aggregate_test_100.csv"), CsvReadOptions::new().schema(&schema), ) .await?; @@ -1095,7 +1094,7 @@ fn populate_csv_partitions( // generate a partitioned file for partition in 0..partition_count { - let filename = format!("partition-{}.{}", partition, file_extension); + let filename = format!("partition-{partition}.{file_extension}"); let file_path = tmp_dir.path().join(filename); let mut file = File::create(file_path)?; @@ -1201,7 +1200,7 @@ async fn register_alltypes_parquet(ctx: &SessionContext) { let testdata = datafusion::test_util::parquet_test_data(); ctx.register_parquet( "alltypes_plain", - &format!("{}/alltypes_plain.parquet", testdata), + &format!("{testdata}/alltypes_plain.parquet"), ParquetReadOptions::default(), ) .await @@ -1376,7 +1375,7 @@ pub fn make_timestamps() -> RecordBatch { let names = ts_nanos .iter() .enumerate() - .map(|(i, _)| format!("Row {}", i)) + .map(|(i, _)| format!("Row {i}")) .collect::>(); let arr_nanos = TimestampNanosecondArray::from(ts_nanos); @@ -1469,7 +1468,7 @@ pub fn make_times() -> RecordBatch { let names = ts_nanos .iter() .enumerate() - .map(|(i, _)| format!("Row {}", i)) + .map(|(i, _)| format!("Row {i}")) .collect::>(); let arr_nanos = Time64NanosecondArray::from(ts_nanos); diff --git a/datafusion/core/tests/sql/parquet.rs b/datafusion/core/tests/sql/parquet.rs index a7a5dad76bfd..1f1630701052 100644 --- a/datafusion/core/tests/sql/parquet.rs +++ b/datafusion/core/tests/sql/parquet.rs @@ -110,7 +110,7 @@ async fn run_query_with_options(options: ListingOptions, num_files: usize) -> St let ctx = SessionContext::new(); let testdata = datafusion::test_util::parquet_test_data(); - let file_path = format!("{}/alltypes_plain.parquet", testdata); + let file_path = format!("{testdata}/alltypes_plain.parquet"); // Create a directory of parquet files with names // 0.parquet @@ -171,7 +171,7 @@ async fn parquet_single_nan_schema() { let testdata = datafusion::test_util::parquet_test_data(); ctx.register_parquet( "single_nan", - &format!("{}/single_nan.parquet", testdata), + &format!("{testdata}/single_nan.parquet"), ParquetReadOptions::default(), ) .await @@ -192,7 +192,7 @@ async fn parquet_list_columns() { let testdata = datafusion::test_util::parquet_test_data(); ctx.register_parquet( "list_columns", - &format!("{}/list_columns.parquet", testdata), + &format!("{testdata}/list_columns.parquet"), ParquetReadOptions::default(), ) .await diff --git a/datafusion/core/tests/sql/parquet_schema.rs b/datafusion/core/tests/sql/parquet_schema.rs index 3e7bfcc2cc7b..7de83046af11 100644 --- a/datafusion/core/tests/sql/parquet_schema.rs +++ b/datafusion/core/tests/sql/parquet_schema.rs @@ -170,7 +170,7 @@ fn write_files(table_path: &Path, schemas: Vec) { for (i, schema) in schemas.into_iter().enumerate() { let schema = Arc::new(schema); - let filename = format!("part-{}.parquet", i); + let filename = format!("part-{i}.parquet"); let path = table_path.join(filename); let file = fs::File::create(path).unwrap(); let mut writer = ArrowWriter::try_new(file, schema.clone(), None).unwrap(); diff --git a/datafusion/core/tests/sql/partitioned_csv.rs b/datafusion/core/tests/sql/partitioned_csv.rs index efb63a296ec8..98cb3b189361 100644 --- a/datafusion/core/tests/sql/partitioned_csv.rs +++ b/datafusion/core/tests/sql/partitioned_csv.rs @@ -59,7 +59,7 @@ fn populate_csv_partitions( // generate a partitioned file for partition in 0..partition_count { - let filename = format!("partition-{}.{}", partition, file_extension); + let filename = format!("partition-{partition}.{file_extension}"); let file_path = tmp_dir.path().join(filename); let mut file = std::fs::File::create(file_path)?; diff --git a/datafusion/core/tests/sql/predicates.rs b/datafusion/core/tests/sql/predicates.rs index d56f95e5513d..61d509a2ddc8 100644 --- a/datafusion/core/tests/sql/predicates.rs +++ b/datafusion/core/tests/sql/predicates.rs @@ -580,7 +580,7 @@ async fn multiple_or_predicates() -> Result<()> { and l_quantity >= 20 and l_quantity <= 20 + 10 and p_size between 1 and 15 )"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let plan = dataframe.into_optimized_plan().unwrap(); // Note that we expect `part.p_partkey = lineitem.l_partkey` to have been @@ -601,8 +601,7 @@ async fn multiple_or_predicates() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); Ok(()) } @@ -660,8 +659,7 @@ where assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); // assert data diff --git a/datafusion/core/tests/sql/projection.rs b/datafusion/core/tests/sql/projection.rs index 2ba80bcd0819..8bb33a91d458 100644 --- a/datafusion/core/tests/sql/projection.rs +++ b/datafusion/core/tests/sql/projection.rs @@ -191,7 +191,7 @@ async fn projection_on_table_scan() -> Result<()> { let expected = "Projection: test.c2\ \n TableScan: test projection=[c2]"; - assert_eq!(format!("{:?}", optimized_plan), expected); + assert_eq!(format!("{optimized_plan:?}"), expected); let physical_plan = state.create_physical_plan(&optimized_plan).await?; @@ -306,11 +306,10 @@ async fn projection_on_memory_scan() -> Result<()> { } let expected = format!( - "Projection: {}.b\ - \n TableScan: {} projection=[b]", - UNNAMED_TABLE, UNNAMED_TABLE + "Projection: {UNNAMED_TABLE}.b\ + \n TableScan: {UNNAMED_TABLE} projection=[b]" ); - assert_eq!(format!("{:?}", optimized_plan), expected); + assert_eq!(format!("{optimized_plan:?}"), expected); let physical_plan = state.create_physical_plan(&optimized_plan).await?; diff --git a/datafusion/core/tests/sql/references.rs b/datafusion/core/tests/sql/references.rs index 1a7866917f35..f006cbb45984 100644 --- a/datafusion/core/tests/sql/references.rs +++ b/datafusion/core/tests/sql/references.rs @@ -27,7 +27,7 @@ async fn qualified_table_references() -> Result<()> { "public.aggregate_test_100", "datafusion.public.aggregate_test_100", ] { - let sql = format!("SELECT COUNT(*) FROM {}", table_ref); + let sql = format!("SELECT COUNT(*) FROM {table_ref}"); let actual = execute_to_batches(&ctx, &sql).await; let expected = vec![ "+-----------------+", diff --git a/datafusion/core/tests/sql/udf.rs b/datafusion/core/tests/sql/udf.rs index d07fdd91d306..9ddc5f6141c5 100644 --- a/datafusion/core/tests/sql/udf.rs +++ b/datafusion/core/tests/sql/udf.rs @@ -86,7 +86,7 @@ async fn scalar_udf() -> Result<()> { .build()?; assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Projection: t.a, t.b, my_add(t.a, t.b)\n TableScan: t projection=[a, b]" ); diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 2f997ca28630..ccccba6c52b1 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -324,7 +324,7 @@ async fn window_expr_eliminate() -> Result<()> { GROUP BY d.b ORDER BY d.b;"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(&("explain ".to_owned() + sql)).await.expect(&msg); let plan = dataframe.into_optimized_plan().unwrap(); let expected = vec![ @@ -351,8 +351,7 @@ async fn window_expr_eliminate() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let results = execute_to_batches(&ctx, sql).await; @@ -415,8 +414,7 @@ async fn window_expr_eliminate() -> Result<()> { let actual: Vec<&str> = formatted.trim().lines().collect(); assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let results = execute_to_batches(&ctx, sql).await; @@ -1644,7 +1642,7 @@ async fn test_window_agg_sort() -> Result<()> { SUM(c9) OVER(ORDER BY c9, c8) as sum2 FROM aggregate_test_100"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1664,8 +1662,7 @@ async fn test_window_agg_sort() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual_trim_last + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual_trim_last:#?}\n\n" ); Ok(()) } @@ -1677,7 +1674,7 @@ async fn over_order_by_sort_keys_sorting_prefix_compacting() -> Result<()> { let sql = "SELECT c2, MAX(c9) OVER (ORDER BY c2), SUM(c9) OVER (), MIN(c9) OVER (ORDER BY c2, c9) from aggregate_test_100"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1698,8 +1695,7 @@ async fn over_order_by_sort_keys_sorting_prefix_compacting() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual_trim_last + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual_trim_last:#?}\n\n" ); Ok(()) } @@ -1711,7 +1707,7 @@ async fn over_order_by_sort_keys_sorting_global_order_compacting() -> Result<()> register_aggregate_csv(&ctx).await?; let sql = "SELECT c2, MAX(c9) OVER (ORDER BY c9, c2), SUM(c9) OVER (), MIN(c9) OVER (ORDER BY c2, c9) from aggregate_test_100 ORDER BY c2"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1735,8 +1731,7 @@ async fn over_order_by_sort_keys_sorting_global_order_compacting() -> Result<()> let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual_trim_last + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual_trim_last:#?}\n\n" ); Ok(()) } @@ -1751,7 +1746,7 @@ async fn test_window_partition_by_order_by() -> Result<()> { COUNT(*) OVER(PARTITION BY c1 ORDER BY c2 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) \ FROM aggregate_test_100"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1775,8 +1770,7 @@ async fn test_window_partition_by_order_by() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual_trim_last + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual_trim_last:#?}\n\n" ); Ok(()) } @@ -1792,7 +1786,7 @@ async fn test_window_agg_sort_reversed_plan() -> Result<()> { FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1813,8 +1807,7 @@ async fn test_window_agg_sort_reversed_plan() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -1849,7 +1842,7 @@ async fn test_window_agg_sort_reversed_plan_builtin() -> Result<()> { FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1870,8 +1863,7 @@ async fn test_window_agg_sort_reversed_plan_builtin() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -1902,7 +1894,7 @@ async fn test_window_agg_sort_non_reversed_plan() -> Result<()> { FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1924,8 +1916,7 @@ async fn test_window_agg_sort_non_reversed_plan() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -1957,7 +1948,7 @@ async fn test_window_agg_sort_multi_layer_non_reversed_plan() -> Result<()> { FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -1980,8 +1971,7 @@ async fn test_window_agg_sort_multi_layer_non_reversed_plan() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -2046,7 +2036,7 @@ async fn test_window_agg_complex_plan() -> Result<()> { FROM null_cases LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2075,8 +2065,7 @@ async fn test_window_agg_complex_plan() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); Ok(()) @@ -2096,7 +2085,7 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_plan() -> Result<()> FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2117,8 +2106,7 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_plan() -> Result<()> let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -2152,7 +2140,7 @@ async fn test_window_agg_sort_partitionby_reversed_plan() -> Result<()> { FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2173,8 +2161,7 @@ async fn test_window_agg_sort_partitionby_reversed_plan() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -2207,7 +2194,7 @@ async fn test_window_agg_sort_orderby_reversed_binary_expr() -> Result<()> { FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2228,8 +2215,7 @@ async fn test_window_agg_sort_orderby_reversed_binary_expr() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -2263,7 +2249,7 @@ async fn test_remove_unnecessary_sort_in_sub_query() -> Result<()> { GROUP BY c1 ORDER BY c1 ) AS a "; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2291,8 +2277,7 @@ async fn test_remove_unnecessary_sort_in_sub_query() -> Result<()> { let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; @@ -2321,7 +2306,7 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re FROM aggregate_test_100 LIMIT 5"; - let msg = format!("Creating logical plan for '{}'", sql); + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2342,8 +2327,7 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re let actual_trim_last = &actual[..actual_len - 1]; assert_eq!( expected, actual_trim_last, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = execute_to_batches(&ctx, sql).await; diff --git a/datafusion/core/tests/sqllogictests/src/setup.rs b/datafusion/core/tests/sqllogictests/src/setup.rs index 1e38d48608eb..24c0cf8d308a 100644 --- a/datafusion/core/tests/sqllogictests/src/setup.rs +++ b/datafusion/core/tests/sqllogictests/src/setup.rs @@ -106,7 +106,7 @@ fn register_median_test_tables(ctx: &SessionContext) { vec![values], ) .unwrap(); - let table_name = &format!("median_{}", name); + let table_name = &format!("median_{name}"); ctx.register_batch(table_name, batch).unwrap(); } } @@ -134,9 +134,8 @@ async fn register_aggregate_csv_by_sql(ctx: &SessionContext) { ) STORED AS CSV WITH HEADER ROW - LOCATION '{}/csv/aggregate_test_100.csv' - ", - test_data + LOCATION '{test_data}/csv/aggregate_test_100.csv' + " )) .await .expect("Creating dataframe for CREATE EXTERNAL TABLE"); @@ -193,7 +192,7 @@ async fn register_aggregate_test_100(ctx: &SessionContext) { let schema = test_util::aggr_test_schema(); ctx.register_csv( "aggregate_test_100", - &format!("{}/csv/aggregate_test_100.csv", test_data), + &format!("{test_data}/csv/aggregate_test_100.csv"), CsvReadOptions::new().schema(&schema), ) .await diff --git a/datafusion/core/tests/user_defined_plan.rs b/datafusion/core/tests/user_defined_plan.rs index 89cfd48e64a5..ff284eaaae67 100644 --- a/datafusion/core/tests/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined_plan.rs @@ -222,7 +222,7 @@ async fn topk_plan() -> Result<()> { "| | TableScan: sales projection=[customer_id,revenue] |", ].join("\n"); - let explain_query = format!("EXPLAIN VERBOSE {}", QUERY); + let explain_query = format!("EXPLAIN VERBOSE {QUERY}"); let actual_output = exec_sql(&mut ctx, &explain_query).await?; // normalize newlines (output on windows uses \r\n) @@ -235,12 +235,10 @@ async fn topk_plan() -> Result<()> { "Expected output not present in actual output\ \nExpected:\ \n---------\ - \n{}\ + \n{expected}\ \nActual:\ \n--------\ - \n{}", - expected, - actual_output + \n{actual_output}" ); Ok(()) } @@ -470,8 +468,7 @@ impl ExecutionPlan for TopKExec { ) -> Result { if 0 != partition { return Err(DataFusionError::Internal(format!( - "TopKExec invalid partition {}", - partition + "TopKExec invalid partition {partition}" ))); } diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index ce03a252e9e5..b7fb7d47d297 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -68,7 +68,7 @@ pub enum AggregateFunction { impl fmt::Display for AggregateFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { // uppercase of the debug. - write!(f, "{}", format!("{:?}", self).to_uppercase()) + write!(f, "{}", format!("{self:?}").to_uppercase()) } } @@ -103,8 +103,7 @@ impl FromStr for AggregateFunction { "grouping" => AggregateFunction::Grouping, _ => { return Err(DataFusionError::Plan(format!( - "There is no built-in function named {}", - name + "There is no built-in function named {name}" ))); } }) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index d78d382628ac..8ea96f185cb0 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -277,7 +277,7 @@ impl BuiltinScalarFunction { impl fmt::Display for BuiltinScalarFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { // lowercase of the debug. - write!(f, "{}", format!("{:?}", self).to_lowercase()) + write!(f, "{}", format!("{self:?}").to_lowercase()) } } @@ -369,8 +369,7 @@ impl FromStr for BuiltinScalarFunction { "arrow_typeof" => BuiltinScalarFunction::ArrowTypeof, _ => { return Err(DataFusionError::Plan(format!( - "There is no built-in function named {}", - name + "There is no built-in function named {name}" ))) } }) diff --git a/datafusion/expr/src/conditional_expressions.rs b/datafusion/expr/src/conditional_expressions.rs index 28ac6e8cd0c1..2518e9e22570 100644 --- a/datafusion/expr/src/conditional_expressions.rs +++ b/datafusion/expr/src/conditional_expressions.rs @@ -103,8 +103,7 @@ impl CaseBuilder { let unique_types: HashSet<&DataType> = then_types.iter().collect(); if unique_types.len() != 1 { return Err(DataFusionError::Plan(format!( - "CASE expression 'then' values had multiple data types: {:?}", - unique_types + "CASE expression 'then' values had multiple data types: {unique_types:?}" ))); } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 003b7a37fbf4..fd045b1ac447 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -288,12 +288,12 @@ impl Display for BinaryExpr { Expr::BinaryExpr(child) => { let p = child.precedence(); if p == 0 || p < precedence { - write!(f, "({})", child)?; + write!(f, "({child})")?; } else { - write!(f, "{}", child)?; + write!(f, "{child}")?; } } - _ => write!(f, "{}", expr)?, + _ => write!(f, "{expr}")?, } Ok(()) } @@ -584,7 +584,7 @@ impl Expr { /// Returns a full and complete string representation of this expression. pub fn canonical_name(&self) -> String { - format!("{}", self) + format!("{self}") } /// Return String representation of the variant represented by `self` @@ -781,7 +781,7 @@ impl Expr { pub fn try_into_col(&self) -> Result { match self { Expr::Column(it) => Ok(it.clone()), - _ => plan_err!(format!("Could not coerce '{}' into Column!", self)), + _ => plan_err!(format!("Could not coerce '{self}' into Column!")), } } @@ -826,7 +826,7 @@ impl Not for Expr { /// similar output to `Expr.name()` except that column names will be prefixed with '#'. impl fmt::Display for Expr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{:?}", self) + write!(f, "{self:?}") } } @@ -835,68 +835,68 @@ impl fmt::Display for Expr { impl fmt::Debug for Expr { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { - Expr::Alias(expr, alias) => write!(f, "{:?} AS {}", expr, alias), - Expr::Column(c) => write!(f, "{}", c), + Expr::Alias(expr, alias) => write!(f, "{expr:?} AS {alias}"), + Expr::Column(c) => write!(f, "{c}"), Expr::ScalarVariable(_, var_names) => write!(f, "{}", var_names.join(".")), - Expr::Literal(v) => write!(f, "{:?}", v), + Expr::Literal(v) => write!(f, "{v:?}"), Expr::Case(case) => { write!(f, "CASE ")?; if let Some(e) = &case.expr { - write!(f, "{:?} ", e)?; + write!(f, "{e:?} ")?; } for (w, t) in &case.when_then_expr { - write!(f, "WHEN {:?} THEN {:?} ", w, t)?; + write!(f, "WHEN {w:?} THEN {t:?} ")?; } if let Some(e) = &case.else_expr { - write!(f, "ELSE {:?} ", e)?; + write!(f, "ELSE {e:?} ")?; } write!(f, "END") } Expr::Cast(Cast { expr, data_type }) => { - write!(f, "CAST({:?} AS {:?})", expr, data_type) + write!(f, "CAST({expr:?} AS {data_type:?})") } Expr::TryCast(TryCast { expr, data_type }) => { - write!(f, "TRY_CAST({:?} AS {:?})", expr, data_type) + write!(f, "TRY_CAST({expr:?} AS {data_type:?})") } - Expr::Not(expr) => write!(f, "NOT {:?}", expr), - Expr::Negative(expr) => write!(f, "(- {:?})", expr), - Expr::IsNull(expr) => write!(f, "{:?} IS NULL", expr), - Expr::IsNotNull(expr) => write!(f, "{:?} IS NOT NULL", expr), - Expr::IsTrue(expr) => write!(f, "{:?} IS TRUE", expr), - Expr::IsFalse(expr) => write!(f, "{:?} IS FALSE", expr), - Expr::IsUnknown(expr) => write!(f, "{:?} IS UNKNOWN", expr), - Expr::IsNotTrue(expr) => write!(f, "{:?} IS NOT TRUE", expr), - Expr::IsNotFalse(expr) => write!(f, "{:?} IS NOT FALSE", expr), - Expr::IsNotUnknown(expr) => write!(f, "{:?} IS NOT UNKNOWN", expr), + Expr::Not(expr) => write!(f, "NOT {expr:?}"), + Expr::Negative(expr) => write!(f, "(- {expr:?})"), + Expr::IsNull(expr) => write!(f, "{expr:?} IS NULL"), + Expr::IsNotNull(expr) => write!(f, "{expr:?} IS NOT NULL"), + Expr::IsTrue(expr) => write!(f, "{expr:?} IS TRUE"), + Expr::IsFalse(expr) => write!(f, "{expr:?} IS FALSE"), + Expr::IsUnknown(expr) => write!(f, "{expr:?} IS UNKNOWN"), + Expr::IsNotTrue(expr) => write!(f, "{expr:?} IS NOT TRUE"), + Expr::IsNotFalse(expr) => write!(f, "{expr:?} IS NOT FALSE"), + Expr::IsNotUnknown(expr) => write!(f, "{expr:?} IS NOT UNKNOWN"), Expr::Exists { subquery, negated: true, - } => write!(f, "NOT EXISTS ({:?})", subquery), + } => write!(f, "NOT EXISTS ({subquery:?})"), Expr::Exists { subquery, negated: false, - } => write!(f, "EXISTS ({:?})", subquery), + } => write!(f, "EXISTS ({subquery:?})"), Expr::InSubquery { expr, subquery, negated: true, - } => write!(f, "{:?} NOT IN ({:?})", expr, subquery), + } => write!(f, "{expr:?} NOT IN ({subquery:?})"), Expr::InSubquery { expr, subquery, negated: false, - } => write!(f, "{:?} IN ({:?})", expr, subquery), - Expr::ScalarSubquery(subquery) => write!(f, "({:?})", subquery), - Expr::BinaryExpr(expr) => write!(f, "{}", expr), + } => write!(f, "{expr:?} IN ({subquery:?})"), + Expr::ScalarSubquery(subquery) => write!(f, "({subquery:?})"), + Expr::BinaryExpr(expr) => write!(f, "{expr}"), Expr::Sort(Sort { expr, asc, nulls_first, }) => { if *asc { - write!(f, "{:?} ASC", expr)?; + write!(f, "{expr:?} ASC")?; } else { - write!(f, "{:?} DESC", expr)?; + write!(f, "{expr:?} DESC")?; } if *nulls_first { write!(f, " NULLS FIRST") @@ -919,10 +919,10 @@ impl fmt::Debug for Expr { }) => { fmt_function(f, &fun.to_string(), false, args, false)?; if !partition_by.is_empty() { - write!(f, " PARTITION BY {:?}", partition_by)?; + write!(f, " PARTITION BY {partition_by:?}")?; } if !order_by.is_empty() { - write!(f, " ORDER BY {:?}", order_by)?; + write!(f, " ORDER BY {order_by:?}")?; } write!( f, @@ -940,7 +940,7 @@ impl fmt::Debug for Expr { }) => { fmt_function(f, &fun.to_string(), *distinct, args, true)?; if let Some(fe) = filter { - write!(f, " FILTER (WHERE {})", fe)?; + write!(f, " FILTER (WHERE {fe})")?; } Ok(()) } @@ -952,7 +952,7 @@ impl fmt::Debug for Expr { } => { fmt_function(f, &fun.name, false, args, false)?; if let Some(fe) = filter { - write!(f, " FILTER (WHERE {})", fe)?; + write!(f, " FILTER (WHERE {fe})")?; } Ok(()) } @@ -963,9 +963,9 @@ impl fmt::Debug for Expr { high, }) => { if *negated { - write!(f, "{:?} NOT BETWEEN {:?} AND {:?}", expr, low, high) + write!(f, "{expr:?} NOT BETWEEN {low:?} AND {high:?}") } else { - write!(f, "{:?} BETWEEN {:?} AND {:?}", expr, low, high) + write!(f, "{expr:?} BETWEEN {low:?} AND {high:?}") } } Expr::Like(Like { @@ -974,14 +974,14 @@ impl fmt::Debug for Expr { pattern, escape_char, }) => { - write!(f, "{:?}", expr)?; + write!(f, "{expr:?}")?; if *negated { write!(f, " NOT")?; } if let Some(char) = escape_char { - write!(f, " LIKE {:?} ESCAPE '{}'", pattern, char) + write!(f, " LIKE {pattern:?} ESCAPE '{char}'") } else { - write!(f, " LIKE {:?}", pattern) + write!(f, " LIKE {pattern:?}") } } Expr::ILike(Like { @@ -990,14 +990,14 @@ impl fmt::Debug for Expr { pattern, escape_char, }) => { - write!(f, "{:?}", expr)?; + write!(f, "{expr:?}")?; if *negated { write!(f, " NOT")?; } if let Some(char) = escape_char { - write!(f, " ILIKE {:?} ESCAPE '{}'", pattern, char) + write!(f, " ILIKE {pattern:?} ESCAPE '{char}'") } else { - write!(f, " ILIKE {:?}", pattern) + write!(f, " ILIKE {pattern:?}") } } Expr::SimilarTo(Like { @@ -1006,14 +1006,14 @@ impl fmt::Debug for Expr { pattern, escape_char, }) => { - write!(f, "{:?}", expr)?; + write!(f, "{expr:?}")?; if *negated { write!(f, " NOT")?; } if let Some(char) = escape_char { - write!(f, " SIMILAR TO {:?} ESCAPE '{}'", pattern, char) + write!(f, " SIMILAR TO {pattern:?} ESCAPE '{char}'") } else { - write!(f, " SIMILAR TO {:?}", pattern) + write!(f, " SIMILAR TO {pattern:?}") } } Expr::InList { @@ -1022,15 +1022,15 @@ impl fmt::Debug for Expr { negated, } => { if *negated { - write!(f, "{:?} NOT IN ({:?})", expr, list) + write!(f, "{expr:?} NOT IN ({list:?})") } else { - write!(f, "{:?} IN ({:?})", expr, list) + write!(f, "{expr:?} IN ({list:?})") } } Expr::Wildcard => write!(f, "*"), - Expr::QualifiedWildcard { qualifier } => write!(f, "{}.*", qualifier), + Expr::QualifiedWildcard { qualifier } => write!(f, "{qualifier}.*"), Expr::GetIndexedField(GetIndexedField { key, expr }) => { - write!(f, "({:?})[{}]", expr, key) + write!(f, "({expr:?})[{key}]") } Expr::GroupingSet(grouping_sets) => match grouping_sets { GroupingSet::Rollup(exprs) => { @@ -1040,7 +1040,7 @@ impl fmt::Debug for Expr { "ROLLUP ({})", exprs .iter() - .map(|e| format!("{}", e)) + .map(|e| format!("{e}")) .collect::>() .join(", ") ) @@ -1052,7 +1052,7 @@ impl fmt::Debug for Expr { "CUBE ({})", exprs .iter() - .map(|e| format!("{}", e)) + .map(|e| format!("{e}")) .collect::>() .join(", ") ) @@ -1068,7 +1068,7 @@ impl fmt::Debug for Expr { "({})", exprs .iter() - .map(|e| format!("{}", e)) + .map(|e| format!("{e}")) .collect::>() .join(", ") )) @@ -1077,7 +1077,7 @@ impl fmt::Debug for Expr { ) } }, - Expr::Placeholder { id, .. } => write!(f, "{}", id), + Expr::Placeholder { id, .. } => write!(f, "{id}"), } } } @@ -1090,8 +1090,8 @@ fn fmt_function( display: bool, ) -> fmt::Result { let args: Vec = match display { - true => args.iter().map(|arg| format!("{}", arg)).collect(), - false => args.iter().map(|arg| format!("{:?}", arg)).collect(), + true => args.iter().map(|arg| format!("{arg}")).collect(), + false => args.iter().map(|arg| format!("{arg:?}")).collect(), }; // let args: Vec = args.iter().map(|arg| format!("{:?}", arg)).collect(); @@ -1118,7 +1118,7 @@ fn create_name(e: &Expr) -> Result { Expr::Alias(_, name) => Ok(name.clone()), Expr::Column(c) => Ok(c.flat_name()), Expr::ScalarVariable(_, variable_names) => Ok(variable_names.join(".")), - Expr::Literal(value) => Ok(format!("{:?}", value)), + Expr::Literal(value) => Ok(format!("{value:?}")), Expr::BinaryExpr(binary_expr) => { let left = create_name(binary_expr.left.as_ref())?; let right = create_name(binary_expr.right.as_ref())?; @@ -1136,7 +1136,7 @@ fn create_name(e: &Expr) -> Result { if *negated { "NOT LIKE" } else { "LIKE" }, pattern, if let Some(char) = escape_char { - format!("CHAR '{}'", char) + format!("CHAR '{char}'") } else { "".to_string() } @@ -1155,7 +1155,7 @@ fn create_name(e: &Expr) -> Result { if *negated { "NOT ILIKE" } else { "ILIKE" }, pattern, if let Some(char) = escape_char { - format!("CHAR '{}'", char) + format!("CHAR '{char}'") } else { "".to_string() } @@ -1178,7 +1178,7 @@ fn create_name(e: &Expr) -> Result { }, pattern, if let Some(char) = escape_char { - format!("CHAR '{}'", char) + format!("CHAR '{char}'") } else { "".to_string() } @@ -1189,16 +1189,16 @@ fn create_name(e: &Expr) -> Result { let mut name = "CASE ".to_string(); if let Some(e) = &case.expr { let e = create_name(e)?; - let _ = write!(name, "{} ", e); + let _ = write!(name, "{e} "); } for (w, t) in &case.when_then_expr { let when = create_name(w)?; let then = create_name(t)?; - let _ = write!(name, "WHEN {} THEN {} ", when, then); + let _ = write!(name, "WHEN {when} THEN {then} "); } if let Some(e) = &case.else_expr { let e = create_name(e)?; - let _ = write!(name, "ELSE {} ", e); + let _ = write!(name, "ELSE {e} "); } name += "END"; Ok(name) @@ -1213,43 +1213,43 @@ fn create_name(e: &Expr) -> Result { } Expr::Not(expr) => { let expr = create_name(expr)?; - Ok(format!("NOT {}", expr)) + Ok(format!("NOT {expr}")) } Expr::Negative(expr) => { let expr = create_name(expr)?; - Ok(format!("(- {})", expr)) + Ok(format!("(- {expr})")) } Expr::IsNull(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS NULL", expr)) + Ok(format!("{expr} IS NULL")) } Expr::IsNotNull(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS NOT NULL", expr)) + Ok(format!("{expr} IS NOT NULL")) } Expr::IsTrue(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS TRUE", expr)) + Ok(format!("{expr} IS TRUE")) } Expr::IsFalse(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS FALSE", expr)) + Ok(format!("{expr} IS FALSE")) } Expr::IsUnknown(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS UNKNOWN", expr)) + Ok(format!("{expr} IS UNKNOWN")) } Expr::IsNotTrue(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS NOT TRUE", expr)) + Ok(format!("{expr} IS NOT TRUE")) } Expr::IsNotFalse(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS NOT FALSE", expr)) + Ok(format!("{expr} IS NOT FALSE")) } Expr::IsNotUnknown(expr) => { let expr = create_name(expr)?; - Ok(format!("{} IS NOT UNKNOWN", expr)) + Ok(format!("{expr} IS NOT UNKNOWN")) } Expr::Exists { negated: true, .. } => Ok("NOT EXISTS".to_string()), Expr::Exists { negated: false, .. } => Ok("EXISTS".to_string()), @@ -1260,7 +1260,7 @@ fn create_name(e: &Expr) -> Result { } Expr::GetIndexedField(GetIndexedField { key, expr }) => { let expr = create_name(expr)?; - Ok(format!("{}[{}]", expr, key)) + Ok(format!("{expr}[{key}]")) } Expr::ScalarFunction { fun, args, .. } => { create_function_name(&fun.to_string(), false, args) @@ -1276,12 +1276,12 @@ fn create_name(e: &Expr) -> Result { let mut parts: Vec = vec![create_function_name(&fun.to_string(), false, args)?]; if !partition_by.is_empty() { - parts.push(format!("PARTITION BY {:?}", partition_by)); + parts.push(format!("PARTITION BY {partition_by:?}")); } if !order_by.is_empty() { - parts.push(format!("ORDER BY {:?}", order_by)); + parts.push(format!("ORDER BY {order_by:?}")); } - parts.push(format!("{}", window_frame)); + parts.push(format!("{window_frame}")); Ok(parts.join(" ")) } Expr::AggregateFunction(AggregateFunction { @@ -1292,7 +1292,7 @@ fn create_name(e: &Expr) -> Result { }) => { let name = create_function_name(&fun.to_string(), *distinct, args)?; if let Some(fe) = filter { - Ok(format!("{} FILTER (WHERE {})", name, fe)) + Ok(format!("{name} FILTER (WHERE {fe})")) } else { Ok(name) } @@ -1303,7 +1303,7 @@ fn create_name(e: &Expr) -> Result { names.push(create_name(e)?); } let filter = if let Some(fe) = filter { - format!(" FILTER (WHERE {})", fe) + format!(" FILTER (WHERE {fe})") } else { "".to_string() }; @@ -1332,9 +1332,9 @@ fn create_name(e: &Expr) -> Result { let expr = create_name(expr)?; let list = list.iter().map(create_name); if *negated { - Ok(format!("{} NOT IN ({:?})", expr, list)) + Ok(format!("{expr} NOT IN ({list:?})")) } else { - Ok(format!("{} IN ({:?})", expr, list)) + Ok(format!("{expr} IN ({list:?})")) } } Expr::Between(Between { @@ -1347,9 +1347,9 @@ fn create_name(e: &Expr) -> Result { let low = create_name(low)?; let high = create_name(high)?; if *negated { - Ok(format!("{} NOT BETWEEN {} AND {}", expr, low, high)) + Ok(format!("{expr} NOT BETWEEN {low} AND {high}")) } else { - Ok(format!("{} BETWEEN {} AND {}", expr, low, high)) + Ok(format!("{expr} BETWEEN {low} AND {high}")) } } Expr::Sort { .. } => Err(DataFusionError::Internal( @@ -1389,8 +1389,8 @@ mod test { .otherwise(lit(ScalarValue::Null))?; let expected = "CASE a WHEN Int32(1) THEN Boolean(true) WHEN Int32(0) THEN Boolean(false) ELSE NULL END"; assert_eq!(expected, expr.canonical_name()); - assert_eq!(expected, format!("{}", expr)); - assert_eq!(expected, format!("{:?}", expr)); + assert_eq!(expected, format!("{expr}")); + assert_eq!(expected, format!("{expr:?}")); assert_eq!(expected, expr.display_name()?); Ok(()) } @@ -1403,8 +1403,8 @@ mod test { }); let expected_canonical = "CAST(Float32(1.23) AS Utf8)"; assert_eq!(expected_canonical, expr.canonical_name()); - assert_eq!(expected_canonical, format!("{}", expr)); - assert_eq!(expected_canonical, format!("{:?}", expr)); + assert_eq!(expected_canonical, format!("{expr}")); + assert_eq!(expected_canonical, format!("{expr:?}")); // note that CAST intentionally has a name that is different from its `Display` // representation. CAST does not change the name of expressions. assert_eq!("Float32(1.23)", expr.display_name()?); diff --git a/datafusion/expr/src/expr_rewriter.rs b/datafusion/expr/src/expr_rewriter.rs index 6ad7b5eecbc4..cb907f6487e9 100644 --- a/datafusion/expr/src/expr_rewriter.rs +++ b/datafusion/expr/src/expr_rewriter.rs @@ -564,12 +564,12 @@ mod test { impl ExprRewriter for RecordingRewriter { fn mutate(&mut self, expr: Expr) -> Result { - self.v.push(format!("Mutated {:?}", expr)); + self.v.push(format!("Mutated {expr:?}")); Ok(expr) } fn pre_visit(&mut self, expr: &Expr) -> Result { - self.v.push(format!("Previsited {:?}", expr)); + self.v.push(format!("Previsited {expr:?}")); Ok(RewriteRecursion::Continue) } } diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index c1a625cf47c2..782f23c7c06b 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -268,8 +268,7 @@ impl ExprSchemable for Expr { Ok(Expr::Cast(Cast::new(Box::new(self), cast_to_type.clone()))) } else { Err(DataFusionError::Plan(format!( - "Cannot automatically convert {:?} to {:?}", - this_type, cast_to_type + "Cannot automatically convert {this_type:?} to {cast_to_type:?}" ))) } } diff --git a/datafusion/expr/src/field_util.rs b/datafusion/expr/src/field_util.rs index 94efeda80648..1bc88a54e8c4 100644 --- a/datafusion/expr/src/field_util.rs +++ b/datafusion/expr/src/field_util.rs @@ -39,8 +39,7 @@ pub fn get_indexed_field(data_type: &DataType, key: &ScalarValue) -> Result Err(DataFusionError::Plan(format!( - "Field {} not found in struct", - s + "Field {s} not found in struct" ))), Some(f) => Ok(f.clone()), } diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index 459152192bbb..50cc6bcd6c82 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -85,8 +85,7 @@ fn utf8_or_binary_to_binary_type(arg_type: &DataType, name: &str) -> Result { // this error is internal as `data_types` should have captured this. return Err(DataFusionError::Internal(format!( - "The {:?} function can only accept strings or binary arrays.", - name + "The {name:?} function can only accept strings or binary arrays." ))); } }) @@ -102,8 +101,7 @@ pub fn return_type( if input_expr_types.is_empty() && !fun.supports_zero_argument() { return Err(DataFusionError::Internal(format!( - "Builtin scalar function {} does not support empty arguments", - fun + "Builtin scalar function {fun} does not support empty arguments" ))); } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 50325cfdee3b..d49af378b16f 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -160,7 +160,7 @@ impl LogicalPlanBuilder { let data_type = expr.get_type(&empty_schema)?; if let Some(prev_data_type) = &field_types[j] { if prev_data_type != &data_type { - let err = format!("Inconsistent data type across values list at row {} column {}", i, j); + let err = format!("Inconsistent data type across values list at row {i} column {j}"); return Err(DataFusionError::Plan(err)); } } @@ -778,8 +778,7 @@ impl LogicalPlanBuilder { if left_len != right_len { return Err(DataFusionError::Plan(format!( - "INTERSECT/EXCEPT query must have the same number of columns. Left is {} and right is {}.", - left_len, right_len + "INTERSECT/EXCEPT query must have the same number of columns. Left is {left_len} and right is {right_len}." ))); } @@ -861,8 +860,7 @@ impl LogicalPlanBuilder { right.schema().clone(), )?.ok_or_else(|| DataFusionError::Plan(format!( - "can't create join plan, join key should belong to one input, error key: ({},{})", - normalized_left_key, normalized_right_key + "can't create join plan, join key should belong to one input, error key: ({normalized_left_key},{normalized_right_key})" ))) }) .collect::>>()?; @@ -927,10 +925,9 @@ pub(crate) fn validate_unique_names<'a>( }, Some((existing_position, existing_expr)) => { Err(DataFusionError::Plan( - format!("{} require unique expression names \ - but the expression \"{:?}\" at position {} and \"{:?}\" \ - at position {} have the same name. Consider aliasing (\"AS\") one of them.", - node_name, existing_expr, existing_position, expr, position, + format!("{node_name} require unique expression names \ + but the expression \"{existing_expr:?}\" at position {existing_position} and \"{expr:?}\" \ + at position {position} have the same name. Consider aliasing (\"AS\") one of them.", ) )) } @@ -965,8 +962,7 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result String { let label = label.replace('"', "_"); - format!("\"{}\"", label) + format!("\"{label}\"") } } @@ -214,8 +214,7 @@ impl<'a, 'b> PlanVisitor for GraphvizVisitor<'a, 'b> { if let Some(parent_id) = self.parent_ids.last() { writeln!( self.f, - " {} -> {} [arrowhead=none, arrowtail=normal, dir=back]", - parent_id, id + " {parent_id} -> {id} [arrowhead=none, arrowtail=normal, dir=back]" )?; } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 0a3a68dae1e6..576ba161a534 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -618,15 +618,13 @@ impl LogicalPlan { // convert id (in format $1, $2, ..) to idx (0, 1, ..) let idx = id[1..].parse::().map_err(|e| { DataFusionError::Internal(format!( - "Failed to parse placeholder id: {}", - e + "Failed to parse placeholder id: {e}" )) })? - 1; // value at the idx-th position in param_values should be the value for the placeholder let value = self.param_values.get(idx).ok_or_else(|| { DataFusionError::Internal(format!( - "No value found for placeholder with id {}", - id + "No value found for placeholder with id {id}" )) })?; // check if the data type of the value matches the data type of the placeholder @@ -836,7 +834,7 @@ impl LogicalPlan { .map(|expr| expr.to_string()) .collect::>() .join(", "); - format!("({})", item) + format!("({item})") }) .collect(); @@ -864,7 +862,7 @@ impl LogicalPlan { _ => "".to_string(), }; - write!(f, "TableScan: {}{}", table_name, projected_fields)?; + write!(f, "TableScan: {table_name}{projected_fields}")?; if !filters.is_empty() { let mut full_filter = vec![]; @@ -888,22 +886,21 @@ impl LogicalPlan { }); if !full_filter.is_empty() { - write!(f, ", full_filters={:?}", full_filter)?; + write!(f, ", full_filters={full_filter:?}")?; }; if !partial_filter.is_empty() { - write!(f, ", partial_filters={:?}", partial_filter)?; + write!(f, ", partial_filters={partial_filter:?}")?; } if !unsupported_filters.is_empty() { write!( f, - ", unsupported_filters={:?}", - unsupported_filters + ", unsupported_filters={unsupported_filters:?}" )?; } } if let Some(n) = fetch { - write!(f, ", fetch={}", n)?; + write!(f, ", fetch={n}")?; } Ok(()) @@ -914,18 +911,18 @@ impl LogicalPlan { if i > 0 { write!(f, ", ")?; } - write!(f, "{:?}", expr_item)?; + write!(f, "{expr_item:?}")?; } Ok(()) } LogicalPlan::Filter(Filter { predicate: ref expr, .. - }) => write!(f, "Filter: {:?}", expr), + }) => write!(f, "Filter: {expr:?}"), LogicalPlan::Window(Window { ref window_expr, .. }) => { - write!(f, "WindowAggr: windowExpr=[{:?}]", window_expr) + write!(f, "WindowAggr: windowExpr=[{window_expr:?}]") } LogicalPlan::Aggregate(Aggregate { ref group_expr, @@ -933,8 +930,7 @@ impl LogicalPlan { .. }) => write!( f, - "Aggregate: groupBy=[{:?}], aggr=[{:?}]", - group_expr, aggr_expr + "Aggregate: groupBy=[{group_expr:?}], aggr=[{aggr_expr:?}]" ), LogicalPlan::Sort(Sort { expr, fetch, .. }) => { write!(f, "Sort: ")?; @@ -942,10 +938,10 @@ impl LogicalPlan { if i > 0 { write!(f, ", ")?; } - write!(f, "{:?}", expr_item)?; + write!(f, "{expr_item:?}")?; } if let Some(a) = fetch { - write!(f, ", fetch={}", a)?; + write!(f, ", fetch={a}")?; } Ok(()) @@ -958,10 +954,10 @@ impl LogicalPlan { .. }) => { let join_expr: Vec = - keys.iter().map(|(l, r)| format!("{} = {}", l, r)).collect(); + keys.iter().map(|(l, r)| format!("{l} = {r}")).collect(); let filter_expr = filter .as_ref() - .map(|expr| format!(" Filter: {}", expr)) + .map(|expr| format!(" Filter: {expr}")) .unwrap_or_else(|| "".to_string()); match join_constraint { JoinConstraint::On => { @@ -991,14 +987,12 @@ impl LogicalPlan { partitioning_scheme, .. }) => match partitioning_scheme { - Partitioning::RoundRobinBatch(n) => write!( - f, - "Repartition: RoundRobinBatch partition_count={}", - n - ), + Partitioning::RoundRobinBatch(n) => { + write!(f, "Repartition: RoundRobinBatch partition_count={n}") + } Partitioning::Hash(expr, n) => { let hash_expr: Vec = - expr.iter().map(|e| format!("{:?}", e)).collect(); + expr.iter().map(|e| format!("{e:?}")).collect(); write!( f, "Repartition: Hash({}) partition_count={}", @@ -1008,7 +1002,7 @@ impl LogicalPlan { } Partitioning::DistributeBy(expr) => { let dist_by_expr: Vec = - expr.iter().map(|e| format!("{:?}", e)).collect(); + expr.iter().map(|e| format!("{e:?}")).collect(); write!( f, "Repartition: DistributeBy({})", @@ -1032,47 +1026,47 @@ impl LogicalPlan { write!(f, "Subquery:") } LogicalPlan::SubqueryAlias(SubqueryAlias { ref alias, .. }) => { - write!(f, "SubqueryAlias: {}", alias) + write!(f, "SubqueryAlias: {alias}") } LogicalPlan::CreateExternalTable(CreateExternalTable { ref name, .. }) => { - write!(f, "CreateExternalTable: {:?}", name) + write!(f, "CreateExternalTable: {name:?}") } LogicalPlan::CreateMemoryTable(CreateMemoryTable { name, .. }) => { - write!(f, "CreateMemoryTable: {:?}", name) + write!(f, "CreateMemoryTable: {name:?}") } LogicalPlan::CreateView(CreateView { name, .. }) => { - write!(f, "CreateView: {:?}", name) + write!(f, "CreateView: {name:?}") } LogicalPlan::CreateCatalogSchema(CreateCatalogSchema { schema_name, .. }) => { - write!(f, "CreateCatalogSchema: {:?}", schema_name) + write!(f, "CreateCatalogSchema: {schema_name:?}") } LogicalPlan::CreateCatalog(CreateCatalog { catalog_name, .. }) => { - write!(f, "CreateCatalog: {:?}", catalog_name) + write!(f, "CreateCatalog: {catalog_name:?}") } LogicalPlan::DropTable(DropTable { name, if_exists, .. }) => { - write!(f, "DropTable: {:?} if not exist:={}", name, if_exists) + write!(f, "DropTable: {name:?} if not exist:={if_exists}") } LogicalPlan::DropView(DropView { name, if_exists, .. }) => { - write!(f, "DropView: {:?} if not exist:={}", name, if_exists) + write!(f, "DropView: {name:?} if not exist:={if_exists}") } LogicalPlan::SetVariable(SetVariable { variable, value, .. }) => { - write!(f, "SetVariable: set {:?} to {:?}", variable, value) + write!(f, "SetVariable: set {variable:?} to {value:?}") } LogicalPlan::Distinct(Distinct { .. }) => { write!(f, "Distinct:") @@ -1084,7 +1078,7 @@ impl LogicalPlan { LogicalPlan::Prepare(Prepare { name, data_types, .. }) => { - write!(f, "Prepare: {:?} {:?} ", name, data_types) + write!(f, "Prepare: {name:?} {data_types:?} ") } } } @@ -1144,7 +1138,7 @@ impl Display for JoinType { JoinType::LeftAnti => "LeftAnti", JoinType::RightAnti => "RightAnti", }; - write!(f, "{}", join_type) + write!(f, "{join_type}") } } @@ -1361,8 +1355,7 @@ impl Filter { if let Ok(predicate_type) = predicate.get_type(input.schema()) { if predicate_type != DataType::Boolean { return Err(DataFusionError::Plan(format!( - "Cannot create filter with non-boolean predicate '{}' returning {}", - predicate, predicate_type + "Cannot create filter with non-boolean predicate '{predicate}' returning {predicate_type}" ))); } } @@ -1371,9 +1364,8 @@ impl Filter { if let Expr::Alias(expr, alias) = predicate { return Err(DataFusionError::Plan(format!( "Attempted to create Filter predicate with \ - expression `{}` aliased as '{}'. Filter predicates should not be \ - aliased.", - expr, alias + expression `{expr}` aliased as '{alias}'. Filter predicates should not be \ + aliased." ))); } @@ -1793,12 +1785,12 @@ impl Display for PlanType { match self { PlanType::InitialLogicalPlan => write!(f, "initial_logical_plan"), PlanType::OptimizedLogicalPlan { optimizer_name } => { - write!(f, "logical_plan after {}", optimizer_name) + write!(f, "logical_plan after {optimizer_name}") } PlanType::FinalLogicalPlan => write!(f, "logical_plan"), PlanType::InitialPhysicalPlan => write!(f, "initial_physical_plan"), PlanType::OptimizedPhysicalPlan { optimizer_name } => { - write!(f, "physical_plan after {}", optimizer_name) + write!(f, "physical_plan after {optimizer_name}") } PlanType::FinalPhysicalPlan => write!(f, "physical_plan"), } diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operator.rs index b32b58167eac..4c623bb2c29d 100644 --- a/datafusion/expr/src/operator.rs +++ b/datafusion/expr/src/operator.rs @@ -193,7 +193,7 @@ impl fmt::Display for Operator { Operator::BitwiseShiftLeft => "<<", Operator::StringConcat => "||", }; - write!(f, "{}", display) + write!(f, "{display}") } } diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 14f691d979c7..fca851ce63c9 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -265,8 +265,7 @@ fn check_arg_count( } _ => { return Err(DataFusionError::Internal(format!( - "Aggregate functions do not support this {:?}", - signature + "Aggregate functions do not support this {signature:?}" ))); } } @@ -308,8 +307,7 @@ pub fn sum_return_type(arg_type: &DataType) -> Result { Ok(DataType::Decimal128(new_precision, *scale)) } other => Err(DataFusionError::Plan(format!( - "SUM does not support type \"{:?}\"", - other + "SUM does not support type \"{other:?}\"" ))), } } @@ -328,8 +326,7 @@ pub fn variance_return_type(arg_type: &DataType) -> Result { | DataType::Float32 | DataType::Float64 => Ok(DataType::Float64), other => Err(DataFusionError::Plan(format!( - "VAR does not support {:?}", - other + "VAR does not support {other:?}" ))), } } @@ -348,8 +345,7 @@ pub fn covariance_return_type(arg_type: &DataType) -> Result { | DataType::Float32 | DataType::Float64 => Ok(DataType::Float64), other => Err(DataFusionError::Plan(format!( - "COVAR does not support {:?}", - other + "COVAR does not support {other:?}" ))), } } @@ -368,8 +364,7 @@ pub fn correlation_return_type(arg_type: &DataType) -> Result { | DataType::Float32 | DataType::Float64 => Ok(DataType::Float64), other => Err(DataFusionError::Plan(format!( - "CORR does not support {:?}", - other + "CORR does not support {other:?}" ))), } } @@ -388,8 +383,7 @@ pub fn stddev_return_type(arg_type: &DataType) -> Result { | DataType::Float32 | DataType::Float64 => Ok(DataType::Float64), other => Err(DataFusionError::Plan(format!( - "STDDEV does not support {:?}", - other + "STDDEV does not support {other:?}" ))), } } @@ -415,8 +409,7 @@ pub fn avg_return_type(arg_type: &DataType) -> Result { | DataType::Float32 | DataType::Float64 => Ok(DataType::Float64), other => Err(DataFusionError::Plan(format!( - "AVG does not support {:?}", - other + "AVG does not support {other:?}" ))), } } diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 270ed5c8d306..2de1c7a6612a 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -131,9 +131,8 @@ pub fn coerce_types( DataType::Date32 | DataType::Date64 | DataType::Timestamp(_, _) => { return Err(DataFusionError::Plan( format!( - "'{:?} {} {:?}' is an unsupported operation. \ - addition/subtraction on dates/timestamps only supported with interval types", - lhs_type, op, rhs_type + "'{lhs_type:?} {op} {rhs_type:?}' is an unsupported operation. \ + addition/subtraction on dates/timestamps only supported with interval types" ), )); } @@ -162,8 +161,7 @@ pub fn coerce_types( match result { None => Err(DataFusionError::Plan( format!( - "'{:?} {} {:?}' can't be evaluated because there isn't a common type to coerce the types to", - lhs_type, op, rhs_type + "'{lhs_type:?} {op} {rhs_type:?}' can't be evaluated because there isn't a common type to coerce the types to" ), )), Some(t) => Ok(t) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 758cb75148e6..8cf79c612e32 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -195,8 +195,7 @@ pub fn expand_qualified_wildcard( .collect(); if qualified_fields.is_empty() { return Err(DataFusionError::Plan(format!( - "Invalid qualifier {}", - qualifier + "Invalid qualifier {qualifier}" ))); } let qualifier_schema = @@ -339,8 +338,7 @@ pub fn group_window_expr_by_sort_keys( Ok(()) } other => Err(DataFusionError::Internal(format!( - "Impossibly got non-window expr {:?}", - other, + "Impossibly got non-window expr {other:?}", ))), })?; Ok(result) @@ -592,8 +590,7 @@ pub fn from_plan( Ok((*left, *right)) } else { Err(DataFusionError::Internal(format!( - "The front part expressions should be an binary equiality expression, actual:{}", - equi_expr + "The front part expressions should be an binary equiality expression, actual:{equi_expr}" ))) } }).collect::>>()?; @@ -707,7 +704,7 @@ pub fn from_plan( input: Arc::new(inputs[0].clone()), })), LogicalPlan::TableScan(ts) => { - assert!(inputs.is_empty(), "{:?} should have no inputs", plan); + assert!(inputs.is_empty(), "{plan:?} should have no inputs"); Ok(LogicalPlan::TableScan(TableScan { filters: expr.to_vec(), ..ts.clone() @@ -721,8 +718,8 @@ pub fn from_plan( | LogicalPlan::CreateCatalogSchema(_) | LogicalPlan::CreateCatalog(_) => { // All of these plan types have no inputs / exprs so should not be called - assert!(expr.is_empty(), "{:?} should have no exprs", plan); - assert!(inputs.is_empty(), "{:?} should have no inputs", plan); + assert!(expr.is_empty(), "{plan:?} should have no exprs"); + assert!(inputs.is_empty(), "{plan:?} should have no inputs"); Ok(plan.clone()) } } diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 94020a4f8f0f..bf74d02b7005 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -216,18 +216,18 @@ pub fn convert_frame_bound_to_scalar_value(v: ast::Expr) -> Result let result = match *value { ast::Expr::Value(ast::Value::SingleQuotedString(item)) => item, e => { - let msg = format!("INTERVAL expression cannot be {:?}", e); + let msg = format!("INTERVAL expression cannot be {e:?}"); return Err(DataFusionError::SQL(ParserError(msg))); } }; if let Some(leading_field) = leading_field { - format!("{} {}", result, leading_field) + format!("{result} {leading_field}") } else { result } } e => { - let msg = format!("Window frame bound cannot be {:?}", e); + let msg = format!("Window frame bound cannot be {e:?}"); return Err(DataFusionError::Internal(msg)); } }))) @@ -240,7 +240,7 @@ impl fmt::Display for WindowFrameBound { if n.is_null() { f.write_str("UNBOUNDED PRECEDING") } else { - write!(f, "{} PRECEDING", n) + write!(f, "{n} PRECEDING") } } WindowFrameBound::CurrentRow => f.write_str("CURRENT ROW"), @@ -248,7 +248,7 @@ impl fmt::Display for WindowFrameBound { if n.is_null() { f.write_str("UNBOUNDED FOLLOWING") } else { - write!(f, "{} FOLLOWING", n) + write!(f, "{n} FOLLOWING") } } } diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 86c63e376446..2d91dca8cc06 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -129,8 +129,7 @@ impl FromStr for BuiltInWindowFunction { "NTH_VALUE" => BuiltInWindowFunction::NthValue, _ => { return Err(DataFusionError::Plan(format!( - "There is no built-in window function named {}", - name + "There is no built-in window function named {name}" ))) } }) diff --git a/datafusion/jit/src/api.rs b/datafusion/jit/src/api.rs index 7020985a733a..a3989b5e5a9d 100644 --- a/datafusion/jit/src/api.rs +++ b/datafusion/jit/src/api.rs @@ -626,11 +626,11 @@ impl Display for GeneratedFunction { if i != 0 { write!(f, ", ")?; } - write!(f, "{}: {}", name, ty)?; + write!(f, "{name}: {ty}")?; } write!(f, ") -> ")?; if let Some((name, ty)) = &self.ret { - write!(f, "{}: {}", name, ty)?; + write!(f, "{name}: {ty}")?; } else { write!(f, "()")?; } diff --git a/datafusion/jit/src/ast.rs b/datafusion/jit/src/ast.rs index 5774cee11dfa..36741432ec25 100644 --- a/datafusion/jit/src/ast.rs +++ b/datafusion/jit/src/ast.rs @@ -170,8 +170,7 @@ impl TryFrom<(datafusion_expr::Expr, DFSchemaRef)> for Expr { datafusion_expr::Operator::Divide => BinaryExpr::Div, _ => { return Err(DataFusionError::NotImplemented(format!( - "Compiling binary expression {} not yet supported", - value + "Compiling binary expression {value} not yet supported" ))) } }; @@ -196,16 +195,14 @@ impl TryFrom<(datafusion_expr::Expr, DFSchemaRef)> for Expr { ScalarValue::Int64(Some(i)) => TypedLit::Int(*i), _ => { return Err(DataFusionError::NotImplemented(format!( - "Compiling Scalar {} not yet supported in JIT mode", - s + "Compiling Scalar {s} not yet supported in JIT mode" ))) } }; Ok(Expr::Literal(Literal::Typed(lit))) } _ => Err(DataFusionError::NotImplemented(format!( - "Compiling {} not yet supported", - value + "Compiling {value} not yet supported" ))), } } @@ -286,8 +283,7 @@ impl TryFrom<&DataType> for JITType { DataType::Boolean => Ok(BOOL), _ => Err(DataFusionError::NotImplemented(format!( - "Compiling Expression with type {} not yet supported in JIT mode", - df_type + "Compiling Expression with type {df_type} not yet supported in JIT mode" ))), } } @@ -302,25 +298,25 @@ impl Stmt { } match self { Stmt::IfElse(cond, then_stmts, else_stmts) => { - writeln!(f, "{}if {} {{", ident_str, cond)?; + writeln!(f, "{ident_str}if {cond} {{")?; for stmt in then_stmts { stmt.fmt_ident(ident + 4, f)?; } - writeln!(f, "{}}} else {{", ident_str)?; + writeln!(f, "{ident_str}}} else {{")?; for stmt in else_stmts { stmt.fmt_ident(ident + 4, f)?; } - writeln!(f, "{}}}", ident_str) + writeln!(f, "{ident_str}}}") } Stmt::WhileLoop(cond, stmts) => { - writeln!(f, "{}while {} {{", ident_str, cond)?; + writeln!(f, "{ident_str}while {cond} {{")?; for stmt in stmts { stmt.fmt_ident(ident + 4, f)?; } - writeln!(f, "{}}}", ident_str) + writeln!(f, "{ident_str}}}") } Stmt::Assign(name, expr) => { - writeln!(f, "{}{} = {};", ident_str, name, expr) + writeln!(f, "{ident_str}{name} = {expr};") } Stmt::Call(name, args) => { writeln!( @@ -335,10 +331,10 @@ impl Stmt { ) } Stmt::Declare(name, ty) => { - writeln!(f, "{}let {}: {};", ident_str, name, ty) + writeln!(f, "{ident_str}let {name}: {ty};") } Stmt::Store(value, ptr) => { - writeln!(f, "{}*({}) = {}", ident_str, ptr, value) + writeln!(f, "{ident_str}*({ptr}) = {value}") } } } @@ -354,9 +350,9 @@ impl Display for Stmt { impl Display for Expr { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - Expr::Literal(l) => write!(f, "{}", l), - Expr::Identifier(name, _) => write!(f, "{}", name), - Expr::Binary(be) => write!(f, "{}", be), + Expr::Literal(l) => write!(f, "{l}"), + Expr::Identifier(name, _) => write!(f, "{name}"), + Expr::Binary(be) => write!(f, "{be}"), Expr::Call(name, exprs, _) => { write!( f, @@ -369,7 +365,7 @@ impl Display for Expr { .join(", ") ) } - Expr::Load(ptr, _) => write!(f, "*({})", ptr,), + Expr::Load(ptr, _) => write!(f, "*({ptr})",), } } } @@ -377,8 +373,8 @@ impl Display for Expr { impl Display for Literal { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - Literal::Parsing(str, _) => write!(f, "{}", str), - Literal::Typed(tl) => write!(f, "{}", tl), + Literal::Parsing(str, _) => write!(f, "{str}"), + Literal::Typed(tl) => write!(f, "{tl}"), } } } @@ -386,10 +382,10 @@ impl Display for Literal { impl Display for TypedLit { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - TypedLit::Bool(b) => write!(f, "{}", b), - TypedLit::Int(i) => write!(f, "{}", i), - TypedLit::Float(fl) => write!(f, "{}", fl), - TypedLit::Double(d) => write!(f, "{}", d), + TypedLit::Bool(b) => write!(f, "{b}"), + TypedLit::Int(i) => write!(f, "{i}"), + TypedLit::Float(fl) => write!(f, "{fl}"), + TypedLit::Double(d) => write!(f, "{d}"), } } } @@ -397,23 +393,23 @@ impl Display for TypedLit { impl Display for BinaryExpr { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - BinaryExpr::Eq(lhs, rhs) => write!(f, "{} == {}", lhs, rhs), - BinaryExpr::Ne(lhs, rhs) => write!(f, "{} != {}", lhs, rhs), - BinaryExpr::Lt(lhs, rhs) => write!(f, "{} < {}", lhs, rhs), - BinaryExpr::Le(lhs, rhs) => write!(f, "{} <= {}", lhs, rhs), - BinaryExpr::Gt(lhs, rhs) => write!(f, "{} > {}", lhs, rhs), - BinaryExpr::Ge(lhs, rhs) => write!(f, "{} >= {}", lhs, rhs), - BinaryExpr::Add(lhs, rhs) => write!(f, "{} + {}", lhs, rhs), - BinaryExpr::Sub(lhs, rhs) => write!(f, "{} - {}", lhs, rhs), - BinaryExpr::Mul(lhs, rhs) => write!(f, "{} * {}", lhs, rhs), - BinaryExpr::Div(lhs, rhs) => write!(f, "{} / {}", lhs, rhs), + BinaryExpr::Eq(lhs, rhs) => write!(f, "{lhs} == {rhs}"), + BinaryExpr::Ne(lhs, rhs) => write!(f, "{lhs} != {rhs}"), + BinaryExpr::Lt(lhs, rhs) => write!(f, "{lhs} < {rhs}"), + BinaryExpr::Le(lhs, rhs) => write!(f, "{lhs} <= {rhs}"), + BinaryExpr::Gt(lhs, rhs) => write!(f, "{lhs} > {rhs}"), + BinaryExpr::Ge(lhs, rhs) => write!(f, "{lhs} >= {rhs}"), + BinaryExpr::Add(lhs, rhs) => write!(f, "{lhs} + {rhs}"), + BinaryExpr::Sub(lhs, rhs) => write!(f, "{lhs} - {rhs}"), + BinaryExpr::Mul(lhs, rhs) => write!(f, "{lhs} * {rhs}"), + BinaryExpr::Div(lhs, rhs) => write!(f, "{lhs} / {rhs}"), } } } impl std::fmt::Display for JITType { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{:?}", self) + write!(f, "{self:?}") } } @@ -447,7 +443,7 @@ impl From<&str> for JITType { "f64" => F64, "small_ptr" => R32, "ptr" => R64, - _ => panic!("unknown type: {}", x), + _ => panic!("unknown type: {x}"), } } } diff --git a/datafusion/jit/src/compile.rs b/datafusion/jit/src/compile.rs index 4e68b52104c0..93b463d5cb1c 100644 --- a/datafusion/jit/src/compile.rs +++ b/datafusion/jit/src/compile.rs @@ -42,7 +42,7 @@ pub fn build_calc_fn( // Each input takes one position, following by a pointer to place result, // and the last is the length of inputs/output arrays. for (name, _) in &inputs { - builder = builder.param(format!("{}_array", name), PTR_TYPE); + builder = builder.param(format!("{name}_array"), PTR_TYPE); } let mut builder = builder.param("result", ret_type).param("len", I64); @@ -56,10 +56,10 @@ pub fn build_calc_fn( w.declare_as("offset", w.mul(w.id("index")?, w.lit_i(PTR_SIZE as i64))?)?; for (name, ty) in &inputs { w.declare_as( - format!("{}_ptr", name), - w.add(w.id(format!("{}_array", name))?, w.id("offset")?)?, + format!("{name}_ptr"), + w.add(w.id(format!("{name}_array"))?, w.id("offset")?)?, )?; - w.declare_as(name, w.load(w.id(format!("{}_ptr", name))?, *ty)?)?; + w.declare_as(name, w.load(w.id(format!("{name}_ptr"))?, *ty)?)?; } w.declare_as("res_ptr", w.add(w.id("result")?, w.id("offset")?)?)?; w.declare_as("res", jit_expr.clone())?; diff --git a/datafusion/jit/src/jit.rs b/datafusion/jit/src/jit.rs index 9c28940fe701..57e4c8c5d1dc 100644 --- a/datafusion/jit/src/jit.rs +++ b/datafusion/jit/src/jit.rs @@ -62,12 +62,12 @@ impl Default for JIT { flag_builder.set("use_colocated_libcalls", "false").unwrap(); flag_builder.set("is_pic", "false").unwrap(); let isa_builder = cranelift_native::builder().unwrap_or_else(|msg| { - panic!("host machine is not supported: {}", msg); + panic!("host machine is not supported: {msg}"); }); let isa = isa_builder .finish(settings::Flags::new(flag_builder)) .unwrap_or_else(|msg| { - panic!("host machine is not supported: {}", msg); + panic!("host machine is not supported: {msg}"); }); let builder = JITBuilder::with_isa(isa, cranelift_module::default_libcall_names()); @@ -99,7 +99,7 @@ impl JIT { flag_builder.set("opt_level", "speed").unwrap(); flag_builder.set("enable_simd", "true").unwrap(); let isa_builder = cranelift_native::builder().unwrap_or_else(|msg| { - panic!("host machine is not supported: {}", msg); + panic!("host machine is not supported: {msg}"); }); let isa = isa_builder .finish(settings::Flags::new(flag_builder)) diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index c8bddcfbfacd..0ac5dc69ba79 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -388,7 +388,7 @@ enum VisitRecord { impl ExprIdentifierVisitor<'_> { fn desc_expr(expr: &Expr) -> String { - format!("{}", expr) + format!("{expr}") } /// Find the first `EnterMark` in the stack, and accumulates every `ExprItem` @@ -596,7 +596,7 @@ mod test { .try_optimize(plan, &OptimizerContext::new()) .unwrap() .expect("failed to optimize plan"); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(expected, formatted_plan); } diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index 28aae51d221b..458eab95905a 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -351,8 +351,7 @@ mod tests { assert_eq!( expected, actual, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected, actual + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); assert_eq!(plan.schema(), optimized_plan.schema()) diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index caea145dd2f2..7844ca7909fc 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -103,7 +103,7 @@ mod tests { )? .unwrap_or_else(|| plan.clone()); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); assert_eq!(plan.schema(), optimized_plan.schema()); Ok(()) @@ -122,7 +122,7 @@ mod tests { let optimized_plan = optimizer .optimize(plan, &config, observe) .expect("failed to optimize plan"); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); assert_eq!(plan.schema(), optimized_plan.schema()); Ok(()) diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index b27c49d2ffcc..5d2dd45af3cb 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -291,7 +291,7 @@ impl Optimizer { let mut new_plan = plan.clone(); let mut i = 0; while i < config.max_passes() { - log_plan(&format!("Optimizer input (pass {})", i), &new_plan); + log_plan(&format!("Optimizer input (pass {i})"), &new_plan); for rule in &self.rules { if !config.rule_enabled(rule.name()) { @@ -342,7 +342,7 @@ impl Optimizer { } } } - log_plan(&format!("Optimized plan (pass {})", i), &new_plan); + log_plan(&format!("Optimized plan (pass {i})"), &new_plan); // TODO this is an expensive way to see if the optimizer did anything and // it would be better to change the OptimizerRule trait to return an Option @@ -531,9 +531,8 @@ mod tests { .iter() .enumerate() .map(|(i, f)| { - let metadata = [("key".into(), format!("value {}", i))] - .into_iter() - .collect(); + let metadata = + [("key".into(), format!("value {i}"))].into_iter().collect(); let new_arrow_field = f.field().clone().with_metadata(metadata); if let Some(qualifier) = f.qualifier() { diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index e3a86381fbc1..d533803e549e 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -232,7 +232,7 @@ mod tests { let optimized_plan = optimizer .optimize(plan, config, observe) .expect("failed to optimize plan"); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); assert_eq!(plan.schema(), optimized_plan.schema()); Ok(()) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 64fc988b59b9..ff0ea5b23bef 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -795,7 +795,7 @@ mod tests { .try_optimize(plan, &OptimizerContext::new()) .unwrap() .expect("failed to optimize plan"); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(plan.schema(), optimized_plan.schema()); assert_eq!(expected, formatted_plan); Ok(()) @@ -964,7 +964,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "\ Filter: b = Int64(1)\ \n Projection: test.a * Int32(2) + test.c AS b, test.c\ @@ -995,7 +995,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "\ Filter: a = Int64(1)\ \n Projection: b * Int32(3) AS a, test.c\ @@ -1027,7 +1027,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "\ Filter: SUM(test.c) > Int64(10)\ \n Filter: b > Int64(10)\ @@ -1063,7 +1063,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "\ Filter: SUM(test.c) > Int64(10) AND b > Int64(10) AND SUM(test.c) < Int64(20)\ \n Aggregate: groupBy=[[b]], aggr=[[SUM(test.c)]]\ @@ -1221,7 +1221,7 @@ mod tests { // not part of the test assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test.a >= Int64(1)\ \n Projection: test.a\ \n Limit: skip=0, fetch=1\ @@ -1254,7 +1254,7 @@ mod tests { // not part of the test assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Projection: test.a\ \n Filter: test.a >= Int64(1)\ \n Filter: test.a <= Int64(1)\ @@ -1288,7 +1288,7 @@ mod tests { \n TableScan: test"; // not part of the test - assert_eq!(format!("{:?}", plan), expected); + assert_eq!(format!("{plan:?}"), expected); assert_optimized_plan_eq(&plan, expected) } @@ -1314,7 +1314,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test.a <= Int64(1)\ \n Inner Join: test.a = test2.a\ \n TableScan: test\ @@ -1353,7 +1353,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test.a <= Int64(1)\ \n Inner Join: Using test.a = test2.a\ \n TableScan: test\ @@ -1396,7 +1396,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test.c <= test2.b\ \n Inner Join: test.a = test2.a\ \n Projection: test.a, test.c\ @@ -1406,7 +1406,7 @@ mod tests { ); // expected is equal: no push-down - let expected = &format!("{:?}", plan); + let expected = &format!("{plan:?}"); assert_optimized_plan_eq(&plan, expected) } @@ -1434,7 +1434,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test.b <= Int64(1)\ \n Inner Join: test.a = test2.a\ \n Projection: test.a, test.b\ @@ -1474,7 +1474,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test2.a <= Int64(1)\ \n Left Join: Using test.a = test2.a\ \n TableScan: test\ @@ -1512,7 +1512,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test.a <= Int64(1)\ \n Right Join: Using test.a = test2.a\ \n TableScan: test\ @@ -1551,7 +1551,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test.a <= Int64(1)\ \n Left Join: Using test.a = test2.a\ \n TableScan: test\ @@ -1590,7 +1590,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: test2.a <= Int64(1)\ \n Right Join: Using test.a = test2.a\ \n TableScan: test\ @@ -1634,7 +1634,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Inner Join: test.a = test2.a Filter: test.c > UInt32(1) AND test.b < test2.b AND test2.c > UInt32(4)\ \n Projection: test.a, test.b, test.c\ \n TableScan: test\ @@ -1678,7 +1678,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Inner Join: test.a = test2.a Filter: test.b > UInt32(1) AND test2.c > UInt32(4)\ \n Projection: test.a, test.b, test.c\ \n TableScan: test\ @@ -1720,7 +1720,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Inner Join: test.a = test2.b Filter: test.a > UInt32(1)\ \n Projection: test.a\ \n TableScan: test\ @@ -1765,7 +1765,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Left Join: test.a = test2.a Filter: test.a > UInt32(1) AND test.b < test2.b AND test2.c > UInt32(4)\ \n Projection: test.a, test.b, test.c\ \n TableScan: test\ @@ -1809,7 +1809,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Right Join: test.a = test2.a Filter: test.a > UInt32(1) AND test.b < test2.b AND test2.c > UInt32(4)\ \n Projection: test.a, test.b, test.c\ \n TableScan: test\ @@ -1853,7 +1853,7 @@ mod tests { // not part of the test, just good to know: assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Full Join: test.a = test2.a Filter: test.a > UInt32(1) AND test.b < test2.b AND test2.c > UInt32(4)\ \n Projection: test.a, test.b, test.c\ \n TableScan: test\ @@ -1861,7 +1861,7 @@ mod tests { \n TableScan: test2" ); - let expected = &format!("{:?}", plan); + let expected = &format!("{plan:?}"); assert_optimized_plan_eq(&plan, expected) } @@ -2007,7 +2007,7 @@ mod tests { // filter on col b assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: b > Int64(10) AND test.c > Int64(10)\ \n Projection: test.a AS b, test.c\ \n TableScan: test" @@ -2037,7 +2037,7 @@ mod tests { // filter on col b assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: b > Int64(10) AND test.c > Int64(10)\ \n Projection: b, test.c\ \n Projection: test.a AS b, test.c\ @@ -2066,7 +2066,7 @@ mod tests { // filter on col b and d assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: b > Int64(10) AND d > Int64(10)\ \n Projection: test.a AS b, test.c AS d\ \n TableScan: test\ @@ -2105,7 +2105,7 @@ mod tests { .build()?; assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Inner Join: c = d Filter: c > UInt32(1)\ \n Projection: test.a AS c\ \n TableScan: test\ @@ -2139,7 +2139,7 @@ mod tests { // filter on col b assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: b IN ([UInt32(1), UInt32(2), UInt32(3), UInt32(4)])\ \n Projection: test.a AS b, test.c\ \n TableScan: test\ @@ -2171,7 +2171,7 @@ mod tests { // filter on col b assert_eq!( - format!("{:?}", plan), + format!("{plan:?}"), "Filter: b IN ([UInt32(1), UInt32(2), UInt32(3), UInt32(4)])\ \n Projection: b, test.c\ \n Projection: test.a AS b, test.c\ @@ -2214,7 +2214,7 @@ mod tests { \n TableScan: sq\ \n Projection: test.a AS b, test.c\ \n TableScan: test"; - assert_eq!(format!("{:?}", plan), expected_before); + assert_eq!(format!("{plan:?}"), expected_before); // rewrite filter col b to test.a let expected_after = "\ @@ -2246,7 +2246,7 @@ mod tests { \n SubqueryAlias: b\ \n Projection: Int64(0) AS a\ \n EmptyRelation"; - assert_eq!(format!("{:?}", plan), expected_before); + assert_eq!(format!("{plan:?}"), expected_before); // Ensure that the predicate without any columns (0 = 1) is // still there. diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 45e79180c3f0..482c08fb444e 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -675,7 +675,7 @@ mod tests { \n TableScan: test2 projection=[c1]"; let optimized_plan = optimize(&plan)?; - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); // make sure schema for join node include both join columns @@ -718,7 +718,7 @@ mod tests { \n TableScan: test2 projection=[c1]"; let optimized_plan = optimize(&plan)?; - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); // make sure schema for join node include both join columns @@ -759,7 +759,7 @@ mod tests { \n TableScan: test2 projection=[a]"; let optimized_plan = optimize(&plan)?; - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); // make sure schema for join node include both join columns @@ -951,8 +951,8 @@ mod tests { let optimized_plan2 = optimize(&optimized_plan1).expect("failed to optimize plan"); - let formatted_plan1 = format!("{:?}", optimized_plan1); - let formatted_plan2 = format!("{:?}", optimized_plan2); + let formatted_plan1 = format!("{optimized_plan1:?}"); + let formatted_plan2 = format!("{optimized_plan2:?}"); assert_eq!(formatted_plan1, formatted_plan2); Ok(()) } @@ -1011,7 +1011,7 @@ mod tests { fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) { let optimized_plan = optimize(plan).expect("failed to optimize plan"); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); } diff --git a/datafusion/optimizer/src/simplify_expressions/context.rs b/datafusion/optimizer/src/simplify_expressions/context.rs index da44a7e8fd28..379a803f47a5 100644 --- a/datafusion/optimizer/src/simplify_expressions/context.rs +++ b/datafusion/optimizer/src/simplify_expressions/context.rs @@ -118,8 +118,7 @@ impl<'a> SimplifyInfo for SimplifyContext<'a> { // This means we weren't able to compute `Expr::nullable` with // *any* input schemas, signalling a problem DataFusionError::Internal(format!( - "Could not find columns in '{}' during simplify", - expr + "Could not find columns in '{expr}' during simplify" )) }) } diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 08c96168dc5d..c78914167188 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -922,8 +922,7 @@ mod tests { assert_eq!( evaluated_expr, expected_expr, - "Mismatch evaluating {}\n Expected:{}\n Got:{}", - input_expr, expected_expr, evaluated_expr + "Mismatch evaluating {input_expr}\n Expected:{expected_expr}\n Got:{evaluated_expr}" ); } diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index e4fe2e13751a..fe5d62c64a59 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -176,7 +176,7 @@ mod tests { .try_optimize(plan, &OptimizerContext::new()) .unwrap() .expect("failed to optimize plan"); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); Ok(()) } @@ -402,7 +402,7 @@ mod tests { .try_optimize(plan, &config) .unwrap() .expect("failed to optimize plan"); - format!("{:?}", optimized_plan) + format!("{optimized_plan:?}") } #[test] diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index 94f6e86b37a9..7bf7b4eccf79 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -165,8 +165,7 @@ pub fn as_bool_lit(expr: Expr) -> Result> { match expr { Expr::Literal(ScalarValue::Boolean(v)) => Ok(v), _ => Err(DataFusionError::Internal(format!( - "Expected boolean literal, got {:?}", - expr + "Expected boolean literal, got {expr:?}" ))), } } @@ -260,8 +259,7 @@ pub fn simpl_concat(args: Vec) -> Result { ) => contiguous_scalar += &v, Expr::Literal(x) => { return Err(DataFusionError::Internal(format!( - "The scalar {} should be casted to string type during the type coercion.", - x + "The scalar {x} should be casted to string type during the type coercion." ))) } // If the arg is not a literal, we should first push the current `contiguous_scalar` @@ -318,7 +316,7 @@ pub fn simpl_concat_ws(delimiter: &Expr, args: &[Expr]) -> Result { } } } - Expr::Literal(s) => return Err(DataFusionError::Internal(format!("The scalar {} should be casted to string type during the type coercion.", s))), + Expr::Literal(s) => return Err(DataFusionError::Internal(format!("The scalar {s} should be casted to string type during the type coercion."))), // If the arg is not a literal, we should first push the current `contiguous_scalar` // to the `new_args` and reset it to None. // Then pushing this arg to the `new_args`. @@ -344,8 +342,7 @@ pub fn simpl_concat_ws(delimiter: &Expr, args: &[Expr]) -> Result { } } Expr::Literal(d) => Err(DataFusionError::Internal(format!( - "The scalar {} should be casted to string type during the type coercion.", - d + "The scalar {d} should be casted to string type during the type coercion." ))), d => Ok(concat_ws( d.clone(), diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index c1e5f411afd0..cee31b5b3352 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -108,7 +108,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { .iter() .enumerate() .map(|(i, group_expr)| { - let alias_str = format!("group_alias_{}", i); + let alias_str = format!("group_alias_{i}"); let alias_expr = group_expr.clone().alias(&alias_str); group_expr_alias .push((alias_str, schema.fields()[i].clone())); diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index 7532a9d1a378..bd8c1f98a0c5 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -115,7 +115,7 @@ pub fn assert_optimized_plan_eq( &OptimizerContext::new(), )? .unwrap_or_else(|| plan.clone()); - let formatted_plan = format!("{:?}", optimized_plan); + let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); Ok(()) @@ -153,7 +153,7 @@ pub fn assert_optimizer_err( match res { Ok(plan) => assert_eq!(format!("{}", plan.unwrap().display_indent()), "An error"), Err(ref e) => { - let actual = format!("{}", e); + let actual = format!("{e}"); if expected.is_empty() || !actual.contains(expected) { assert_eq!(actual, expected) } diff --git a/datafusion/optimizer/src/type_coercion.rs b/datafusion/optimizer/src/type_coercion.rs index a957bb41b6f8..23d1d37065a0 100644 --- a/datafusion/optimizer/src/type_coercion.rs +++ b/datafusion/optimizer/src/type_coercion.rs @@ -252,24 +252,21 @@ impl ExprRewriter for TypeCoercionRewriter { let low_coerced_type = comparison_coercion(&expr_type, &low_type) .ok_or_else(|| { DataFusionError::Internal(format!( - "Failed to coerce types {} and {} in BETWEEN expression", - expr_type, low_type + "Failed to coerce types {expr_type} and {low_type} in BETWEEN expression" )) })?; let high_type = high.get_type(&self.schema)?; let high_coerced_type = comparison_coercion(&expr_type, &low_type) .ok_or_else(|| { DataFusionError::Internal(format!( - "Failed to coerce types {} and {} in BETWEEN expression", - expr_type, high_type + "Failed to coerce types {expr_type} and {high_type} in BETWEEN expression" )) })?; let coercion_type = comparison_coercion(&low_coerced_type, &high_coerced_type) .ok_or_else(|| { DataFusionError::Internal(format!( - "Failed to coerce types {} and {} in BETWEEN expression", - expr_type, high_type + "Failed to coerce types {expr_type} and {high_type} in BETWEEN expression" )) })?; let expr = Expr::Between(Between::new( @@ -294,8 +291,7 @@ impl ExprRewriter for TypeCoercionRewriter { get_coerce_type_for_list(&expr_data_type, &list_data_types); match result_type { None => Err(DataFusionError::Plan(format!( - "Can not find compatible types to compare {:?} with {:?}", - expr_data_type, list_data_types + "Can not find compatible types to compare {expr_data_type:?} with {list_data_types:?}" ))), Some(coerced_type) => { // find the coerced type @@ -331,8 +327,7 @@ impl ExprRewriter for TypeCoercionRewriter { get_coerce_type_for_case_when(&then_types, else_type.as_ref()); match case_when_coerce_type { None => Err(DataFusionError::Internal(format!( - "Failed to coerce then ({:?}) and else ({:?}) to common types in CASE WHEN expression", - then_types, else_type + "Failed to coerce then ({then_types:?}) and else ({else_type:?}) to common types in CASE WHEN expression" ))), Some(data_type) => { let left = case.when_then_expr @@ -487,8 +482,7 @@ fn get_coerced_window_frame( Ok(DataType::Interval(IntervalUnit::MonthDayNano)) } else { Err(DataFusionError::Internal(format!( - "Cannot run range queries on datatype: {:?}", - column_type + "Cannot run range queries on datatype: {column_type:?}" ))) } } @@ -611,7 +605,7 @@ mod test { let rule = TypeCoercion::new(); let config = OptimizerContext::default(); let plan = rule.try_optimize(plan, &config)?.unwrap(); - assert_eq!(expected, &format!("{:?}", plan)); + assert_eq!(expected, &format!("{plan:?}")); Ok(()) } @@ -696,7 +690,7 @@ mod test { let err = assert_optimized_plan_eq(&plan, "").err().unwrap(); assert_eq!( "Plan(\"Coercion from [Utf8] to the signature Uniform(1, [Int32]) failed.\")", - &format!("{:?}", err) + &format!("{err:?}") ); Ok(()) } @@ -764,7 +758,7 @@ mod test { let err = assert_optimized_plan_eq(&plan, "").err().unwrap(); assert_eq!( "Plan(\"Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed.\")", - &format!("{:?}", err) + &format!("{err:?}") ); Ok(()) } @@ -810,7 +804,7 @@ mod test { let err = Projection::try_new(vec![agg_expr], empty).err().unwrap(); assert_eq!( "Plan(\"The function Avg does not support inputs of type Utf8.\")", - &format!("{:?}", err) + &format!("{err:?}") ); Ok(()) } diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 9c4887699ea7..17065433d523 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -325,8 +325,7 @@ fn try_cast_literal_to_type( DataType::Decimal128(_, scale) => 10_i128.pow(*scale as u32), other_type => { return Err(DataFusionError::Internal(format!( - "Error target data type {:?}", - other_type + "Error target data type {other_type:?}" ))); } }; @@ -349,8 +348,7 @@ fn try_cast_literal_to_type( ), other_type => { return Err(DataFusionError::Internal(format!( - "Error target data type {:?}", - other_type + "Error target data type {other_type:?}" ))); } }; @@ -388,8 +386,7 @@ fn try_cast_literal_to_type( } other_value => { return Err(DataFusionError::Internal(format!( - "Invalid literal value {:?}", - other_value + "Invalid literal value {other_value:?}" ))); } }; @@ -426,8 +423,7 @@ fn try_cast_literal_to_type( } other_type => { return Err(DataFusionError::Internal(format!( - "Error target data type {:?}", - other_type + "Error target data type {other_type:?}" ))); } }; @@ -1031,9 +1027,9 @@ mod tests { let actual_result = try_cast_literal_to_type(&literal, &target_type); println!("expect_cast: "); - println!(" {:?} --> {:?}", literal, target_type); - println!(" expected_result: {:?}", expected_result); - println!(" actual_result: {:?}", actual_result); + println!(" {literal:?} --> {target_type:?}"); + println!(" expected_result: {expected_result:?}"); + println!(" actual_result: {actual_result:?}"); match expected_result { ExpectedCast::Value(expected_value) => { @@ -1057,8 +1053,7 @@ mod tests { assert_eq!( &expected_array, &cast_array, - "Result of casing {:?} with arrow was\n {:#?}\nbut expected\n{:#?}", - literal, cast_array, expected_array + "Result of casing {literal:?} with arrow was\n {cast_array:#?}\nbut expected\n{expected_array:#?}" ); // Verify that for timestamp types the timezones are the same @@ -1077,8 +1072,7 @@ mod tests { assert!( actual_value.is_none(), - "Expected no cast value, but got {:?}", - actual_value + "Expected no cast value, but got {actual_value:?}" ); } } diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 071ca149b082..d9c7477cbb8a 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -422,7 +422,7 @@ pub fn find_join_exprs( Operator::Eq => {} Operator::NotEq => {} _ => { - plan_err!(format!("can't optimize {} column comparison", op))?; + plan_err!(format!("can't optimize {op} column comparison"))?; } } @@ -471,7 +471,7 @@ pub fn exprs_to_join_cols( continue; } } - _ => plan_err!(format!("Correlation operator unsupported: {}", op))?, + _ => plan_err!(format!("Correlation operator unsupported: {op}"))?, } let left = left.try_into_col()?; let right = right.try_into_col()?; diff --git a/datafusion/optimizer/tests/integration-test.rs b/datafusion/optimizer/tests/integration-test.rs index 62a8f1ef2cd9..3a09af2f2113 100644 --- a/datafusion/optimizer/tests/integration-test.rs +++ b/datafusion/optimizer/tests/integration-test.rs @@ -43,13 +43,13 @@ fn case_when() -> Result<()> { let expected = "Projection: CASE WHEN test.col_int32 > Int32(0) THEN Int64(1) ELSE Int64(0) END AS CASE WHEN test.col_int32 > Int64(0) THEN Int64(1) ELSE Int64(0) END\ \n TableScan: test projection=[col_int32]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); let sql = "SELECT CASE WHEN col_uint32 > 0 THEN 1 ELSE 0 END FROM test"; let plan = test_sql(sql)?; let expected = "Projection: CASE WHEN test.col_uint32 > UInt32(0) THEN Int64(1) ELSE Int64(0) END AS CASE WHEN test.col_uint32 > Int64(0) THEN Int64(1) ELSE Int64(0) END\ \n TableScan: test projection=[col_uint32]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -72,7 +72,7 @@ fn subquery_filter_with_cast() -> Result<()> { \n Aggregate: groupBy=[[]], aggr=[[AVG(test.col_int32)]]\ \n Filter: test.col_utf8 >= Utf8(\"2002-05-08\") AND test.col_utf8 <= Utf8(\"2002-05-13\")\ \n TableScan: test projection=[col_int32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -83,7 +83,7 @@ fn case_when_aggregate() -> Result<()> { let expected = "Projection: test.col_utf8, SUM(CASE WHEN test.col_int32 > Int64(0) THEN Int64(1) ELSE Int64(0) END) AS n\ \n Aggregate: groupBy=[[test.col_utf8]], aggr=[[SUM(CASE WHEN test.col_int32 > Int32(0) THEN Int64(1) ELSE Int64(0) END) AS SUM(CASE WHEN test.col_int32 > Int64(0) THEN Int64(1) ELSE Int64(0) END)]]\ \n TableScan: test projection=[col_int32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -94,7 +94,7 @@ fn unsigned_target_type() -> Result<()> { let expected = "Projection: test.col_utf8\ \n Filter: test.col_uint32 > UInt32(0)\ \n TableScan: test projection=[col_uint32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -106,7 +106,7 @@ fn distribute_by() -> Result<()> { let expected = "Repartition: DistributeBy(col_utf8)\ \n Projection: test.col_int32, test.col_utf8\ \n TableScan: test projection=[col_int32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -122,7 +122,7 @@ fn semi_join_with_join_filter() -> Result<()> { \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ \n SubqueryAlias: t2\ \n TableScan: test projection=[col_int32, col_uint32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -138,7 +138,7 @@ fn anti_join_with_join_filter() -> Result<()> { \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ \n SubqueryAlias: t2\ \n TableScan: test projection=[col_int32, col_uint32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -153,7 +153,7 @@ fn where_exists_distinct() -> Result<()> { \n TableScan: test projection=[col_int32]\ \n SubqueryAlias: t2\ \n TableScan: test projection=[col_int32]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -171,7 +171,7 @@ fn intersect() -> Result<()> { \n TableScan: test projection=[col_int32, col_utf8]\ \n TableScan: test projection=[col_int32, col_utf8]\ \n TableScan: test projection=[col_int32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -184,7 +184,7 @@ fn between_date32_plus_interval() -> Result<()> { "Projection: COUNT(Int64(1))\n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ \n Filter: test.col_date32 >= Date32(\"10303\") AND test.col_date32 <= Date32(\"10393\")\ \n TableScan: test projection=[col_date32]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -197,7 +197,7 @@ fn between_date64_plus_interval() -> Result<()> { "Projection: COUNT(Int64(1))\n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ \n Filter: test.col_date64 >= Date64(\"890179200000\") AND test.col_date64 <= Date64(\"897955200000\")\ \n TableScan: test projection=[col_date64]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -210,7 +210,7 @@ fn concat_literals() -> Result<()> { let expected = "Projection: concat(Utf8(\"1\"), CAST(test.col_int32 AS Utf8), Utf8(\"0hello\"), test.col_utf8, Utf8(\"123.4\")) AS col\ \n TableScan: test projection=[col_int32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -223,7 +223,7 @@ fn concat_ws_literals() -> Result<()> { let expected = "Projection: concatwithseparator(Utf8(\"-\"), Utf8(\"1\"), CAST(test.col_int32 AS Utf8), Utf8(\"0-hello\"), test.col_utf8, Utf8(\"12--3.4\")) AS col\ \n TableScan: test projection=[col_int32, col_utf8]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -240,7 +240,7 @@ fn timestamp_nano_ts_none_predicates() -> Result<()> { "Projection: test.col_int32\ \n Filter: test.col_ts_nano_none < TimestampNanosecond(1666612093000000000, None)\ \n TableScan: test projection=[col_int32, col_ts_nano_none]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -256,7 +256,7 @@ fn timestamp_nano_ts_utc_predicates() { let expected = "Projection: test.col_int32\n Filter: test.col_ts_nano_utc < TimestampNanosecond(1666612093000000000, Some(\"+00:00\"))\ \n TableScan: test projection=[col_int32, col_ts_nano_utc]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); } #[test] @@ -265,7 +265,7 @@ fn propagate_empty_relation() { let plan = test_sql(sql).unwrap(); // when children exist EmptyRelation, it will bottom-up propagate. let expected = "EmptyRelation"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); } #[test] @@ -281,7 +281,7 @@ fn join_keys_in_subquery_alias() { \n Projection: test.col_int32 AS key\ \n Filter: test.col_int32 IS NOT NULL\ \n TableScan: test projection=[col_int32]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); } #[test] @@ -301,7 +301,7 @@ fn join_keys_in_subquery_alias_1() { \n SubqueryAlias: c\ \n Filter: test.col_int32 IS NOT NULL\ \n TableScan: test projection=[col_int32]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); } #[test] @@ -313,7 +313,7 @@ fn push_down_filter_groupby_expr_contains_alias() { \n Aggregate: groupBy=[[test.col_int32 + CAST(test.col_uint32 AS Int32)]], aggr=[[COUNT(UInt8(1))]]\ \n Filter: test.col_int32 + CAST(test.col_uint32 AS Int32) > Int32(3)\ \n TableScan: test projection=[col_int32, col_uint32]"; - assert_eq!(expected, format!("{:?}", plan)); + assert_eq!(expected, format!("{plan:?}")); } fn test_sql(sql: &str) -> Result { diff --git a/datafusion/physical-expr/benches/in_list.rs b/datafusion/physical-expr/benches/in_list.rs index 415d124dc7fa..db017326083a 100644 --- a/datafusion/physical-expr/benches/in_list.rs +++ b/datafusion/physical-expr/benches/in_list.rs @@ -63,8 +63,7 @@ fn do_benches( do_bench( c, &format!( - "in_list_utf8({}) ({}, {}) IN ({}, 0)", - string_length, array_length, null_percent, in_list_length + "in_list_utf8({string_length}) ({array_length}, {null_percent}) IN ({in_list_length}, 0)" ), Arc::new(values), &in_list, @@ -81,10 +80,7 @@ fn do_benches( do_bench( c, - &format!( - "in_list_f32 ({}, {}) IN ({}, 0)", - array_length, null_percent, in_list_length - ), + &format!("in_list_f32 ({array_length}, {null_percent}) IN ({in_list_length}, 0)"), Arc::new(values), &in_list, ); @@ -99,10 +95,7 @@ fn do_benches( do_bench( c, - &format!( - "in_list_i32 ({}, {}) IN ({}, 0)", - array_length, null_percent, in_list_length - ), + &format!("in_list_i32 ({array_length}, {null_percent}) IN ({in_list_length}, 0)"), Arc::new(values), &in_list, ) diff --git a/datafusion/physical-expr/src/aggregate/approx_distinct.rs b/datafusion/physical-expr/src/aggregate/approx_distinct.rs index e327f39076d9..73fa7f79df98 100644 --- a/datafusion/physical-expr/src/aggregate/approx_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/approx_distinct.rs @@ -102,9 +102,8 @@ impl AggregateExpr for ApproxDistinct { DataType::LargeBinary => Box::new(BinaryHLLAccumulator::::new()), other => { return Err(DataFusionError::NotImplemented(format!( - "Support for 'approx_distinct' for data type {} is not implemented", - other - ))) + "Support for 'approx_distinct' for data type {other} is not implemented" + ))) } }; Ok(accumulator) diff --git a/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs b/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs index b768c07256fb..670030b09e66 100644 --- a/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs +++ b/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs @@ -106,8 +106,7 @@ impl ApproxPercentileCont { } other => { return Err(DataFusionError::NotImplemented(format!( - "Support for 'APPROX_PERCENTILE_CONT' for data type {} is not implemented", - other + "Support for 'APPROX_PERCENTILE_CONT' for data type {other} is not implemented" ))) } }; @@ -138,8 +137,7 @@ fn validate_input_percentile_expr(expr: &Arc) -> Result { // Ensure the percentile is between 0 and 1. if !(0.0..=1.0).contains(&percentile) { return Err(DataFusionError::Plan(format!( - "Percentile value must be between 0.0 and 1.0 inclusive, {} is invalid", - percentile + "Percentile value must be between 0.0 and 1.0 inclusive, {percentile} is invalid" ))); } Ok(percentile) @@ -349,8 +347,7 @@ impl ApproxPercentileAccumulator { .collect::>>()?) } e => Err(DataFusionError::Internal(format!( - "APPROX_PERCENTILE_CONT is not expected to receive the type {:?}", - e + "APPROX_PERCENTILE_CONT is not expected to receive the type {e:?}" ))), } } diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 597b5157554b..5e609f125bc8 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -1062,8 +1062,7 @@ mod tests { )?; if coerced_phy_exprs.is_empty() { return Err(DataFusionError::Plan(format!( - "Invalid or wrong number of arguments passed to aggregate: '{}'", - name, + "Invalid or wrong number of arguments passed to aggregate: '{name}'", ))); } create_aggregate_expr(fun, distinct, &coerced_phy_exprs, input_schema, name) diff --git a/datafusion/physical-expr/src/aggregate/count_distinct.rs b/datafusion/physical-expr/src/aggregate/count_distinct.rs index 2a2c7e940cd3..87c0518d189b 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct.rs @@ -136,8 +136,7 @@ impl DistinctCountAccumulator { .map(|state| match state { ScalarValue::List(Some(values), _) => Ok(values), _ => Err(DataFusionError::Internal(format!( - "Unexpected accumulator state {:?}", - state + "Unexpected accumulator state {state:?}" ))), }) .collect::>>()?; @@ -191,8 +190,7 @@ impl Accumulator for DistinctCountAccumulator { .map(|c| match c { ScalarValue::List(Some(ref mut v), _) => Ok(v), t => Err(DataFusionError::Internal(format!( - "cols_out should only consist of ScalarValue::List. {:?} is found", - t + "cols_out should only consist of ScalarValue::List. {t:?} is found" ))), }) .into_iter() @@ -213,8 +211,7 @@ impl Accumulator for DistinctCountAccumulator { match &self.count_data_type { DataType::Int64 => Ok(ScalarValue::Int64(Some(self.values.len() as i64))), t => Err(DataFusionError::Internal(format!( - "Invalid data type {:?} for count distinct aggregation", - t + "Invalid data type {t:?} for count distinct aggregation" ))), } } @@ -544,8 +541,7 @@ mod tests { DataFusionError::Internal("Found None count".to_string()) }), scalar => Err(DataFusionError::Internal(format!( - "Found non int64 scalar value from count: {}", - scalar + "Found non int64 scalar value from count: {scalar}" ))), }?; Ok((state_vec, count)) diff --git a/datafusion/physical-expr/src/aggregate/median.rs b/datafusion/physical-expr/src/aggregate/median.rs index 2e2675ce62cf..9daabc01d8be 100644 --- a/datafusion/physical-expr/src/aggregate/median.rs +++ b/datafusion/physical-expr/src/aggregate/median.rs @@ -134,8 +134,7 @@ impl Accumulator for MedianAccumulator { ScalarValue::List(None, _) => {} // skip empty state v => { return Err(DataFusionError::Internal(format!( - "unexpected state in median. Expected DataType::List, got {:?}", - v + "unexpected state in median. Expected DataType::List, got {v:?}" ))) } } @@ -181,8 +180,7 @@ impl Accumulator for MedianAccumulator { ScalarValue::Float64(Some(v)) => ScalarValue::Float64(Some(v / 2.0)), v => { return Err(DataFusionError::Internal(format!( - "Unsupported type in MedianAccumulator: {:?}", - v + "Unsupported type in MedianAccumulator: {v:?}" ))) } } diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 947336596292..528a5cc73f40 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -98,8 +98,7 @@ pub trait AggregateExpr: Send + Sync + Debug { _start_index: usize, ) -> Result> { Err(DataFusionError::NotImplemented(format!( - "RowAccumulator hasn't been implemented for {:?} yet", - self + "RowAccumulator hasn't been implemented for {self:?} yet" ))) } @@ -114,8 +113,7 @@ pub trait AggregateExpr: Send + Sync + Debug { /// Creates accumulator implementation that supports retract fn create_sliding_accumulator(&self) -> Result> { Err(DataFusionError::NotImplemented(format!( - "Retractable Accumulator hasn't been implemented for {:?} yet", - self + "Retractable Accumulator hasn't been implemented for {self:?} yet" ))) } } diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index c2d54c40ed7c..5de9a9296bc9 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -193,8 +193,7 @@ pub(crate) fn sum_batch(values: &ArrayRef, sum_type: &DataType) -> Result typed_sum_delta_batch!(values, UInt8Array, UInt8), e => { return Err(DataFusionError::Internal(format!( - "Sum is not expected to receive the type {:?}", - e + "Sum is not expected to receive the type {e:?}" ))); } }) @@ -229,10 +228,8 @@ pub(crate) fn add_to_row( sum_row!(index, accessor, rhs, i64) } _ => { - let msg = format!( - "Row sum updater is not expected to receive a scalar {:?}", - s - ); + let msg = + format!("Row sum updater is not expected to receive a scalar {s:?}"); return Err(DataFusionError::Internal(msg)); } } diff --git a/datafusion/physical-expr/src/aggregate/sum_distinct.rs b/datafusion/physical-expr/src/aggregate/sum_distinct.rs index e9e220e5f10f..f18aeb0671c7 100644 --- a/datafusion/physical-expr/src/aggregate/sum_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/sum_distinct.rs @@ -119,8 +119,7 @@ impl DistinctSumAccumulator { states.iter().try_for_each(|state| match state { ScalarValue::List(Some(values), _) => self.update(values.as_ref()), _ => Err(DataFusionError::Internal(format!( - "Unexpected accumulator state {:?}", - state + "Unexpected accumulator state {state:?}" ))), }) } diff --git a/datafusion/physical-expr/src/aggregate/tdigest.rs b/datafusion/physical-expr/src/aggregate/tdigest.rs index 73fe8788a699..7e6d2dcf8f4f 100644 --- a/datafusion/physical-expr/src/aggregate/tdigest.rs +++ b/datafusion/physical-expr/src/aggregate/tdigest.rs @@ -596,7 +596,7 @@ impl TDigest { let max_size = match &state[0] { ScalarValue::UInt64(Some(v)) => *v as usize, - v => panic!("invalid max_size type {:?}", v), + v => panic!("invalid max_size type {v:?}"), }; let centroids: Vec<_> = match &state[5] { @@ -604,7 +604,7 @@ impl TDigest { .chunks(2) .map(|v| Centroid::new(cast_scalar_f64!(v[0]), cast_scalar_f64!(v[1]))) .collect(), - v => panic!("invalid centroids type {:?}", v), + v => panic!("invalid centroids type {v:?}"), }; let max = cast_scalar_f64!(&state[3]); diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index ff57761f7ff0..7c41299e9e6f 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -97,8 +97,7 @@ fn array_array(args: &[ArrayRef]) -> Result { DataType::UInt64 => array!(args, UInt64Array, UInt64Builder), data_type => { return Err(DataFusionError::NotImplemented(format!( - "Array is not implemented for type '{:?}'.", - data_type + "Array is not implemented for type '{data_type:?}'." ))) } }; diff --git a/datafusion/physical-expr/src/crypto_expressions.rs b/datafusion/physical-expr/src/crypto_expressions.rs index 33806ac99ccf..205cb989ad76 100644 --- a/datafusion/physical-expr/src/crypto_expressions.rs +++ b/datafusion/physical-expr/src/crypto_expressions.rs @@ -66,8 +66,7 @@ fn digest_process( digest_algorithm.digest_binary_array::(a.as_ref()) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function {}", - other, digest_algorithm, + "Unsupported data type {other:?} for function {digest_algorithm}", ))), }, ColumnarValue::Scalar(scalar) => match scalar { @@ -78,8 +77,7 @@ fn digest_process( ScalarValue::Binary(a) | ScalarValue::LargeBinary(a) => Ok(digest_algorithm .digest_scalar(a.as_ref().map(|v: &Vec| v.as_slice()))), other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function {}", - other, digest_algorithm, + "Unsupported data type {other:?} for function {digest_algorithm}", ))), }, } @@ -195,7 +193,7 @@ impl DigestAlgorithm { impl fmt::Display for DigestAlgorithm { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", format!("{:?}", self).to_lowercase()) + write!(f, "{}", format!("{self:?}").to_lowercase()) } } @@ -227,9 +225,7 @@ impl FromStr for DigestAlgorithm { .collect::>() .join(", "); return Err(DataFusionError::Plan(format!( - "There is no built-in digest algorithm named '{}', currently supported algorithms are: {}", - name, - options, + "There is no built-in digest algorithm named '{name}', currently supported algorithms are: {options}", ))); } }) @@ -259,7 +255,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, "{:02x}", b).unwrap(); + write!(&mut s, "{b:02x}").unwrap(); } s } @@ -347,8 +343,7 @@ pub fn digest(args: &[ColumnarValue]) -> Result { method.parse::() } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function digest", - other, + "Unsupported data type {other:?} for function digest", ))), }, ColumnarValue::Array(_) => Err(DataFusionError::Internal( diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 76ac4c3bc6b7..4b3eeb4424d9 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -101,8 +101,7 @@ where unary_string_to_primitive_function::(&[a.as_ref()], op, name)?, ))), other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function {}", - other, name, + "Unsupported data type {other:?} for function {name}", ))), }, ColumnarValue::Scalar(scalar) => match scalar { @@ -115,8 +114,7 @@ where Ok(ColumnarValue::Scalar(S::scalar(result))) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function {}", - other, name + "Unsupported data type {other:?} for function {name}" ))), }, } @@ -219,7 +217,7 @@ fn quarter_month(date: &NaiveDateTime) -> u32 { fn date_trunc_single(granularity: &str, value: i64) -> Result { let value = timestamp_ns_to_datetime(value) .ok_or_else(|| { - DataFusionError::Execution(format!("Timestamp {} out of range", value)) + DataFusionError::Execution(format!("Timestamp {value} out of range")) })? .with_nanosecond(0); let value = match granularity { @@ -256,8 +254,7 @@ fn date_trunc_single(granularity: &str, value: i64) -> Result { .and_then(|d| d.with_month0(0)), unsupported => { return Err(DataFusionError::Execution(format!( - "Unsupported date_trunc granularity: {}", - unsupported + "Unsupported date_trunc granularity: {unsupported}" ))); } }; @@ -482,8 +479,7 @@ pub fn date_part(args: &[ColumnarValue]) -> Result { "microsecond" => extract_date_part!(&array, micros), "nanosecond" => extract_date_part!(&array, nanos), _ => Err(DataFusionError::Execution(format!( - "Date part '{}' not supported", - date_part + "Date part '{date_part}' not supported" ))), }?; @@ -670,7 +666,7 @@ mod tests { let left = string_to_timestamp_nanos(original).unwrap(); let right = string_to_timestamp_nanos(expected).unwrap(); let result = date_trunc_single(granularity, left).unwrap(); - assert_eq!(result, right, "{} = {}", original, expected); + assert_eq!(result, right, "{original} = {expected}"); }); } @@ -730,7 +726,7 @@ mod tests { let expected1 = string_to_timestamp_nanos(expected).unwrap(); let result = date_bin_single(stride1, source1, origin1); - assert_eq!(result, expected1, "{} = {}", source, expected); + assert_eq!(result, expected1, "{source} = {expected}"); }) } @@ -862,9 +858,7 @@ mod tests { Err(e) => { assert!( e.to_string().contains(expected_err), - "Can not find expected error '{}'. Actual error '{}'", - expected_err, - e + "Can not find expected error '{expected_err}'. Actual error '{e}'" ); } } diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index d88fa07c5bea..82e675425024 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -1140,8 +1140,7 @@ pub fn binary( let rhs_type = &rhs.data_type(input_schema)?; if !lhs_type.eq(rhs_type) { return Err(DataFusionError::Internal(format!( - "The type of {} {} {} of binary physical should be same", - lhs_type, op, rhs_type + "The type of {lhs_type} {op} {rhs_type} of binary physical should be same" ))); } Ok(Arc::new(BinaryExpr::new(lhs, op, rhs))) @@ -1230,7 +1229,7 @@ mod tests { let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)])?; - assert_eq!("a@0 < b@1 OR a@0 = b@1", format!("{}", expr)); + assert_eq!("a@0 < b@1 OR a@0 = b@1", format!("{expr}")); let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); assert_eq!(result.len(), 5); diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index f499f5ebf905..c4150e6cb0d7 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -64,13 +64,13 @@ impl std::fmt::Display for CaseExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "CASE ")?; if let Some(e) = &self.expr { - write!(f, "{} ", e)?; + write!(f, "{e} ")?; } for (w, t) in &self.when_then_expr { - write!(f, "WHEN {} THEN {} ", w, t)?; + write!(f, "WHEN {w} THEN {t} ")?; } if let Some(e) = &self.else_expr { - write!(f, "ELSE {} ", e)?; + write!(f, "ELSE {e} ")?; } write!(f, "END") } @@ -915,8 +915,7 @@ mod tests { get_case_common_type(&when_thens, else_expr.clone(), input_schema); let (when_thens, else_expr) = match coerce_type { None => Err(DataFusionError::Plan(format!( - "Can't get a common type for then {:?} and else {:?} expression", - when_thens, else_expr + "Can't get a common type for then {when_thens:?} and else {else_expr:?} expression" ))), Some(data_type) => { // cast then expr diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index d0904da1023f..4d01131353d8 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -164,8 +164,7 @@ pub fn cast_with_options( Ok(Arc::new(CastExpr::new(expr, cast_type, cast_options))) } else { Err(DataFusionError::NotImplemented(format!( - "Unsupported CAST from {:?} to {:?}", - expr_type, cast_type + "Unsupported CAST from {expr_type:?} to {cast_type:?}" ))) } } diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs index edbe6e27c6fd..eb2be5ef217c 100644 --- a/datafusion/physical-expr/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -225,7 +225,7 @@ mod test { assert_eq!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \ but input schema only has 1 columns: [\"foo\"]. This was likely caused by a bug in \ DataFusion's code and we would welcome that you file an bug report in our issue tracker", - &format!("{}", error)) + &format!("{error}")) } #[test] @@ -236,7 +236,7 @@ mod test { assert_eq!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \ but input schema only has 1 columns: [\"foo\"]. This was likely caused by a bug in \ DataFusion's code and we would welcome that you file an bug report in our issue tracker", - &format!("{}", error)) + &format!("{error}")) } #[test] @@ -249,7 +249,7 @@ mod test { assert_eq!("Internal error: PhysicalExpr Column references column 'id' at index 9 (zero-based) \ but input schema only has 1 columns: [\"foo\"]. This was likely caused by a bug in \ DataFusion's code and we would welcome that you file an bug report in our issue tracker", - &format!("{}", error)); + &format!("{error}")); Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/datetime.rs b/datafusion/physical-expr/src/expressions/datetime.rs index a43eed090821..655cb07f03b1 100644 --- a/datafusion/physical-expr/src/expressions/datetime.rs +++ b/datafusion/physical-expr/src/expressions/datetime.rs @@ -70,19 +70,16 @@ impl DateTimeIntervalExpr { input_schema: input_schema.clone(), }), _ => Err(DataFusionError::Execution(format!( - "Invalid operator '{}' for DateIntervalExpr", - op + "Invalid operator '{op}' for DateIntervalExpr" ))), }, other => Err(DataFusionError::Execution(format!( - "Operation '{}' not support for type {}", - op, other + "Operation '{op}' not support for type {other}" ))), } } other => Err(DataFusionError::Execution(format!( - "Invalid lhs type '{}' for DateIntervalExpr", - other + "Invalid lhs type '{other}' for DateIntervalExpr" ))), } } @@ -259,42 +256,42 @@ mod tests { fn add_11_months() { let prior = NaiveDate::from_ymd_opt(2000, 1, 1).unwrap(); let actual = shift_months(prior, 11); - assert_eq!(format!("{:?}", actual).as_str(), "2000-12-01"); + assert_eq!(format!("{actual:?}").as_str(), "2000-12-01"); } #[test] fn add_12_months() { let prior = NaiveDate::from_ymd_opt(2000, 1, 1).unwrap(); let actual = shift_months(prior, 12); - assert_eq!(format!("{:?}", actual).as_str(), "2001-01-01"); + assert_eq!(format!("{actual:?}").as_str(), "2001-01-01"); } #[test] fn add_13_months() { let prior = NaiveDate::from_ymd_opt(2000, 1, 1).unwrap(); let actual = shift_months(prior, 13); - assert_eq!(format!("{:?}", actual).as_str(), "2001-02-01"); + assert_eq!(format!("{actual:?}").as_str(), "2001-02-01"); } #[test] fn sub_11_months() { let prior = NaiveDate::from_ymd_opt(2000, 1, 1).unwrap(); let actual = shift_months(prior, -11); - assert_eq!(format!("{:?}", actual).as_str(), "1999-02-01"); + assert_eq!(format!("{actual:?}").as_str(), "1999-02-01"); } #[test] fn sub_12_months() { let prior = NaiveDate::from_ymd_opt(2000, 1, 1).unwrap(); let actual = shift_months(prior, -12); - assert_eq!(format!("{:?}", actual).as_str(), "1999-01-01"); + assert_eq!(format!("{actual:?}").as_str(), "1999-01-01"); } #[test] fn sub_13_months() { let prior = NaiveDate::from_ymd_opt(2000, 1, 1).unwrap(); let actual = shift_months(prior, -13); - assert_eq!(format!("{:?}", actual).as_str(), "1998-12-01"); + assert_eq!(format!("{actual:?}").as_str(), "1998-12-01"); } #[test] @@ -312,7 +309,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Date32(Some(d))) => { let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); let res = epoch.add(Duration::days(d as i64)); - assert_eq!(format!("{:?}", res).as_str(), "1970-01-02"); + assert_eq!(format!("{res:?}").as_str(), "1970-01-02"); } _ => Err(DataFusionError::NotImplemented( "Unexpected result!".to_string(), @@ -337,7 +334,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Date32(Some(d))) => { let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); let res = epoch.add(Duration::days(d as i64)); - assert_eq!(format!("{:?}", res).as_str(), "1968-12-01"); + assert_eq!(format!("{res:?}").as_str(), "1968-12-01"); } _ => Err(DataFusionError::NotImplemented( "Unexpected result!".to_string(), @@ -363,7 +360,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Date64(Some(d))) => { let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); let res = epoch.add(Duration::milliseconds(d)); - assert_eq!(format!("{:?}", res).as_str(), "1969-12-16"); + assert_eq!(format!("{res:?}").as_str(), "1969-12-16"); } _ => Err(DataFusionError::NotImplemented( "Unexpected result!".to_string(), @@ -388,7 +385,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Date32(Some(d))) => { let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); let res = epoch.add(Duration::days(d as i64)); - assert_eq!(format!("{:?}", res).as_str(), "1970-02-01"); + assert_eq!(format!("{res:?}").as_str(), "1970-02-01"); } _ => Err(DataFusionError::NotImplemented( "Unexpected result!".to_string(), @@ -413,7 +410,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Date32(Some(d))) => { let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); let res = epoch.add(Duration::days(d as i64)); - assert_eq!(format!("{:?}", res).as_str(), "1968-12-17"); + assert_eq!(format!("{res:?}").as_str(), "1968-12-17"); } _ => Err(DataFusionError::NotImplemented( "Unexpected result!".to_string(), @@ -631,8 +628,8 @@ mod tests { let lhs = create_physical_expr(dt, &dfs, &schema, &props)?; let rhs = create_physical_expr(interval, &dfs, &schema, &props)?; - let lhs_str = format!("{}", lhs); - let rhs_str = format!("{}", rhs); + let lhs_str = format!("{lhs}"); + let rhs_str = format!("{rhs}"); let cut = DateTimeIntervalExpr::try_new(lhs, op, rhs, &schema)?; diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index 8fbb68d61a9a..9665a82728df 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -123,13 +123,13 @@ impl PhysicalExpr for GetIndexedFieldExpr { (DataType::Struct(_), ScalarValue::Utf8(Some(k))) => { let as_struct_array = as_struct_array(&array)?; match as_struct_array.column_by_name(k) { - None => Err(DataFusionError::Execution(format!("get indexed field {} not found in struct", k))), + None => Err(DataFusionError::Execution(format!("get indexed field {k} not found in struct"))), Some(col) => Ok(ColumnarValue::Array(col.clone())) } } - (DataType::List(_), key) => Err(DataFusionError::Execution(format!("get indexed field is only possible on lists with int64 indexes. Tried with {:?} index", key))), - (DataType::Struct(_), key) => Err(DataFusionError::Execution(format!("get indexed field is only possible on struct with utf8 indexes. Tried with {:?} index", key))), - (dt, key) => Err(DataFusionError::Execution(format!("get indexed field is only possible on lists with int64 indexes or struct with utf8 indexes. Tried {:?} with {:?} index", dt, key))), + (DataType::List(_), key) => Err(DataFusionError::Execution(format!("get indexed field is only possible on lists with int64 indexes. Tried with {key:?} index"))), + (DataType::Struct(_), key) => Err(DataFusionError::Execution(format!("get indexed field is only possible on struct with utf8 indexes. Tried with {key:?} index"))), + (dt, key) => Err(DataFusionError::Execution(format!("get indexed field is only possible on lists with int64 indexes or struct with utf8 indexes. Tried {dt:?} with {key:?} index"))), } } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 10efb8c3027b..fc4e102fefcc 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -204,7 +204,7 @@ fn make_set(array: &dyn Array) -> Result> { Box::new(ArraySet::new(array, make_hash_set(array))) } DataType::Dictionary(_, _) => unreachable!("dictionary should have been flattened"), - d => return Err(DataFusionError::NotImplemented(format!("DataType::{} not supported in InList", d))) + d => return Err(DataFusionError::NotImplemented(format!("DataType::{d} not supported in InList"))) }) } @@ -360,8 +360,7 @@ pub fn in_list( let list_expr_data_type = list_expr.data_type(schema)?; if !expr_data_type.eq(&list_expr_data_type) { return Err(DataFusionError::Internal(format!( - "The data type inlist should be same, the value type is {}, one of list expr type is {}", - expr_data_type, list_expr_data_type + "The data type inlist should be same, the value type is {expr_data_type}, one of list expr type is {list_expr_data_type}" ))); } } @@ -395,8 +394,7 @@ mod tests { let result_type = get_coerce_type(expr_type, &list_types); match result_type { None => Err(DataFusionError::Plan(format!( - "Can not find compatible types to compare {:?} with {:?}", - expr_type, list_types + "Can not find compatible types to compare {expr_type:?} with {list_types:?}" ))), Some(data_type) => { // find the coerced type diff --git a/datafusion/physical-expr/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs index ea6f51027575..403768aa39e7 100644 --- a/datafusion/physical-expr/src/expressions/literal.rs +++ b/datafusion/physical-expr/src/expressions/literal.rs @@ -129,7 +129,7 @@ mod tests { // create and evaluate a literal expression let literal_expr = lit(42i32); - assert_eq!("42", format!("{}", literal_expr)); + assert_eq!("42", format!("{literal_expr}")); let literal_array = literal_expr.evaluate(&batch)?.into_array(batch.num_rows()); let literal_array = as_int32_array(&literal_array)?; diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index fc91d91cf612..8294c0ef7ace 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -91,7 +91,7 @@ pub use try_cast::{try_cast, TryCastExpr}; /// returns the name of the state pub fn format_state_name(name: &str, state_name: &str) -> String { - format!("{}[{}]", name, state_name) + format!("{name}[{state_name}]") } pub use crate::PhysicalSortExpr; diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 1fed005ce3b2..91e07619b107 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -147,7 +147,7 @@ pub fn negative( Ok(arg) } else if !is_signed_numeric(&data_type) { Err(DataFusionError::Internal( - format!("Can't create negative physical expr for (- '{:?}'), the type of child expr is {}, not signed numeric", arg, data_type), + format!("Can't create negative physical expr for (- '{arg:?}'), the type of child expr is {data_type}, not signed numeric"), )) } else { Ok(Arc::new(NegativeExpr::new(arg))) diff --git a/datafusion/physical-expr/src/expressions/try_cast.rs b/datafusion/physical-expr/src/expressions/try_cast.rs index 9fa6fc0d9b3d..ce5f1c2794b5 100644 --- a/datafusion/physical-expr/src/expressions/try_cast.rs +++ b/datafusion/physical-expr/src/expressions/try_cast.rs @@ -132,8 +132,7 @@ pub fn try_cast( Ok(Arc::new(TryCastExpr::new(expr, cast_type))) } else { Err(DataFusionError::NotImplemented(format!( - "Unsupported CAST from {:?} to {:?}", - expr_type, cast_type + "Unsupported CAST from {expr_type:?} to {cast_type:?}" ))) } } diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 81a6b3af154e..c72ab161ffbd 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -83,8 +83,7 @@ pub fn create_physical_expr( Ok(DataType::Utf8) => datetime_expressions::to_timestamp, other => { return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp", - other, + "Unsupported data type {other:?} for function to_timestamp", ))); } }) @@ -103,8 +102,7 @@ pub fn create_physical_expr( Ok(DataType::Utf8) => datetime_expressions::to_timestamp_millis, other => { return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp_millis", - other, + "Unsupported data type {other:?} for function to_timestamp_millis", ))); } }) @@ -123,8 +121,7 @@ pub fn create_physical_expr( Ok(DataType::Utf8) => datetime_expressions::to_timestamp_micros, other => { return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp_micros", - other, + "Unsupported data type {other:?} for function to_timestamp_micros", ))); } }) @@ -143,8 +140,7 @@ pub fn create_physical_expr( Ok(DataType::Utf8) => datetime_expressions::to_timestamp_seconds, other => { return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp_seconds", - other, + "Unsupported data type {other:?} for function to_timestamp_seconds", ))); } } @@ -160,8 +156,7 @@ pub fn create_physical_expr( }, other => { return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function from_unixtime", - other, + "Unsupported data type {other:?} for function from_unixtime", ))); } } @@ -170,8 +165,7 @@ pub fn create_physical_expr( let input_data_type = input_phy_exprs[0].data_type(input_schema)?; Arc::new(move |_| { Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(format!( - "{}", - input_data_type + "{input_data_type}" ))))) }) } @@ -180,7 +174,7 @@ pub fn create_physical_expr( }; Ok(Arc::new(ScalarFunctionExpr::new( - &format!("{}", fun), + &format!("{fun}"), fun_expr, input_phy_exprs.to_vec(), &data_type, @@ -358,8 +352,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::ascii::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function ascii", - other, + "Unsupported data type {other:?} for function ascii", ))), }), BuiltinScalarFunction::BitLength => Arc::new(|args| match &args[0] { @@ -382,8 +375,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::btrim::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function btrim", - other, + "Unsupported data type {other:?} for function btrim", ))), }), BuiltinScalarFunction::CharacterLength => { @@ -405,8 +397,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function character_length", - other, + "Unsupported data type {other:?} for function character_length", ))), }) } @@ -447,8 +438,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::initcap::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function initcap", - other, + "Unsupported data type {other:?} for function initcap", ))), }), BuiltinScalarFunction::Left => Arc::new(|args| match args[0].data_type() { @@ -461,8 +451,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function left", - other, + "Unsupported data type {other:?} for function left", ))), }), BuiltinScalarFunction::Lower => Arc::new(string_expressions::lower), @@ -476,8 +465,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function lpad", - other, + "Unsupported data type {other:?} for function lpad", ))), }), BuiltinScalarFunction::Ltrim => Arc::new(|args| match args[0].data_type() { @@ -488,8 +476,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::ltrim::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function ltrim", - other, + "Unsupported data type {other:?} for function ltrim", ))), }), BuiltinScalarFunction::MD5 => { @@ -530,8 +517,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function regexp_match", - other + "Unsupported data type {other:?} for function regexp_match" ))), }) } @@ -556,8 +542,7 @@ pub fn create_physical_fun( func(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function regexp_replace", - other, + "Unsupported data type {other:?} for function regexp_replace", ))), }) } @@ -569,8 +554,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::repeat::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function repeat", - other, + "Unsupported data type {other:?} for function repeat", ))), }), BuiltinScalarFunction::Replace => Arc::new(|args| match args[0].data_type() { @@ -581,8 +565,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::replace::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function replace", - other, + "Unsupported data type {other:?} for function replace", ))), }), BuiltinScalarFunction::Reverse => Arc::new(|args| match args[0].data_type() { @@ -597,8 +580,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function reverse", - other, + "Unsupported data type {other:?} for function reverse", ))), }), BuiltinScalarFunction::Right => Arc::new(|args| match args[0].data_type() { @@ -613,8 +595,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function right", - other, + "Unsupported data type {other:?} for function right", ))), }), BuiltinScalarFunction::Rpad => Arc::new(|args| match args[0].data_type() { @@ -627,8 +608,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function rpad", - other, + "Unsupported data type {other:?} for function rpad", ))), }), BuiltinScalarFunction::Rtrim => Arc::new(|args| match args[0].data_type() { @@ -639,8 +619,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::rtrim::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function rtrim", - other, + "Unsupported data type {other:?} for function rtrim", ))), }), BuiltinScalarFunction::SHA224 => { @@ -663,8 +642,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::split_part::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function split_part", - other, + "Unsupported data type {other:?} for function split_part", ))), }), BuiltinScalarFunction::StartsWith => Arc::new(|args| match args[0].data_type() { @@ -675,8 +653,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::starts_with::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function starts_with", - other, + "Unsupported data type {other:?} for function starts_with", ))), }), BuiltinScalarFunction::Strpos => Arc::new(|args| match args[0].data_type() { @@ -693,8 +670,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function strpos", - other, + "Unsupported data type {other:?} for function strpos", ))), }), BuiltinScalarFunction::Substr => Arc::new(|args| match args[0].data_type() { @@ -709,8 +685,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function substr", - other, + "Unsupported data type {other:?} for function substr", ))), }), BuiltinScalarFunction::ToHex => Arc::new(|args| match args[0].data_type() { @@ -721,8 +696,7 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::to_hex::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_hex", - other, + "Unsupported data type {other:?} for function to_hex", ))), }), BuiltinScalarFunction::Translate => Arc::new(|args| match args[0].data_type() { @@ -743,8 +717,7 @@ pub fn create_physical_fun( make_scalar_function(func)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function translate", - other, + "Unsupported data type {other:?} for function translate", ))), }), BuiltinScalarFunction::Trim => Arc::new(|args| match args[0].data_type() { @@ -755,16 +728,14 @@ pub fn create_physical_fun( make_scalar_function(string_expressions::btrim::)(args) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function trim", - other, + "Unsupported data type {other:?} for function trim", ))), }), BuiltinScalarFunction::Upper => Arc::new(string_expressions::upper), BuiltinScalarFunction::Uuid => Arc::new(string_expressions::uuid), _ => { return Err(DataFusionError::Internal(format!( - "create_physical_fun: Unsupported scalar function {:?}", - fun + "create_physical_fun: Unsupported scalar function {fun:?}" ))); } }) @@ -2733,24 +2704,21 @@ mod tests { match expr { Ok(..) => { return Err(DataFusionError::Plan(format!( - "Builtin scalar function {} does not support empty arguments", - fun + "Builtin scalar function {fun} does not support empty arguments" ))); } Err(DataFusionError::Internal(err)) => { if err != format!( - "Builtin scalar function {} does not support empty arguments", - fun - ) - { + "Builtin scalar function {fun} does not support empty arguments" + ) { return Err(DataFusionError::Internal(format!( - "Builtin scalar function {} didn't got the right error message with empty arguments", fun))); + "Builtin scalar function {fun} didn't got the right error message with empty arguments"))); } } Err(..) => { return Err(DataFusionError::Internal(format!( - "Builtin scalar function {} didn't got the right error with empty arguments", fun))); + "Builtin scalar function {fun} didn't got the right error with empty arguments"))); } } } diff --git a/datafusion/physical-expr/src/math_expressions.rs b/datafusion/physical-expr/src/math_expressions.rs index edf2b98bae80..afaff49b8e60 100644 --- a/datafusion/physical-expr/src/math_expressions.rs +++ b/datafusion/physical-expr/src/math_expressions.rs @@ -170,8 +170,7 @@ pub fn power(args: &[ArrayRef]) -> Result { )) as ArrayRef), other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function power", - other + "Unsupported data type {other:?} for function power" ))), } } @@ -197,8 +196,7 @@ pub fn atan2(args: &[ArrayRef]) -> Result { )) as ArrayRef), other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function atan2", - other + "Unsupported data type {other:?} for function atan2" ))), } } diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 98955c6ac322..2be6c1f33b99 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -440,8 +440,7 @@ pub fn create_physical_expr( } }, other => Err(DataFusionError::NotImplemented(format!( - "Physical plan does not support logical expression {:?}", - other + "Physical plan does not support logical expression {other:?}" ))), } } diff --git a/datafusion/physical-expr/src/regex_expressions.rs b/datafusion/physical-expr/src/regex_expressions.rs index 68c2c20e7962..c5edf320fc0e 100644 --- a/datafusion/physical-expr/src/regex_expressions.rs +++ b/datafusion/physical-expr/src/regex_expressions.rs @@ -71,8 +71,7 @@ pub fn regexp_match(args: &[ArrayRef]) -> Result { } } other => Err(DataFusionError::Internal(format!( - "regexp_match was called with {} arguments. It requires at least 2 and at most 3.", - other + "regexp_match was called with {other} arguments. It requires at least 2 and at most 3." ))), } } @@ -155,7 +154,7 @@ pub fn regexp_replace(args: &[ArrayRef]) -> Result } else if flags.contains('g') { (format!("(?{}){}", flags.to_string().replace('g', ""), pattern), true) } else { - (format!("(?{}){}", flags, pattern), false) + (format!("(?{flags}){pattern}"), false) }; // if patterns hashmap already has regexp then use else else create and return @@ -187,8 +186,7 @@ pub fn regexp_replace(args: &[ArrayRef]) -> Result Ok(Arc::new(result) as ArrayRef) } other => Err(DataFusionError::Internal(format!( - "regexp_replace was called with {} arguments. It requires at least 3 and at most 4.", - other + "regexp_replace was called with {other} arguments. It requires at least 3 and at most 4." ))), } } @@ -221,8 +219,7 @@ fn _regexp_replace_static_pattern_replace( 4 => Some(fetch_string_arg!(&args[3], "flags", T, _regexp_replace_early_abort)), other => { return Err(DataFusionError::Internal(format!( - "regexp_replace was called with {} arguments. It requires at least 3 and at most 4.", - other + "regexp_replace was called with {other} arguments. It requires at least 3 and at most 4." ))) } }; diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index cdcaf42b2458..da47a55aa9e3 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -107,7 +107,7 @@ impl fmt::Display for ScalarFunctionExpr { self.name, self.args .iter() - .map(|e| format!("{}", e)) + .map(|e| format!("{e}")) .collect::>() .join(", ") ) diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index a173cc8ba8b3..f8172dabf65a 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -60,8 +60,7 @@ impl PhysicalSortExpr { ColumnarValue::Array(array) => array, ColumnarValue::Scalar(scalar) => { return Err(DataFusionError::Plan(format!( - "Sort operation is not applicable to scalar value {}", - scalar + "Sort operation is not applicable to scalar value {scalar}" ))); } }; diff --git a/datafusion/physical-expr/src/string_expressions.rs b/datafusion/physical-expr/src/string_expressions.rs index 7048354f12d6..93d4d86da532 100644 --- a/datafusion/physical-expr/src/string_expressions.rs +++ b/datafusion/physical-expr/src/string_expressions.rs @@ -99,8 +99,7 @@ where )?))) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function {}", - other, name, + "Unsupported data type {other:?} for function {name}", ))), }, ColumnarValue::Scalar(scalar) => match scalar { @@ -113,8 +112,7 @@ where Ok(ColumnarValue::Scalar(ScalarValue::LargeUtf8(result))) } other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function {}", - other, name, + "Unsupported data type {other:?} for function {name}", ))), }, } @@ -180,8 +178,7 @@ pub fn btrim(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } other => Err(DataFusionError::Internal(format!( - "btrim was called with {} arguments. It requires at least 1 and at most 2.", - other + "btrim was called with {other} arguments. It requires at least 1 and at most 2." ))), } } @@ -337,9 +334,9 @@ pub fn initcap(args: &[ArrayRef]) -> Result { } else { char_vector.push(c.to_ascii_uppercase()); } - previous_character_letter_or_number = ('A'..='Z').contains(&c) - || ('a'..='z').contains(&c) - || ('0'..='9').contains(&c); + previous_character_letter_or_number = c.is_ascii_uppercase() + || c.is_ascii_lowercase() + || c.is_ascii_digit(); } char_vector.iter().collect::() }) @@ -388,8 +385,7 @@ pub fn ltrim(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } other => Err(DataFusionError::Internal(format!( - "ltrim was called with {} arguments. It requires at least 1 and at most 2.", - other + "ltrim was called with {other} arguments. It requires at least 1 and at most 2." ))), } } @@ -465,8 +461,7 @@ pub fn rtrim(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } other => Err(DataFusionError::Internal(format!( - "rtrim was called with {} arguments. It requires at least 1 and at most 2.", - other + "rtrim was called with {other} arguments. It requires at least 1 and at most 2." ))), } } @@ -533,13 +528,12 @@ where .map(|integer| { if let Some(value) = integer { if let Some(value_usize) = value.to_usize() { - Ok(Some(format!("{:x}", value_usize))) + Ok(Some(format!("{value_usize:x}"))) } else if let Some(value_isize) = value.to_isize() { - Ok(Some(format!("{:x}", value_isize))) + Ok(Some(format!("{value_isize:x}"))) } else { Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_hex", - integer, + "Unsupported data type {integer:?} for function to_hex", ))) } } else { diff --git a/datafusion/physical-expr/src/struct_expressions.rs b/datafusion/physical-expr/src/struct_expressions.rs index c749038a82e7..d03f94a49791 100644 --- a/datafusion/physical-expr/src/struct_expressions.rs +++ b/datafusion/physical-expr/src/struct_expressions.rs @@ -35,7 +35,7 @@ fn array_struct(args: &[ArrayRef]) -> Result { .iter() .enumerate() .map(|(i, arg)| -> Result<(Field, ArrayRef)> { - let field_name = format!("c{}", i); + let field_name = format!("c{i}"); match arg.data_type() { DataType::Utf8 | DataType::LargeUtf8 @@ -54,8 +54,7 @@ fn array_struct(args: &[ArrayRef]) -> Result { arg.clone(), )), data_type => Err(DataFusionError::NotImplemented(format!( - "Struct is not implemented for type '{:?}'.", - data_type + "Struct is not implemented for type '{data_type:?}'." ))), } }) diff --git a/datafusion/physical-expr/src/type_coercion.rs b/datafusion/physical-expr/src/type_coercion.rs index 7b8d2c40a1ce..8e55b739f373 100644 --- a/datafusion/physical-expr/src/type_coercion.rs +++ b/datafusion/physical-expr/src/type_coercion.rs @@ -78,7 +78,7 @@ mod tests { Schema::new( t.iter() .enumerate() - .map(|(i, t)| Field::new(format!("c{}", i), t.clone(), true)) + .map(|(i, t)| Field::new(format!("c{i}"), t.clone(), true)) .collect(), ) }; @@ -88,7 +88,7 @@ mod tests { t.iter() .enumerate() .map(|(i, t)| { - try_cast(col(&format!("c{}", i), &schema)?, &schema, t.clone()) + try_cast(col(&format!("c{i}"), &schema)?, &schema, t.clone()) }) .collect::>>() }; @@ -190,8 +190,7 @@ mod tests { for case in cases { if coerce(&case.0, &case.1, &case.2).is_ok() { return Err(DataFusionError::Plan(format!( - "Error was expected in {:?}", - case + "Error was expected in {case:?}" ))); } } diff --git a/datafusion/physical-expr/src/unicode_expressions.rs b/datafusion/physical-expr/src/unicode_expressions.rs index a89cfd8d156b..6654904cf1b7 100644 --- a/datafusion/physical-expr/src/unicode_expressions.rs +++ b/datafusion/physical-expr/src/unicode_expressions.rs @@ -103,8 +103,7 @@ pub fn lpad(args: &[ArrayRef]) -> Result { (Some(string), Some(length)) => { if length > i32::MAX as i64 { return Err(DataFusionError::Internal(format!( - "lpad requested length {} too large", - length + "lpad requested length {length} too large" ))); } @@ -141,8 +140,7 @@ pub fn lpad(args: &[ArrayRef]) -> Result { (Some(string), Some(length), Some(fill)) => { if length > i32::MAX as i64 { return Err(DataFusionError::Internal(format!( - "lpad requested length {} too large", - length + "lpad requested length {length} too large" ))); } @@ -181,8 +179,7 @@ pub fn lpad(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } other => Err(DataFusionError::Internal(format!( - "lpad was called with {} arguments. It requires at least 2 and at most 3.", - other + "lpad was called with {other} arguments. It requires at least 2 and at most 3." ))), } } @@ -249,8 +246,7 @@ pub fn rpad(args: &[ArrayRef]) -> Result { (Some(string), Some(length)) => { if length > i32::MAX as i64 { return Err(DataFusionError::Internal(format!( - "rpad requested length {} too large", - length + "rpad requested length {length} too large" ))); } @@ -286,8 +282,7 @@ pub fn rpad(args: &[ArrayRef]) -> Result { (Some(string), Some(length), Some(fill)) => { if length > i32::MAX as i64 { return Err(DataFusionError::Internal(format!( - "rpad requested length {} too large", - length + "rpad requested length {length} too large" ))); } @@ -318,8 +313,7 @@ pub fn rpad(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } other => Err(DataFusionError::Internal(format!( - "rpad was called with {} arguments. It requires at least 2 and at most 3.", - other + "rpad was called with {other} arguments. It requires at least 2 and at most 3." ))), } } @@ -398,9 +392,7 @@ pub fn substr(args: &[ArrayRef]) -> Result { (Some(string), Some(start), Some(count)) => { if count < 0 { Err(DataFusionError::Execution(format!( - "negative substring length not allowed: substr(, {}, {})", - start, - count + "negative substring length not allowed: substr(, {start}, {count})" ))) } else { let skip = max(0, start - 1); @@ -415,8 +407,7 @@ pub fn substr(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } other => Err(DataFusionError::Internal(format!( - "substr was called with {} arguments. It requires 2 or 3.", - other + "substr was called with {other} arguments. It requires 2 or 3." ))), } } diff --git a/datafusion/physical-expr/src/window/window_frame_state.rs b/datafusion/physical-expr/src/window/window_frame_state.rs index b49bd3a22a78..9c559cabd170 100644 --- a/datafusion/physical-expr/src/window/window_frame_state.rs +++ b/datafusion/physical-expr/src/window/window_frame_state.rs @@ -115,8 +115,7 @@ impl<'a> WindowFrameContext<'a> { // UNBOUNDED FOLLOWING WindowFrameBound::Following(ScalarValue::UInt64(None)) => { return Err(DataFusionError::Internal(format!( - "Frame start cannot be UNBOUNDED FOLLOWING '{:?}'", - window_frame + "Frame start cannot be UNBOUNDED FOLLOWING '{window_frame:?}'" ))) } WindowFrameBound::Following(ScalarValue::UInt64(Some(n))) => { @@ -131,8 +130,7 @@ impl<'a> WindowFrameContext<'a> { // UNBOUNDED PRECEDING WindowFrameBound::Preceding(ScalarValue::UInt64(None)) => { return Err(DataFusionError::Internal(format!( - "Frame end cannot be UNBOUNDED PRECEDING '{:?}'", - window_frame + "Frame end cannot be UNBOUNDED PRECEDING '{window_frame:?}'" ))) } WindowFrameBound::Preceding(ScalarValue::UInt64(Some(n))) => { @@ -356,8 +354,7 @@ impl WindowFrameStateGroups { // UNBOUNDED FOLLOWING WindowFrameBound::Following(ScalarValue::UInt64(None)) => { return Err(DataFusionError::Internal(format!( - "Frame start cannot be UNBOUNDED FOLLOWING '{:?}'", - window_frame + "Frame start cannot be UNBOUNDED FOLLOWING '{window_frame:?}'" ))) } // ERRONEOUS FRAMES @@ -371,8 +368,7 @@ impl WindowFrameStateGroups { // UNBOUNDED PRECEDING WindowFrameBound::Preceding(ScalarValue::UInt64(None)) => { return Err(DataFusionError::Internal(format!( - "Frame end cannot be UNBOUNDED PRECEDING '{:?}'", - window_frame + "Frame end cannot be UNBOUNDED PRECEDING '{window_frame:?}'" ))) } WindowFrameBound::Preceding(ScalarValue::UInt64(Some(n))) => self diff --git a/datafusion/proto/build.rs b/datafusion/proto/build.rs index c9b4e25fde3e..7d3582d2b224 100644 --- a/datafusion/proto/build.rs +++ b/datafusion/proto/build.rs @@ -45,7 +45,7 @@ fn build() -> Result<(), String> { .compile_well_known_types() .extern_path(".google.protobuf", "::pbjson_types") .compile_protos(&["proto/datafusion.proto"], &["proto"]) - .map_err(|e| format!("protobuf compilation failed: {}", e))?; + .map_err(|e| format!("protobuf compilation failed: {e}"))?; let descriptor_set = std::fs::read(&descriptor_path) .unwrap_or_else(|e| panic!("Cannot read {:?}: {}", &descriptor_path, e)); @@ -56,7 +56,7 @@ fn build() -> Result<(), String> { panic!("Cannot register descriptors {:?}: {}", &descriptor_set, e) }) .build(&[".datafusion"]) - .map_err(|e| format!("pbjson compilation failed: {}", e))?; + .map_err(|e| format!("pbjson compilation failed: {e}"))?; let prost = out.join("datafusion.rs"); let pbjson = out.join("datafusion.serde.rs"); diff --git a/datafusion/proto/examples/logical_plan_serde.rs b/datafusion/proto/examples/logical_plan_serde.rs index fb8a31c05705..0f8312372778 100644 --- a/datafusion/proto/examples/logical_plan_serde.rs +++ b/datafusion/proto/examples/logical_plan_serde.rs @@ -27,6 +27,6 @@ async fn main() -> Result<()> { let plan = ctx.table("t1")?.into_optimized_plan()?; let bytes = logical_plan_to_bytes(&plan)?; let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; - assert_eq!(format!("{:?}", plan), format!("{:?}", logical_round_trip)); + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); Ok(()) } diff --git a/datafusion/proto/examples/physical_plan_serde.rs b/datafusion/proto/examples/physical_plan_serde.rs index decdad55bc5c..803b9e3186a4 100644 --- a/datafusion/proto/examples/physical_plan_serde.rs +++ b/datafusion/proto/examples/physical_plan_serde.rs @@ -29,8 +29,8 @@ async fn main() -> Result<()> { let bytes = physical_plan_to_bytes(physical_plan.clone())?; let physical_round_trip = physical_plan_from_bytes(&bytes, &ctx)?; assert_eq!( - format!("{:?}", physical_plan), - format!("{:?}", physical_round_trip) + format!("{physical_plan:?}"), + format!("{physical_round_trip:?}") ); Ok(()) } diff --git a/datafusion/proto/src/bytes/mod.rs b/datafusion/proto/src/bytes/mod.rs index 6163efb1f109..a0b678f63272 100644 --- a/datafusion/proto/src/bytes/mod.rs +++ b/datafusion/proto/src/bytes/mod.rs @@ -81,11 +81,11 @@ impl Serializeable for Expr { fn to_bytes(&self) -> Result { let mut buffer = BytesMut::new(); let protobuf: protobuf::LogicalExprNode = self.try_into().map_err(|e| { - DataFusionError::Plan(format!("Error encoding expr as protobuf: {}", e)) + DataFusionError::Plan(format!("Error encoding expr as protobuf: {e}")) })?; protobuf.encode(&mut buffer).map_err(|e| { - DataFusionError::Plan(format!("Error encoding protobuf as bytes: {}", e)) + DataFusionError::Plan(format!("Error encoding protobuf as bytes: {e}")) })?; let bytes: Bytes = buffer.into(); @@ -134,11 +134,11 @@ impl Serializeable for Expr { registry: &dyn FunctionRegistry, ) -> Result { let protobuf = protobuf::LogicalExprNode::decode(bytes).map_err(|e| { - DataFusionError::Plan(format!("Error decoding expr as protobuf: {}", e)) + DataFusionError::Plan(format!("Error decoding expr as protobuf: {e}")) })?; logical_plan::from_proto::parse_expr(&protobuf, registry).map_err(|e| { - DataFusionError::Plan(format!("Error parsing protobuf into Expr: {}", e)) + DataFusionError::Plan(format!("Error parsing protobuf into Expr: {e}")) }) } } @@ -171,7 +171,7 @@ pub fn logical_plan_to_bytes_with_extension_codec( protobuf::LogicalPlanNode::try_from_logical_plan(plan, extension_codec)?; let mut buffer = BytesMut::new(); protobuf.encode(&mut buffer).map_err(|e| { - DataFusionError::Plan(format!("Error encoding protobuf as bytes: {}", e)) + DataFusionError::Plan(format!("Error encoding protobuf as bytes: {e}")) })?; Ok(buffer.into()) } @@ -201,7 +201,7 @@ pub fn logical_plan_from_bytes_with_extension_codec( extension_codec: &dyn LogicalExtensionCodec, ) -> Result { let protobuf = protobuf::LogicalPlanNode::decode(bytes).map_err(|e| { - DataFusionError::Plan(format!("Error decoding expr as protobuf: {}", e)) + DataFusionError::Plan(format!("Error decoding expr as protobuf: {e}")) })?; protobuf.try_into_logical_plan(ctx, extension_codec) } @@ -234,7 +234,7 @@ pub fn physical_plan_to_bytes_with_extension_codec( protobuf::PhysicalPlanNode::try_from_physical_plan(plan, extension_codec)?; let mut buffer = BytesMut::new(); protobuf.encode(&mut buffer).map_err(|e| { - DataFusionError::Plan(format!("Error encoding protobuf as bytes: {}", e)) + DataFusionError::Plan(format!("Error encoding protobuf as bytes: {e}")) })?; Ok(buffer.into()) } @@ -267,7 +267,7 @@ pub fn physical_plan_from_bytes_with_extension_codec( extension_codec: &dyn PhysicalExtensionCodec, ) -> Result> { let protobuf = protobuf::PhysicalPlanNode::decode(bytes).map_err(|e| { - DataFusionError::Plan(format!("Error decoding expr as protobuf: {}", e)) + DataFusionError::Plan(format!("Error decoding expr as protobuf: {e}")) })?; protobuf.try_into_physical_plan(ctx, &ctx.runtime_env(), extension_codec) } diff --git a/datafusion/proto/src/bytes/registry.rs b/datafusion/proto/src/bytes/registry.rs index 675e40240a76..6c66b33a9fc4 100644 --- a/datafusion/proto/src/bytes/registry.rs +++ b/datafusion/proto/src/bytes/registry.rs @@ -32,13 +32,13 @@ impl FunctionRegistry for NoRegistry { fn udf(&self, name: &str) -> Result> { Err(DataFusionError::Plan( - format!("No function registry provided to deserialize, so can not deserialize User Defined Function '{}'", name)) + format!("No function registry provided to deserialize, so can not deserialize User Defined Function '{name}'")) ) } fn udaf(&self, name: &str) -> Result> { Err(DataFusionError::Plan( - format!("No function registry provided to deserialize, so can not deserialize User Defined Aggregate Function '{}'", name)) + format!("No function registry provided to deserialize, so can not deserialize User Defined Aggregate Function '{name}'")) ) } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 9ba61711652f..b17a9c4c1232 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -68,20 +68,20 @@ pub enum Error { impl std::fmt::Display for Error { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { - Self::General(desc) => write!(f, "General error: {}", desc), + Self::General(desc) => write!(f, "General error: {desc}"), Self::DataFusionError(desc) => { - write!(f, "DataFusion error: {:?}", desc) + write!(f, "DataFusion error: {desc:?}") } Self::MissingRequiredField(name) => { - write!(f, "Missing required field {}", name) + write!(f, "Missing required field {name}") } Self::AtLeastOneValue(name) => { - write!(f, "Must have at least one {}, found 0", name) + write!(f, "Must have at least one {name}, found 0") } Self::UnknownEnumVariant { name, value } => { - write!(f, "Unknown i32 value for {} enum: {}", name, value) + write!(f, "Unknown i32 value for {name} enum: {value}") } } } @@ -372,8 +372,7 @@ impl From<&protobuf::StringifiedPlan> for StringifiedPlan { .and_then(|pt| pt.plan_type_enum.as_ref()) .unwrap_or_else(|| { panic!( - "Cannot create protobuf::StringifiedPlan from {:?}", - stringified_plan + "Cannot create protobuf::StringifiedPlan from {stringified_plan:?}" ) }) { InitialLogicalPlan(_) => PlanType::InitialLogicalPlan, @@ -828,8 +827,7 @@ fn validate_list_values(field: &Field, values: &[ScalarValue]) -> Result<(), Err if field_type != &value_type { return Err(proto_error(format!( - "Expected field type {:?}, got scalar of type: {:?}", - field_type, value_type + "Expected field type {field_type:?}, got scalar of type: {value_type:?}" ))); } } @@ -1328,7 +1326,7 @@ pub fn parse_expr( } ExprType::Placeholder(PlaceholderNode { id, data_type }) => match data_type { None => { - let message = format!("Protobuf deserialization error: data type must be provided for the placeholder {}", id); + let message = format!("Protobuf deserialization error: data type must be provided for the placeholder {id}"); Err(proto_error(message)) } Some(data_type) => Ok(Expr::Placeholder { @@ -1389,8 +1387,7 @@ pub fn from_proto_binary_op(op: &str) -> Result { "RegexNotMatch" => Ok(Operator::RegexNotMatch), "StringConcat" => Ok(Operator::StringConcat), other => Err(proto_error(format!( - "Unsupported binary operator '{:?}'", - other + "Unsupported binary operator '{other:?}'" ))), } } diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index e44764d0f060..2a5626d510ba 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -187,8 +187,7 @@ fn from_owned_table_reference( ) -> Result { let table_ref = table_ref.ok_or_else(|| { DataFusionError::Internal(format!( - "Protobuf deserialization error, {} was missing required field name.", - error_context + "Protobuf deserialization error, {error_context} was missing required field name." )) })?; @@ -201,7 +200,7 @@ impl AsLogicalPlan for LogicalPlanNode { Self: Sized, { LogicalPlanNode::decode(buf).map_err(|e| { - DataFusionError::Internal(format!("failed to decode logical plan: {:?}", e)) + DataFusionError::Internal(format!("failed to decode logical plan: {e:?}")) }) } @@ -211,7 +210,7 @@ impl AsLogicalPlan for LogicalPlanNode { Self: Sized, { self.encode(buf).map_err(|e| { - DataFusionError::Internal(format!("failed to encode logical plan: {:?}", e)) + DataFusionError::Internal(format!("failed to encode logical plan: {e:?}")) }) } @@ -222,8 +221,7 @@ impl AsLogicalPlan for LogicalPlanNode { ) -> Result { let plan = self.logical_plan_type.as_ref().ok_or_else(|| { proto_error(format!( - "logical_plan::from_proto() Unsupported logical plan '{:?}'", - self + "logical_plan::from_proto() Unsupported logical plan '{self:?}'" )) })?; match plan { @@ -349,8 +347,7 @@ impl AsLogicalPlan for LogicalPlanNode { let file_format: Arc = match scan.file_format_type.as_ref().ok_or_else(|| { proto_error(format!( - "logical_plan::from_proto() Unsupported file format '{:?}'", - self + "logical_plan::from_proto() Unsupported file format '{self:?}'" )) })? { &FileFormatType::Parquet(protobuf::ParquetFormat {}) => { @@ -502,8 +499,7 @@ impl AsLogicalPlan for LogicalPlanNode { let env = ctx.runtime_env(); if !env.table_factories.contains_key(file_type) { Err(DataFusionError::Internal(format!( - "No TableProvider for file type: {}", - file_type + "No TableProvider for file type: {file_type}" )))? } @@ -1419,10 +1415,7 @@ mod roundtrip_tests { let proto: protobuf::LogicalExprNode = (&initial_struct).try_into().unwrap(); let round_trip: Expr = parse_expr(&proto, &ctx).unwrap(); - assert_eq!( - format!("{:?}", &initial_struct), - format!("{:?}", round_trip) - ); + assert_eq!(format!("{:?}", &initial_struct), format!("{round_trip:?}")); roundtrip_json_test(&proto); } @@ -1445,10 +1438,7 @@ mod roundtrip_tests { logical_plan_to_bytes_with_extension_codec(&topk_plan, &extension_codec)?; let logical_round_trip = logical_plan_from_bytes_with_extension_codec(&bytes, &ctx, &extension_codec)?; - assert_eq!( - format!("{:?}", topk_plan), - format!("{:?}", logical_round_trip) - ); + assert_eq!(format!("{topk_plan:?}"), format!("{logical_round_trip:?}")); Ok(()) } @@ -1537,7 +1527,7 @@ mod roundtrip_tests { let bytes = logical_plan_to_bytes_with_extension_codec(&scan, &codec)?; let logical_round_trip = logical_plan_from_bytes_with_extension_codec(&bytes, &ctx, &codec)?; - assert_eq!(format!("{:?}", scan), format!("{:?}", logical_round_trip)); + assert_eq!(format!("{scan:?}"), format!("{logical_round_trip:?}")); Ok(()) } @@ -1563,7 +1553,7 @@ mod roundtrip_tests { let bytes = logical_plan_to_bytes(&plan)?; let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; - assert_eq!(format!("{:?}", plan), format!("{:?}", logical_round_trip)); + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); Ok(()) } @@ -1589,7 +1579,7 @@ mod roundtrip_tests { let bytes = logical_plan_to_bytes(&plan)?; let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; - assert_eq!(format!("{:?}", plan), format!("{:?}", logical_round_trip)); + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); Ok(()) } @@ -1602,7 +1592,7 @@ mod roundtrip_tests { let plan = ctx.table("t1")?.into_optimized_plan()?; let bytes = logical_plan_to_bytes(&plan)?; let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; - assert_eq!(format!("{:?}", plan), format!("{:?}", logical_round_trip)); + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); Ok(()) } @@ -1619,7 +1609,7 @@ mod roundtrip_tests { .into_optimized_plan()?; let bytes = logical_plan_to_bytes(&plan)?; let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; - assert_eq!(format!("{:?}", plan), format!("{:?}", logical_round_trip)); + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); Ok(()) } @@ -1711,8 +1701,7 @@ mod roundtrip_tests { if let Some((input, _)) = inputs.split_first() { let proto = proto::TopKPlanProto::decode(buf).map_err(|e| { DataFusionError::Internal(format!( - "failed to decode logical plan: {:?}", - e + "failed to decode logical plan: {e:?}" )) })?; @@ -1751,8 +1740,7 @@ mod roundtrip_tests { proto.encode(buf).map_err(|e| { DataFusionError::Internal(format!( - "failed to encode logical plan: {:?}", - e + "failed to encode logical plan: {e:?}" )) })?; @@ -1859,9 +1847,7 @@ mod roundtrip_tests { let res: Result = (&proto).try_into(); assert!( res.is_err(), - "The value {:?} unexpectedly serialized without error:{:?}", - test_case, - res + "The value {test_case:?} unexpectedly serialized without error:{res:?}" ); } } @@ -2047,8 +2033,7 @@ mod roundtrip_tests { assert_eq!( test_case, roundtrip, "ScalarValue was not the same after round trip!\n\n\ - Input: {:?}\n\nRoundtrip: {:?}", - test_case, roundtrip + Input: {test_case:?}\n\nRoundtrip: {roundtrip:?}" ); } } @@ -2085,7 +2070,7 @@ mod roundtrip_tests { let field = Field::new("item", test_case, true); let proto: super::protobuf::Field = (&field).try_into().unwrap(); let roundtrip: Field = (&proto).try_into().unwrap(); - assert_eq!(format!("{:?}", field), format!("{:?}", roundtrip)); + assert_eq!(format!("{field:?}"), format!("{roundtrip:?}")); } } @@ -2223,7 +2208,7 @@ mod roundtrip_tests { for test_case in test_cases.into_iter() { let proto: super::protobuf::ArrowType = (&test_case).try_into().unwrap(); let roundtrip: DataType = (&proto).try_into().unwrap(); - assert_eq!(format!("{:?}", test_case), format!("{:?}", roundtrip)); + assert_eq!(format!("{test_case:?}"), format!("{roundtrip:?}")); } } @@ -2257,10 +2242,7 @@ mod roundtrip_tests { (&test_case).try_into().unwrap(); let returned_scalar: datafusion::scalar::ScalarValue = (&proto_scalar).try_into().unwrap(); - assert_eq!( - format!("{:?}", &test_case), - format!("{:?}", returned_scalar) - ); + assert_eq!(format!("{:?}", &test_case), format!("{returned_scalar:?}")); } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 413e989faef2..7fe380566f2f 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -70,39 +70,36 @@ impl std::error::Error for Error {} impl std::fmt::Display for Error { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { - Self::General(desc) => write!(f, "General error: {}", desc), + Self::General(desc) => write!(f, "General error: {desc}"), Self::InconsistentListTyping(type1, type2) => { write!( f, - "Lists with inconsistent typing; {:?} and {:?} found within list", - type1, type2, + "Lists with inconsistent typing; {type1:?} and {type2:?} found within list", ) } Self::InconsistentListDesignated { value, designated } => { write!( f, - "Value {:?} was inconsistent with designated type {:?}", - value, designated + "Value {value:?} was inconsistent with designated type {designated:?}" ) } Self::InvalidScalarValue(value) => { - write!(f, "{:?} is invalid as a DataFusion scalar value", value) + write!(f, "{value:?} is invalid as a DataFusion scalar value") } Self::InvalidScalarType(data_type) => { - write!(f, "{:?} is invalid as a DataFusion scalar type", data_type) + write!(f, "{data_type:?} is invalid as a DataFusion scalar type") } Self::InvalidTimeUnit(time_unit) => { write!( f, - "Only TimeUnit::Microsecond and TimeUnit::Nanosecond are valid time units, found: {:?}", - time_unit + "Only TimeUnit::Microsecond and TimeUnit::Nanosecond are valid time units, found: {time_unit:?}" ) } Self::UnsupportedScalarFunction(function) => { - write!(f, "Unsupported scalar function {:?}", function) + write!(f, "Unsupported scalar function {function:?}") } Self::NotImplemented(s) => { - write!(f, "Not implemented: {}", s) + write!(f, "Not implemented: {s}") } } } @@ -488,7 +485,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { .rev() .map(|expr| expr.try_into()) .collect::, Error>>()?, - op: format!("{:?}", op), + op: format!("{op:?}"), }; Self { expr_type: Some(ExprType::BinaryExpr(binary_expr)), diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 01d8e31b30de..ffc010465946 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -232,7 +232,7 @@ fn parse_required_physical_box_expr( .map(|e| parse_physical_expr(e.as_ref(), registry, input_schema)) .transpose()? .ok_or_else(|| { - DataFusionError::Internal(format!("Missing required field {:?}", field)) + DataFusionError::Internal(format!("Missing required field {field:?}")) }) } @@ -246,7 +246,7 @@ fn parse_required_physical_expr( .map(|e| parse_physical_expr(e, registry, input_schema)) .transpose()? .ok_or_else(|| { - DataFusionError::Internal(format!("Missing required field {:?}", field)) + DataFusionError::Internal(format!("Missing required field {field:?}")) }) } @@ -260,8 +260,7 @@ impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFun protobuf::physical_window_expr_node::WindowFunction::AggrFunction(n) => { let f = protobuf::AggregateFunction::from_i32(*n).ok_or_else(|| { proto_error(format!( - "Received an unknown window aggregate function: {}", - n + "Received an unknown window aggregate function: {n}" )) })?; @@ -271,8 +270,7 @@ impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFun let f = protobuf::BuiltInWindowFunction::from_i32(*n).ok_or_else(|| { proto_error(format!( - "Received an unknown window builtin function: {}", - n + "Received an unknown window builtin function: {n}" )) })?; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index bd3614df067b..cff23974e8c7 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -72,7 +72,7 @@ impl AsExecutionPlan for PhysicalPlanNode { Self: Sized, { protobuf::PhysicalPlanNode::decode(buf).map_err(|e| { - DataFusionError::Internal(format!("failed to decode physical plan: {:?}", e)) + DataFusionError::Internal(format!("failed to decode physical plan: {e:?}")) }) } @@ -82,7 +82,7 @@ impl AsExecutionPlan for PhysicalPlanNode { Self: Sized, { self.encode(buf).map_err(|e| { - DataFusionError::Internal(format!("failed to encode physical plan: {:?}", e)) + DataFusionError::Internal(format!("failed to encode physical plan: {e:?}")) }) } @@ -95,8 +95,7 @@ impl AsExecutionPlan for PhysicalPlanNode { ) -> Result, DataFusionError> { let plan = self.physical_plan_type.as_ref().ok_or_else(|| { proto_error(format!( - "physical_plan::from_proto() Unsupported physical plan '{:?}'", - self + "physical_plan::from_proto() Unsupported physical plan '{self:?}'" )) })?; match plan { @@ -577,8 +576,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|expr| { let expr = expr.expr_type.as_ref().ok_or_else(|| { proto_error(format!( - "physical_plan::from_proto() Unexpected expr {:?}", - self + "physical_plan::from_proto() Unexpected expr {self:?}" )) })?; if let protobuf::physical_expr_node::ExprType::Sort(sort_expr) = expr { @@ -587,8 +585,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .as_ref() .ok_or_else(|| { proto_error(format!( - "physical_plan::from_proto() Unexpected sort expr {:?}", - self + "physical_plan::from_proto() Unexpected sort expr {self:?}" )) })? .as_ref(); @@ -601,8 +598,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) } else { Err(DataFusionError::Internal(format!( - "physical_plan::from_proto() {:?}", - self + "physical_plan::from_proto() {self:?}" ))) } }) @@ -623,8 +619,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|expr| { let expr = expr.expr_type.as_ref().ok_or_else(|| { proto_error(format!( - "physical_plan::from_proto() Unexpected expr {:?}", - self + "physical_plan::from_proto() Unexpected expr {self:?}" )) })?; if let protobuf::physical_expr_node::ExprType::Sort(sort_expr) = expr { @@ -633,8 +628,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .as_ref() .ok_or_else(|| { proto_error(format!( - "physical_plan::from_proto() Unexpected sort expr {:?}", - self + "physical_plan::from_proto() Unexpected sort expr {self:?}" )) })? .as_ref(); @@ -647,8 +641,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) } else { Err(DataFusionError::Internal(format!( - "physical_plan::from_proto() {:?}", - self + "physical_plan::from_proto() {self:?}" ))) } }) @@ -1117,8 +1110,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) } Err(e) => Err(DataFusionError::Internal(format!( - "Unsupported plan and extension codec failed with [{}]. Plan: {:?}", - e, plan_clone + "Unsupported plan and extension codec failed with [{e}]. Plan: {plan_clone:?}" ))), } } @@ -1257,10 +1249,7 @@ mod roundtrip_tests { let result_exec_plan: Arc = proto .try_into_physical_plan(&ctx, runtime.deref(), &codec) .expect("from proto"); - assert_eq!( - format!("{:?}", exec_plan), - format!("{:?}", result_exec_plan) - ); + assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); Ok(()) } @@ -1276,10 +1265,7 @@ mod roundtrip_tests { let result_exec_plan: Arc = proto .try_into_physical_plan(&ctx, runtime.deref(), &codec) .expect("from proto"); - assert_eq!( - format!("{:?}", exec_plan), - format!("{:?}", result_exec_plan) - ); + assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); Ok(()) } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 928c6098ea9a..c5defbf7764a 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -130,8 +130,7 @@ impl TryFrom> for protobuf::PhysicalExprNode { Ok(AggregateFunction::ApproxMedian.into()) } else { Err(DataFusionError::NotImplemented(format!( - "Aggregate function not supported: {:?}", - a + "Aggregate function not supported: {a:?}" ))) }?; let expressions: Vec = a @@ -333,8 +332,7 @@ impl TryFrom> for protobuf::PhysicalExprNode { }) } else { Err(DataFusionError::Internal(format!( - "physical_plan::to_proto() unsupported expression {:?}", - value + "physical_plan::to_proto() unsupported expression {value:?}" ))) } } diff --git a/datafusion/row/src/layout.rs b/datafusion/row/src/layout.rs index 1518df9bf55a..502812cb9f66 100644 --- a/datafusion/row/src/layout.rs +++ b/datafusion/row/src/layout.rs @@ -96,9 +96,7 @@ impl RowLayout { pub fn new(schema: &Schema, row_type: RowType) -> Self { assert!( row_supported(schema, row_type), - "{:?}Row with {:?} not supported yet.", - row_type, - schema, + "{row_type:?}Row with {schema:?} not supported yet.", ); let null_free = schema_null_free(schema); let field_count = schema.fields().len(); diff --git a/datafusion/sql/examples/sql.rs b/datafusion/sql/examples/sql.rs index c2e4d257b2ba..8d6135aabea4 100644 --- a/datafusion/sql/examples/sql.rs +++ b/datafusion/sql/examples/sql.rs @@ -52,7 +52,7 @@ fn main() { let plan = sql_to_rel.sql_statement_to_plan(statement.clone()).unwrap(); // show the plan - println!("{:?}", plan); + println!("{plan:?}"); } struct MySchemaProvider { diff --git a/datafusion/sql/src/parser.rs b/datafusion/sql/src/parser.rs index 425135ee51f2..68e368e49807 100644 --- a/datafusion/sql/src/parser.rs +++ b/datafusion/sql/src/parser.rs @@ -166,7 +166,7 @@ impl<'a> DFParser<'a> { /// Report an unexpected token fn expected(&self, expected: &str, found: Token) -> Result { - parser_err!(format!("Expected {}, found: {}", expected, found)) + parser_err!(format!("Expected {expected}, found: {found}")) } /// Parse a new expression @@ -474,17 +474,14 @@ mod tests { match DFParser::parse_sql(sql) { Ok(statements) => { panic!( - "Expected parse error for '{}', but was successful: {:?}", - sql, statements + "Expected parse error for '{sql}', but was successful: {statements:?}" ); } Err(e) => { let error_message = e.to_string(); assert!( error_message.contains(expected_error), - "Expected error '{}' not found in actual error '{}'", - expected_error, - error_message + "Expected error '{expected_error}' not found in actual error '{error_message}'" ); } } diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index a4cc0b7753d3..2ca1550d88e1 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -149,15 +149,14 @@ fn plan_key(key: SQLExpr) -> Result { let scalar = match key { SQLExpr::Value(Value::Number(s, _)) => ScalarValue::Int64(Some( s.parse() - .map_err(|_| ParserError(format!("Cannot parse {} as i64.", s)))?, + .map_err(|_| ParserError(format!("Cannot parse {s} as i64.")))?, )), SQLExpr::Value(Value::SingleQuotedString(s) | Value::DoubleQuotedString(s)) => { ScalarValue::Utf8(Some(s)) } _ => { return Err(DataFusionError::SQL(ParserError(format!( - "Unsuported index key expression: {:?}", - key + "Unsuported index key expression: {key:?}" )))); } }; @@ -407,8 +406,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { filter, } => self.show_columns_to_plan(extended, full, table_name, filter), _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported SQL statement: {:?}", - sql + "Unsupported SQL statement: {sql:?}" ))), } } @@ -485,8 +483,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let cte_name = normalize_ident(cte.alias.name.clone()); if planner_context.ctes.contains_key(&cte_name) { return Err(DataFusionError::SQL(ParserError(format!( - "WITH query name {:?} specified more than once", - cte_name + "WITH query name {cte_name:?} specified more than once" )))); } // create logical plan & pass backreferencing CTEs @@ -558,8 +555,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } SetExpr::Query(q) => self.query_to_plan(*q, planner_context), _ => Err(DataFusionError::NotImplemented(format!( - "Query {} not implemented yet", - set_expr + "Query {set_expr} not implemented yet" ))), } } @@ -781,8 +777,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } JoinOperator::CrossJoin => self.parse_cross_join(left, right), other => Err(DataFusionError::NotImplemented(format!( - "Unsupported JOIN operator {:?}", - other + "Unsupported JOIN operator {other:?}" ))), } } @@ -894,8 +889,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // @todo Support TableFactory::TableFunction? _ => { return Err(DataFusionError::NotImplemented(format!( - "Unsupported ast node {:?} in create_relation", - relation + "Unsupported ast node {relation:?} in create_relation" ))); } }; @@ -1143,7 +1137,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } else { match having_expr_opt { Some(having_expr) => return Err(DataFusionError::Plan( - format!("HAVING clause references: {} must appear in the GROUP BY clause or be used in an aggregate function", having_expr))), + format!("HAVING clause references: {having_expr} must appear in the GROUP BY clause or be used in an aggregate function"))), None => (plan, select_exprs, having_expr_opt) } }; @@ -1369,8 +1363,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Expr::Literal(ScalarValue::Int64(Some(s))) => { if s < 0 { return Err(DataFusionError::Plan(format!( - "Offset must be >= 0, '{}' was provided.", - s + "Offset must be >= 0, '{s}' was provided." ))); } Ok(s as usize) @@ -1562,7 +1555,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SelectItem::QualifiedWildcard(ref object_name, options) => { Self::check_wildcard_options(options)?; - let qualifier = format!("{}", object_name); + let qualifier = format!("{object_name}"); // do not expand from outer schema expand_qualified_wildcard(&qualifier, plan.schema().as_ref(), plan) } @@ -1608,7 +1601,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .find(|field| match field.qualifier() { Some(field_q) => { field.name() == &col.name - && field_q.ends_with(&format!(".{}", q)) + && field_q.ends_with(&format!(".{q}")) } _ => false, }) { @@ -1645,8 +1638,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } FunctionArg::Unnamed(FunctionArgExpr::Wildcard) => Ok(Expr::Wildcard), _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported qualified wildcard argument: {:?}", - sql + "Unsupported qualified wildcard argument: {sql:?}" ))), } } @@ -1684,8 +1676,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { BinaryOperator::PGBitwiseShiftLeft => Ok(Operator::BitwiseShiftLeft), BinaryOperator::StringConcat => Ok(Operator::StringConcat), _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported SQL binary operator {:?}", - op + "Unsupported SQL binary operator {op:?}" ))), }?; @@ -1720,8 +1711,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .parse::() .map_err(|_e| { DataFusionError::Internal(format!( - "negative operator can be only applied to integer and float operands, got: {}", - n)) + "negative operator can be only applied to integer and float operands, got: {n}")) })?)), }, // not a literal, apply negative operator on expression @@ -1729,8 +1719,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported SQL unary operator {:?}", - op + "Unsupported SQL unary operator {op:?}" ))), } } @@ -1792,8 +1781,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { self.convert_data_type(&data_type)?, ))), other => Err(DataFusionError::NotImplemented(format!( - "Unsupported value {:?} in a values list expression", - other + "Unsupported value {other:?} in a values list expression" ))), }) .collect::>>() @@ -1815,16 +1803,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(index) => index - 1, Err(_) => { return Err(DataFusionError::Internal(format!( - "Invalid placeholder, not a number: {}", - param + "Invalid placeholder, not a number: {param}" ))); } }; // Check if the placeholder is in the parameter list if param_data_types.len() <= idx { return Err(DataFusionError::Internal(format!( - "Placehoder {} does not exist in the parameter list: {:?}", - param, param_data_types + "Placehoder {param} does not exist in the parameter list: {param_data_types:?}" ))); } // Data type of the parameter @@ -1855,7 +1841,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Extract { field, expr } => Ok(Expr::ScalarFunction { fun: BuiltinScalarFunction::DatePart, args: vec![ - Expr::Literal(ScalarValue::Utf8(Some(format!("{}", field)))), + Expr::Literal(ScalarValue::Utf8(Some(format!("{field}")))), self.sql_expr_to_logical_expr(*expr, schema, planner_context)?, ], }), @@ -1883,8 +1869,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .get_variable_type(&var_names) .ok_or_else(|| { DataFusionError::Execution(format!( - "variable {:?} has no type information", - var_names + "variable {var_names:?} has no type information" )) })?; Ok(Expr::ScalarVariable(ty, var_names)) @@ -1906,8 +1891,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { plan_indexed(col(normalize_ident(id)), keys) } else { Err(DataFusionError::NotImplemented(format!( - "map access requires an identifier, found column {} instead", - column + "map access requires an identifier, found column {column} instead" ))) } } @@ -1925,8 +1909,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .get_variable_type(&var_names) .ok_or_else(|| { DataFusionError::Execution(format!( - "variable {:?} has no type information", - var_names + "variable {var_names:?} has no type information" )) })?; Ok(Expr::ScalarVariable(ty, var_names)) @@ -1937,7 +1920,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { r @ OwnedTableReference::Bare { .. } | r @ OwnedTableReference::Full { .. } => { return Err(DataFusionError::Plan(format!( - "Unsupported compound identifier '{:?}'", r, + "Unsupported compound identifier '{r:?}'", ))); } }; @@ -2183,8 +2166,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; return Err(DataFusionError::Plan(format!( - "Substring without for/from is not valid {:?}", - orig_sql + "Substring without for/from is not valid {orig_sql:?}" ))); } }; @@ -2353,8 +2335,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(Expr::AggregateUDF { fun: fm, args, filter: None }) } _ => Err(DataFusionError::Plan(format!( - "Invalid function '{}'", - name + "Invalid function '{name}'" ))), }, } @@ -2383,8 +2364,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::ArrayAgg(array_agg) => self.parse_array_agg(array_agg, schema, planner_context), _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported ast node in sqltorel: {:?}", - sql + "Unsupported ast node in sqltorel: {sql:?}" ))), } } @@ -2397,10 +2377,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(WindowFunction::AggregateUDF) }) .ok_or_else(|| { - DataFusionError::Plan(format!( - "There is no window function named {}", - name - )) + DataFusionError::Plan(format!("There is no window function named {name}")) }) } @@ -2476,15 +2453,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if let Some(order_by) = order_by { return Err(DataFusionError::NotImplemented(format!( - "ORDER BY not supported in ARRAY_AGG: {}", - order_by + "ORDER BY not supported in ARRAY_AGG: {order_by}" ))); } if let Some(limit) = limit { return Err(DataFusionError::NotImplemented(format!( - "LIMIT not supported in ARRAY_AGG: {}", - limit + "LIMIT not supported in ARRAY_AGG: {limit}" ))); } @@ -2553,22 +2528,19 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { if leading_precision.is_some() { return Err(DataFusionError::NotImplemented(format!( - "Unsupported Interval Expression with leading_precision {:?}", - leading_precision + "Unsupported Interval Expression with leading_precision {leading_precision:?}" ))); } if last_field.is_some() { return Err(DataFusionError::NotImplemented(format!( - "Unsupported Interval Expression with last_field {:?}", - last_field + "Unsupported Interval Expression with last_field {last_field:?}" ))); } if fractional_seconds_precision.is_some() { return Err(DataFusionError::NotImplemented(format!( - "Unsupported Interval Expression with fractional_seconds_precision {:?}", - fractional_seconds_precision + "Unsupported Interval Expression with fractional_seconds_precision {fractional_seconds_precision:?}" ))); } @@ -2579,8 +2551,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) => s, _ => { return Err(DataFusionError::NotImplemented(format!( - "Unsupported interval argument. Expected string literal, got: {:?}", - value + "Unsupported interval argument. Expected string literal, got: {value:?}" ))); } }; @@ -2615,8 +2586,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { String::from("SELECT name, setting FROM information_schema.df_settings WHERE name = 'datafusion.execution.time_zone'") } else { format!( - "SELECT name, setting FROM information_schema.df_settings WHERE name = '{}'", - variable + "SELECT name, setting FROM information_schema.df_settings WHERE name = '{variable}'" ) }; @@ -2674,8 +2644,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }, // for capture signed number e.g. +8, -8 SQLExpr::UnaryOp { op, expr } => match op { - UnaryOperator::Plus => format!("+{}", expr), - UnaryOperator::Minus => format!("-{}", expr), + UnaryOperator::Plus => format!("+{expr}"), + UnaryOperator::Minus => format!("-{expr}"), _ => { return Err(DataFusionError::Plan(format!( "Unsupported Value {}", @@ -2734,8 +2704,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; let query = format!( - "SELECT {} FROM information_schema.columns WHERE {}", - select_list, where_clause + "SELECT {select_list} FROM information_schema.columns WHERE {where_clause}" ); let mut rewrite = DFParser::parse_sql(&query)?; @@ -2763,8 +2732,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .get_table_provider((&table_ref).into())?; let query = format!( - "SELECT table_catalog, table_schema, table_name, definition FROM information_schema.views WHERE {}", - where_clause + "SELECT table_catalog, table_schema, table_name, definition FROM information_schema.views WHERE {where_clause}" ); let mut rewrite = DFParser::parse_sql(&query)?; @@ -2799,8 +2767,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } _ => { return Err(DataFusionError::NotImplemented(format!( - "Arrays with elements other than literal are not supported: {}", - value + "Arrays with elements other than literal are not supported: {value}" ))); } } @@ -2813,8 +2780,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(lit(ScalarValue::new_list(None, DataType::Utf8))) } else if data_types.len() > 1 { Err(DataFusionError::NotImplemented(format!( - "Arrays with different types are not supported: {:?}", - data_types, + "Arrays with different types are not supported: {data_types:?}", ))) } else { let data_type = values[0].get_datatype(); @@ -2843,8 +2809,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let str = str.replace('.', ""); let n = str.parse::().map_err(|_| { DataFusionError::from(ParserError(format!( - "Cannot parse {} as i128 when building decimal", - str + "Cannot parse {str} as i128 when building decimal" ))) })?; Ok(Expr::Literal(ScalarValue::Decimal128( @@ -2855,15 +2820,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } else { let number = n.parse::().map_err(|_| { DataFusionError::from(ParserError(format!( - "Cannot parse {} as i128 when building decimal", - n + "Cannot parse {n} as i128 when building decimal" ))) })?; Ok(Expr::Literal(ScalarValue::Decimal128(Some(number), 38, 0))) } } else { n.parse::().map(lit).map_err(|_| { - DataFusionError::from(ParserError(format!("Cannot parse {} as f64", n))) + DataFusionError::from(ParserError(format!("Cannot parse {n} as f64"))) }) } } @@ -2941,8 +2905,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } else { // We dont support TIMETZ and TIME WITH TIME ZONE for now Err(DataFusionError::NotImplemented(format!( - "Unsupported SQL type {:?}", - sql_type + "Unsupported SQL type {sql_type:?}" ))) } } @@ -2986,8 +2949,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { | SQLDataType::Time(Some(_), _) | SQLDataType::Dec(_) | SQLDataType::Clob(_) => Err(DataFusionError::NotImplemented(format!( - "Unsupported SQL type {:?}", - sql_type + "Unsupported SQL type {sql_type:?}" ))), } } @@ -3151,8 +3113,8 @@ mod tests { ), ]; for (a, b) in test_data { - let sql = format!("SELECT {}", a); - let expected = format!("Projection: {}\n EmptyRelation", b); + let sql = format!("SELECT {a}"); + let expected = format!("Projection: {b}\n EmptyRelation"); quick_test_with_options( &sql, &expected, @@ -3236,7 +3198,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Internal("Decimal(precision = 0, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.")"##, - format!("{:?}", err) + format!("{err:?}") ); } // precision > 38 @@ -3245,7 +3207,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Internal("Decimal(precision = 39, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.")"##, - format!("{:?}", err) + format!("{err:?}") ); } // precision < scale @@ -3254,7 +3216,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Internal("Decimal(precision = 5, scale = 10) should satisfy `0 < precision <= 38`, and `scale <= precision`.")"##, - format!("{:?}", err) + format!("{err:?}") ); } } @@ -3272,7 +3234,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Plan("Projections require unique expression names but the expression \"person.age\" at position 0 and \"person.age\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"##, - format!("{:?}", err) + format!("{err:?}") ); } @@ -3282,7 +3244,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Plan("Projections require unique expression names but the expression \"person.age\" at position 3 and \"person.age\" at position 8 have the same name. Consider aliasing (\"AS\") one of them.")"##, - format!("{:?}", err) + format!("{err:?}") ); } @@ -3471,7 +3433,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"Source table contains 3 columns but only 2 names given as column alias\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3481,7 +3443,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Internal(\"column reference id is ambiguous\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3506,7 +3468,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"HAVING clause references: person.age > Int64(100) AND person.age < Int64(200) must appear in the GROUP BY clause or be used in an aggregate function\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3518,7 +3480,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"HAVING clause references: person.first_name = Utf8(\\\"M\\\") must appear in the GROUP BY clause or be used in an aggregate function\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3531,7 +3493,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"HAVING clause references non-aggregate values: Expression person.first_name could not be resolved from available columns: person.id, MAX(person.age)\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3543,7 +3505,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"HAVING clause references: person.age > Int64(100) must appear in the GROUP BY clause or be used in an aggregate function\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3555,7 +3517,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"Projection references non-aggregate values: Expression person.first_name could not be resolved from available columns: MAX(person.age)\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3593,7 +3555,7 @@ mod tests { "Plan(\"HAVING clause references non-aggregate values: \ Expression person.first_name could not be resolved from available columns: \ COUNT(UInt8(1))\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3717,7 +3679,7 @@ mod tests { "Plan(\"HAVING clause references non-aggregate values: \ Expression person.last_name could not be resolved from available columns: \ person.first_name, MAX(person.age)\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -3870,7 +3832,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Plan("Projections require unique expression names but the expression \"MIN(person.age)\" at position 0 and \"MIN(person.age)\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"##, - format!("{:?}", err) + format!("{err:?}") ); } @@ -3911,7 +3873,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Plan("Projections require unique expression names but the expression \"MIN(person.age) AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"##, - format!("{:?}", err) + format!("{err:?}") ); } @@ -3941,7 +3903,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Plan("Projections require unique expression names but the expression \"person.state AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.")"##, - format!("{:?}", err) + format!("{err:?}") ); } @@ -3961,7 +3923,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!("Schema error: No field named 'doesnotexist'. Valid fields are 'SUM(person.age)', \ 'person'.'id', 'person'.'first_name', 'person'.'last_name', 'person'.'age', 'person'.'state', \ - 'person'.'salary', 'person'.'birth_date', 'person'.'😀'.", format!("{}", err)); + 'person'.'salary', 'person'.'birth_date', 'person'.'😀'.", format!("{err}")); } #[test] @@ -3977,7 +3939,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r#"NotImplemented("Interval field value out of range: \"100000000000000000 day\"")"#, - format!("{:?}", err) + format!("{err:?}") ); } @@ -4005,7 +3967,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r#"NotImplemented("Recursive CTEs are not supported")"#, - format!("{:?}", err) + format!("{err:?}") ); } @@ -4015,7 +3977,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r#"NotImplemented("Arrays with elements other than literal are not supported: now()")"#, - format!("{:?}", err) + format!("{err:?}") ); } @@ -4051,14 +4013,14 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"Projection references non-aggregate values: Expression person.state could not be resolved from available columns: Int64(0), MIN(person.age)\")", - format!("{:?}", err) + format!("{err:?}") ); let sql2 = "SELECT state, MIN(age) FROM person GROUP BY 5"; let err2 = logical_plan(sql2).expect_err("query should have failed"); assert_eq!( "Plan(\"Projection references non-aggregate values: Expression person.state could not be resolved from available columns: Int64(5), MIN(person.age)\")", - format!("{:?}", err2) + format!("{err2:?}") ); } @@ -4078,7 +4040,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( r##"Plan("Projections require unique expression names but the expression \"MIN(person.age)\" at position 1 and \"MIN(person.age)\" at position 2 have the same name. Consider aliasing (\"AS\") one of them.")"##, - format!("{:?}", err) + format!("{err:?}") ); } @@ -4139,7 +4101,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4149,7 +4111,7 @@ mod tests { let sql = "SELECT age, MIN(first_name) FROM person GROUP BY age + 1"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!("Plan(\"Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4299,7 +4261,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"Order by index starts at 1 for column indexes\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4309,7 +4271,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"Order by column out of bounds, specified: 2, max: 1\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4406,7 +4368,7 @@ mod tests { "Plan(\"Projection references non-aggregate values: \ Expression aggregate_test_100.c13 could not be resolved from available columns: \ aggregate_test_100.c1, MIN(aggregate_test_100.c12)\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4456,7 +4418,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"File compression type can be specified for CSV/JSON files.\")", - format!("{:?}", err) + format!("{err:?}") ); } } @@ -4468,7 +4430,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"Column definitions can not be specified for PARQUET files.\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4686,7 +4648,7 @@ mod tests { not compatible with column IntervalMonthDayNano\ (\\\"950737950189618795196236955648\\\") \ (type: Interval(MonthDayNano))\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4792,7 +4754,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"UNION Column a (type: Boolean) is not compatible with column a (type: Utf8)\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4929,7 +4891,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"With window frame of type RANGE, the order by expression must be of length 1, got 0\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -4940,7 +4902,7 @@ mod tests { let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( "Plan(\"With window frame of type RANGE, the order by expression must be of length 1, got 2\")", - format!("{:?}", err) + format!("{err:?}") ); } @@ -5217,12 +5179,12 @@ mod tests { /// Create logical plan, write with formatter, compare to expected output fn quick_test(sql: &str, expected: &str) { let plan = logical_plan(sql).unwrap(); - assert_eq!(format!("{:?}", plan), expected); + assert_eq!(format!("{plan:?}"), expected); } fn quick_test_with_options(sql: &str, expected: &str, options: ParserOptions) { let plan = logical_plan_with_options(sql, options).unwrap(); - assert_eq!(format!("{:?}", plan), expected); + assert_eq!(format!("{plan:?}"), expected); } fn prepare_stmt_quick_test( @@ -5234,11 +5196,11 @@ mod tests { let assert_plan = plan.clone(); // verify plan - assert_eq!(format!("{:?}", assert_plan), expected_plan); + assert_eq!(format!("{assert_plan:?}"), expected_plan); // verify data types if let LogicalPlan::Prepare(Prepare { data_types, .. }) = assert_plan { - let dt = format!("{:?}", data_types); + let dt = format!("{data_types:?}"); assert_eq!(dt, expected_data_types); } @@ -5252,7 +5214,7 @@ mod tests { ) -> LogicalPlan { // replace params let plan = plan.with_param_values(param_values).unwrap(); - assert_eq!(format!("{:?}", plan), expected_plan); + assert_eq!(format!("{plan:?}"), expected_plan); plan } @@ -6448,10 +6410,10 @@ mod tests { fn assert_field_not_found(err: DataFusionError, name: &str) { match err { DataFusionError::SchemaError { .. } => { - let msg = format!("{}", err); - let expected = format!("Schema error: No field named '{}'.", name); + let msg = format!("{err}"); + let expected = format!("Schema error: No field named '{name}'."); if !msg.starts_with(&expected) { - panic!("error [{}] did not start with [{}]", msg, expected); + panic!("error [{msg}] did not start with [{expected}]"); } } _ => panic!("assert_field_not_found wrong error type"), diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index a976712ace95..a7f7a3dd692a 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -125,7 +125,7 @@ fn check_column_satisfies_expr( expr, columns .iter() - .map(|e| format!("{}", e)) + .map(|e| format!("{e}")) .collect::>() .join(", ") ))); @@ -488,14 +488,12 @@ pub fn window_expr_common_partition_keys(window_exprs: &[Expr]) -> Result<&[Expr Ok(partition_by) } expr => Err(DataFusionError::Execution(format!( - "Impossibly got non-window expr {:?}", - expr + "Impossibly got non-window expr {expr:?}" ))), } } expr => Err(DataFusionError::Execution(format!( - "Impossibly got non-window expr {:?}", - expr + "Impossibly got non-window expr {expr:?}" ))), }) .collect::>>()?; @@ -532,8 +530,7 @@ pub(crate) fn make_decimal_type( || scale.unsigned_abs() > precision { Err(DataFusionError::Internal(format!( - "Decimal(precision = {}, scale = {}) should satisfy `0 < precision <= 38`, and `scale <= precision`.", - precision, scale + "Decimal(precision = {precision}, scale = {scale}) should satisfy `0 < precision <= 38`, and `scale <= precision`." ))) } else { Ok(DataType::Decimal128(precision, scale)) diff --git a/parquet-test-utils/src/lib.rs b/parquet-test-utils/src/lib.rs index 7a873b8c702a..54db76133573 100644 --- a/parquet-test-utils/src/lib.rs +++ b/parquet-test-utils/src/lib.rs @@ -93,7 +93,7 @@ impl TestParquetFile { } writer.close().unwrap(); - println!("Generated test dataset with {} rows", num_rows); + println!("Generated test dataset with {num_rows} rows"); let size = std::fs::metadata(&path)?.len() as usize; diff --git a/test-utils/src/data_gen.rs b/test-utils/src/data_gen.rs index e6e3aa39845d..f5ed8510a79e 100644 --- a/test-utils/src/data_gen.rs +++ b/test-utils/src/data_gen.rs @@ -106,10 +106,9 @@ impl BatchBuilder { for pod in pods { let num_containers = rng.gen_range(self.options.containers_per_pod.clone()); for container_idx in 0..num_containers { - let container = format!("{}_container_{}", service, container_idx); + let container = format!("{service}_container_{container_idx}"); let image = format!( - "{}@sha256:30375999bf03beec2187843017b10c9e88d8b1a91615df4eb6350fb39472edd9", - container + "{container}@sha256:30375999bf03beec2187843017b10c9e88d8b1a91615df4eb6350fb39472edd9" ); let num_entries = @@ -162,7 +161,7 @@ impl BatchBuilder { self.request_method .append_value(methods[rng.gen_range(0..methods.len())]); self.request_host - .append_value(format!("https://{}.mydomain.com", service)); + .append_value(format!("https://{service}.mydomain.com")); self.request_bytes .append_option(rng.gen_bool(0.9).then(|| rng.gen()));