From f59761bb9bb674a53736f1afe637430d8e06f9b5 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 12:54:47 -0700 Subject: [PATCH 01/13] Upgrade to datafusion 16 --- ballista-cli/Cargo.toml | 4 +- ballista/client/Cargo.toml | 6 +- ballista/client/src/context.rs | 52 ++++++++-------- ballista/core/Cargo.toml | 8 +-- ballista/core/src/error.rs | 8 +-- .../src/execution_plans/distributed_query.rs | 4 -- .../src/execution_plans/shuffle_reader.rs | 4 -- .../src/execution_plans/shuffle_writer.rs | 4 -- .../src/execution_plans/unresolved_shuffle.rs | 4 -- ballista/core/src/serde/generated/ballista.rs | 4 +- ballista/core/src/serde/mod.rs | 7 +-- .../src/serde/physical_plan/from_proto.rs | 6 +- ballista/core/src/serde/physical_plan/mod.rs | 16 ++--- ballista/core/src/utils.rs | 6 +- ballista/executor/Cargo.toml | 8 +-- ballista/scheduler/Cargo.toml | 9 +-- ballista/scheduler/src/flight_sql.rs | 2 +- ballista/scheduler/src/planner.rs | 18 +++--- .../scheduler/src/scheduler_server/grpc.rs | 11 ++-- .../scheduler/src/scheduler_server/mod.rs | 5 +- .../scheduler/src/state/execution_graph.rs | 61 +++++++++++++------ .../state/execution_graph/execution_stage.rs | 3 +- .../src/state/execution_graph_dot.rs | 8 +-- ballista/scheduler/src/state/mod.rs | 6 +- .../scheduler/src/state/session_manager.rs | 21 ------- benchmarks/Cargo.toml | 4 +- benchmarks/src/bin/nyctaxi.rs | 6 +- benchmarks/src/bin/tpch.rs | 43 +++++++------ examples/Cargo.toml | 2 +- python/Cargo.toml | 2 +- 30 files changed, 169 insertions(+), 173 deletions(-) diff --git a/ballista-cli/Cargo.toml b/ballista-cli/Cargo.toml index 8c295efca..1ca2167a2 100644 --- a/ballista-cli/Cargo.toml +++ b/ballista-cli/Cargo.toml @@ -33,8 +33,8 @@ ballista = { path = "../ballista/client", version = "0.10.0", features = [ "standalone", ] } clap = { version = "3", features = ["derive", "cargo"] } -datafusion = "15.0.0" -datafusion-cli = "15.0.0" +datafusion = "16.0.0" +datafusion-cli = "16.0.0" dirs = "4.0.0" env_logger = "0.10" mimalloc = { version = "0.1", default-features = false } diff --git a/ballista/client/Cargo.toml b/ballista/client/Cargo.toml index 48beaa98e..f51c0b919 100644 --- a/ballista/client/Cargo.toml +++ b/ballista/client/Cargo.toml @@ -31,12 +31,12 @@ rust-version = "1.63" ballista-core = { path = "../core", version = "0.10.0" } ballista-executor = { path = "../executor", version = "0.10.0", optional = true } ballista-scheduler = { path = "../scheduler", version = "0.10.0", optional = true } -datafusion = "15.0.0" -datafusion-proto = "15.0.0" +datafusion = "16.0.0" +datafusion-proto = "16.0.0" futures = "0.3" log = "0.4" parking_lot = "0.12" -sqlparser = "0.27" +sqlparser = "0.30" tempfile = "3" tokio = "1.0" diff --git a/ballista/client/src/context.rs b/ballista/client/src/context.rs index 51a6c3bc2..cd4c94a41 100644 --- a/ballista/client/src/context.rs +++ b/ballista/client/src/context.rs @@ -33,6 +33,7 @@ use ballista_core::utils::{ use datafusion_proto::protobuf::LogicalPlanNode; use datafusion::catalog::TableReference; +use datafusion::common::OwnedTableReference; use datafusion::dataframe::DataFrame; use datafusion::datasource::{source_as_provider, TableProvider}; use datafusion::error::{DataFusionError, Result}; @@ -212,7 +213,7 @@ impl BallistaContext { &self, path: &str, options: AvroReadOptions<'_>, - ) -> Result> { + ) -> Result { let df = self.context.read_avro(path, options).await?; Ok(df) } @@ -223,7 +224,7 @@ impl BallistaContext { &self, path: &str, options: ParquetReadOptions<'_>, - ) -> Result> { + ) -> Result { let df = self.context.read_parquet(path, options).await?; Ok(df) } @@ -234,7 +235,7 @@ impl BallistaContext { &self, path: &str, options: CsvReadOptions<'_>, - ) -> Result> { + ) -> Result { let df = self.context.read_csv(path, options).await?; Ok(df) } @@ -256,13 +257,10 @@ impl BallistaContext { path: &str, options: CsvReadOptions<'_>, ) -> Result<()> { - let plan = self - .read_csv(path, options) - .await - .map_err(|e| { - DataFusionError::Context(format!("Can't read CSV: {}", path), Box::new(e)) - })? - .to_logical_plan()?; + let df = self.read_csv(path, options).await.map_err(|e| { + DataFusionError::Context(format!("Can't read CSV: {}", path), Box::new(e)) + })?; + let plan = df.into_optimized_plan()?; match plan { LogicalPlan::TableScan(TableScan { source, .. }) => { self.register_table(name, source_as_provider(&source)?) @@ -277,7 +275,11 @@ impl BallistaContext { path: &str, options: ParquetReadOptions<'_>, ) -> Result<()> { - match self.read_parquet(path, options).await?.to_logical_plan()? { + match self + .read_parquet(path, options) + .await? + .into_optimized_plan()? + { LogicalPlan::TableScan(TableScan { source, .. }) => { self.register_table(name, source_as_provider(&source)?) } @@ -291,7 +293,7 @@ impl BallistaContext { path: &str, options: AvroReadOptions<'_>, ) -> Result<()> { - match self.read_avro(path, options).await?.to_logical_plan()? { + match self.read_avro(path, options).await?.into_optimized_plan()? { LogicalPlan::TableScan(TableScan { source, .. }) => { self.register_table(name, source_as_provider(&source)?) } @@ -302,7 +304,7 @@ impl BallistaContext { /// is a 'show *' sql pub async fn is_show_statement(&self, sql: &str) -> Result { let mut is_show_variable: bool = false; - let statements = DFParser::parse_sql(sql)?; + let mut statements = DFParser::parse_sql(sql)?; if statements.len() != 1 { return Err(DataFusionError::NotImplemented( @@ -310,9 +312,8 @@ impl BallistaContext { )); } - if let DFStatement::Statement(s) = &statements[0] { - let st: &Statement = s; - match st { + if let Some(DFStatement::Statement(s)) = statements.remove(0) { + match *s { Statement::ShowVariable { .. } => { is_show_variable = true; } @@ -332,7 +333,7 @@ impl BallistaContext { /// /// This method is `async` because queries of type `CREATE EXTERNAL TABLE` /// might require the schema to be inferred. - pub async fn sql(&self, sql: &str) -> Result> { + pub async fn sql(&self, sql: &str) -> Result { let mut ctx = self.context.clone(); let is_show = self.is_show_statement(sql).await?; @@ -361,7 +362,7 @@ impl BallistaContext { } } - let plan = ctx.create_logical_plan(sql)?; + let plan = ctx.state().create_logical_plan(sql).await?; match plan { LogicalPlan::CreateExternalTable(CreateExternalTable { @@ -375,6 +376,11 @@ impl BallistaContext { ref if_not_exists, .. }) => { + let name = match name { + OwnedTableReference::Bare { table, .. } => table, + OwnedTableReference::Partial { table, .. } => table, + OwnedTableReference::Full { table, .. } => table, + }; let table_exists = ctx.table_exist(name.as_str())?; let schema: SchemaRef = Arc::new(schema.as_ref().to_owned().into()); let table_partition_cols = table_partition_cols @@ -398,7 +404,7 @@ impl BallistaContext { options = options.schema(&schema); } self.register_csv(name, location, options).await?; - Ok(Arc::new(DataFrame::new(ctx.state.clone(), &plan))) + Ok(DataFrame::new(ctx.state(), plan)) } "parquet" => { self.register_parquet( @@ -408,7 +414,7 @@ impl BallistaContext { .table_partition_cols(table_partition_cols), ) .await?; - Ok(Arc::new(DataFrame::new(ctx.state.clone(), &plan))) + Ok(DataFrame::new(ctx.state(), plan)) } "avro" => { self.register_avro( @@ -418,16 +424,14 @@ impl BallistaContext { .table_partition_cols(table_partition_cols), ) .await?; - Ok(Arc::new(DataFrame::new(ctx.state.clone(), &plan))) + Ok(DataFrame::new(ctx.state(), plan)) } _ => Err(DataFusionError::NotImplemented(format!( "Unsupported file type {:?}.", file_type ))), }, - (true, true) => { - Ok(Arc::new(DataFrame::new(ctx.state.clone(), &plan))) - } + (true, true) => Ok(DataFrame::new(ctx.state(), plan)), (false, true) => Err(DataFusionError::Execution(format!( "Table '{:?}' already exists", name diff --git a/ballista/core/Cargo.toml b/ballista/core/Cargo.toml index 8aa4e14a1..17dda6e56 100644 --- a/ballista/core/Cargo.toml +++ b/ballista/core/Cargo.toml @@ -46,13 +46,13 @@ simd = ["datafusion/simd"] [dependencies] ahash = { version = "0.8", default-features = false } -arrow-flight = { version = "28.0.0", features = ["flight-sql-experimental"] } +arrow-flight = { version = "29.0.0", features = ["flight-sql-experimental"] } async-trait = "0.1.41" chrono = { version = "0.4", default-features = false } clap = { version = "3", features = ["derive", "cargo"] } -datafusion = "15.0.0" +datafusion = "16.0.0" datafusion-objectstore-hdfs = { version = "0.1.1", default-features = false, optional = true } -datafusion-proto = "15.0.0" +datafusion-proto = "16.0.0" futures = "0.3" hashbrown = "0.13" @@ -68,7 +68,7 @@ prost = "0.11" prost-types = "0.11" rand = "0.8" serde = { version = "1", features = ["derive"] } -sqlparser = "0.27" +sqlparser = "0.30" sys-info = "0.9.0" tokio = "1.0" tokio-stream = { version = "0.1", features = ["net"] } diff --git a/ballista/core/src/error.rs b/ballista/core/src/error.rs index 53d779e15..77a7228c9 100644 --- a/ballista/core/src/error.rs +++ b/ballista/core/src/error.rs @@ -158,14 +158,14 @@ impl From for BallistaError { } } -impl From for BallistaError { - fn from(e: datafusion_proto::from_proto::Error) -> Self { +impl From for BallistaError { + fn from(e: datafusion_proto::logical_plan::to_proto::Error) -> Self { BallistaError::General(e.to_string()) } } -impl From for BallistaError { - fn from(e: datafusion_proto::to_proto::Error) -> Self { +impl From for BallistaError { + fn from(e: datafusion_proto::logical_plan::from_proto::Error) -> Self { BallistaError::General(e.to_string()) } } diff --git a/ballista/core/src/execution_plans/distributed_query.rs b/ballista/core/src/execution_plans/distributed_query.rs index cd56cd369..fdd450e5c 100644 --- a/ballista/core/src/execution_plans/distributed_query.rs +++ b/ballista/core/src/execution_plans/distributed_query.rs @@ -136,10 +136,6 @@ impl ExecutionPlan for DistributedQueryExec { None } - fn relies_on_input_order(&self) -> bool { - false - } - fn children(&self) -> Vec> { vec![] } diff --git a/ballista/core/src/execution_plans/shuffle_reader.rs b/ballista/core/src/execution_plans/shuffle_reader.rs index 97e24fd62..edc44d99c 100644 --- a/ballista/core/src/execution_plans/shuffle_reader.rs +++ b/ballista/core/src/execution_plans/shuffle_reader.rs @@ -98,10 +98,6 @@ impl ExecutionPlan for ShuffleReaderExec { None } - fn relies_on_input_order(&self) -> bool { - false - } - fn children(&self) -> Vec> { vec![] } diff --git a/ballista/core/src/execution_plans/shuffle_writer.rs b/ballista/core/src/execution_plans/shuffle_writer.rs index 475157c3e..da8c32b39 100644 --- a/ballista/core/src/execution_plans/shuffle_writer.rs +++ b/ballista/core/src/execution_plans/shuffle_writer.rs @@ -313,10 +313,6 @@ impl ExecutionPlan for ShuffleWriterExec { None } - fn relies_on_input_order(&self) -> bool { - false - } - fn children(&self) -> Vec> { vec![self.plan.clone()] } diff --git a/ballista/core/src/execution_plans/unresolved_shuffle.rs b/ballista/core/src/execution_plans/unresolved_shuffle.rs index 15d403fb6..fe36134d5 100644 --- a/ballista/core/src/execution_plans/unresolved_shuffle.rs +++ b/ballista/core/src/execution_plans/unresolved_shuffle.rs @@ -81,10 +81,6 @@ impl ExecutionPlan for UnresolvedShuffleExec { None } - fn relies_on_input_order(&self) -> bool { - false - } - fn children(&self) -> Vec> { vec![] } diff --git a/ballista/core/src/serde/generated/ballista.rs b/ballista/core/src/serde/generated/ballista.rs index 8f0bc9bce..d11c6ea58 100644 --- a/ballista/core/src/serde/generated/ballista.rs +++ b/ballista/core/src/serde/generated/ballista.rs @@ -1985,7 +1985,7 @@ pub mod executor_grpc_client { pub mod scheduler_grpc_server { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] use tonic::codegen::*; - ///Generated trait containing gRPC methods that should be implemented for use with SchedulerGrpcServer. + /// Generated trait containing gRPC methods that should be implemented for use with SchedulerGrpcServer. #[async_trait] pub trait SchedulerGrpc: Send + Sync + 'static { /// Executors must poll the scheduler for heartbeat and to receive tasks @@ -2531,7 +2531,7 @@ pub mod scheduler_grpc_server { pub mod executor_grpc_server { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] use tonic::codegen::*; - ///Generated trait containing gRPC methods that should be implemented for use with ExecutorGrpcServer. + /// Generated trait containing gRPC methods that should be implemented for use with ExecutorGrpcServer. #[async_trait] pub trait ExecutorGrpc: Send + Sync + 'static { async fn launch_task( diff --git a/ballista/core/src/serde/mod.rs b/ballista/core/src/serde/mod.rs index b1b2ab386..14333f2dc 100644 --- a/ballista/core/src/serde/mod.rs +++ b/ballista/core/src/serde/mod.rs @@ -262,10 +262,9 @@ mod tests { SendableRecordBatchStream, Statistics, }; use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext}; + use datafusion_proto::logical_plan::from_proto::parse_expr; use prost::Message; use std::any::Any; - - use datafusion_proto::from_proto::parse_expr; use std::convert::TryInto; use std::fmt; use std::fmt::{Debug, Formatter}; @@ -628,13 +627,13 @@ mod tests { let scan = ctx .read_csv("tests/customer.csv", CsvReadOptions::default()) .await? - .to_logical_plan()?; + .into_optimized_plan()?; let topk_plan = LogicalPlan::Extension(Extension { node: Arc::new(TopKPlanNode::new(3, scan, col("revenue"))), }); - let topk_exec = ctx.create_physical_plan(&topk_plan).await?; + let topk_exec = ctx.state().create_physical_plan(&topk_plan).await?; let extension_codec = TopKExtensionCodec {}; diff --git a/ballista/core/src/serde/physical_plan/from_proto.rs b/ballista/core/src/serde/physical_plan/from_proto.rs index eb951c763..11cd5b725 100644 --- a/ballista/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/core/src/serde/physical_plan/from_proto.rs @@ -23,7 +23,6 @@ use std::sync::Arc; use chrono::{TimeZone, Utc}; use datafusion::arrow::datatypes::Schema; -use datafusion::config::ConfigOptions; use datafusion::datasource::listing::{FileRange, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::execution::context::ExecutionProps; @@ -40,10 +39,9 @@ use datafusion::physical_plan::{ functions, Partitioning, }; use datafusion::physical_plan::{ColumnStatistics, PhysicalExpr, Statistics}; -use datafusion_proto::from_proto::from_proto_binary_op; +use datafusion_proto::logical_plan::from_proto::from_proto_binary_op; use object_store::path::Path; use object_store::ObjectMeta; -use parking_lot::RwLock; use crate::serde::protobuf::physical_expr_node::ExprType; @@ -399,7 +397,6 @@ impl TryInto for &protobuf::FileScanExecConf { let statistics = convert_required!(self.statistics)?; Ok(FileScanConfig { - config_options: Arc::new(RwLock::new(ConfigOptions::new())), // TODO add serde object_store_url: ObjectStoreUrl::parse(&self.object_store_url)?, file_schema: schema, file_groups: self @@ -413,6 +410,7 @@ impl TryInto for &protobuf::FileScanExecConf { table_partition_cols: vec![], // TODO add ordering info to the ballista proto file output_ordering: None, + infinite_source: false, }) } } diff --git a/ballista/core/src/serde/physical_plan/mod.rs b/ballista/core/src/serde/physical_plan/mod.rs index cf7e54978..2e1027404 100644 --- a/ballista/core/src/serde/physical_plan/mod.rs +++ b/ballista/core/src/serde/physical_plan/mod.rs @@ -24,7 +24,6 @@ use prost::Message; use datafusion::arrow::compute::SortOptions; use datafusion::arrow::datatypes::SchemaRef; -use datafusion::config::ConfigOptions; use datafusion::datasource::file_format::file_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; @@ -54,8 +53,7 @@ use datafusion::physical_plan::windows::{create_window_expr, WindowAggExec}; use datafusion::physical_plan::{ AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, WindowExpr, }; -use datafusion_proto::from_proto::parse_expr; -use parking_lot::RwLock; +use datafusion_proto::logical_plan::from_proto::parse_expr; use crate::error::BallistaError; use crate::execution_plans::{ @@ -316,7 +314,7 @@ impl AsExecutionPlan for PhysicalPlanNode { &[window_node_expr], &[], &[], - Some(Arc::new(WindowFrame::default())), + Arc::new(WindowFrame::new(false)), &physical_schema, )?) } @@ -566,7 +564,7 @@ impl AsExecutionPlan for PhysicalPlanNode { runtime, extension_codec )?; - Ok(Arc::new(CrossJoinExec::try_new(left, right)?)) + Ok(Arc::new(CrossJoinExec::new(left, right))) } PhysicalPlanType::ShuffleWriter(shuffle_writer) => { let input: Arc = into_physical_plan!( @@ -1268,7 +1266,6 @@ fn decode_scan_config( }; Ok(FileScanConfig { - config_options: Arc::new(RwLock::new(ConfigOptions::new())), // TODO add serde object_store_url, file_schema: schema, file_groups, @@ -1277,6 +1274,7 @@ fn decode_scan_config( limit: proto.limit.as_ref().map(|sl| sl.limit as usize), table_partition_cols: vec![], output_ordering: None, + infinite_source: false, }) } @@ -1300,7 +1298,6 @@ mod roundtrip_tests { use datafusion::arrow::array::ArrayRef; use datafusion::arrow::datatypes::IntervalUnit; - use datafusion::config::ConfigOptions; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::execution::context::ExecutionProps; use datafusion::logical_expr::create_udf; @@ -1338,7 +1335,6 @@ mod roundtrip_tests { use crate::serde::protobuf::PhysicalPlanNode; use crate::serde::{AsExecutionPlan, BallistaCodec}; use datafusion_proto::protobuf::LogicalPlanNode; - use parking_lot::RwLock; use super::super::super::error::Result; use super::super::protobuf; @@ -1584,7 +1580,6 @@ mod roundtrip_tests { #[test] fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { let scan_config = FileScanConfig { - config_options: Arc::new(RwLock::new(ConfigOptions::new())), // TODO add serde object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: Arc::new(Schema::new(vec![Field::new( "col", @@ -1605,6 +1600,7 @@ mod roundtrip_tests { limit: None, table_partition_cols: vec![], output_ordering: None, + infinite_source: false, }; let predicate = datafusion::prelude::col("col").eq(datafusion::prelude::lit("1")); @@ -1673,7 +1669,7 @@ mod roundtrip_tests { let project = ProjectionExec::try_new(vec![(Arc::new(expr), "a".to_string())], input)?; - let mut ctx = SessionContext::new(); + let ctx = SessionContext::new(); ctx.register_udf(udf); diff --git a/ballista/core/src/utils.rs b/ballista/core/src/utils.rs index c83e94aa8..489155d91 100644 --- a/ballista/core/src/utils.rs +++ b/ballista/core/src/utils.rs @@ -319,15 +319,15 @@ pub fn create_df_ctx_with_ballista_query_planner( let session_config = SessionConfig::new() .with_target_partitions(config.default_shuffle_partitions()) .with_information_schema(true); - let mut session_state = SessionState::with_config_rt( + let session_state = SessionState::with_config_rt( session_config, Arc::new( RuntimeEnv::new(with_object_store_provider(RuntimeConfig::default())) .unwrap(), ), ) + .with_session_id(session_id) .with_query_planner(planner); - session_state.session_id = session_id; // the SessionContext created here is the client side context, but the session_id is from server side. SessionContext::with_state(session_state) } @@ -395,7 +395,7 @@ impl QueryPlanner for BallistaQueryPlanner { logical_plan.clone(), self.extension_codec.clone(), self.plan_repr, - session_state.session_id.clone(), + session_state.session_id().to_string(), ))), } } diff --git a/ballista/executor/Cargo.toml b/ballista/executor/Cargo.toml index 12df674f2..4355e711e 100644 --- a/ballista/executor/Cargo.toml +++ b/ballista/executor/Cargo.toml @@ -38,15 +38,15 @@ default = ["mimalloc"] [dependencies] anyhow = "1" -arrow = { version = "28.0.0" } -arrow-flight = { version = "28.0.0" } +arrow = { version = "29.0.0" } +arrow-flight = { version = "29.0.0" } async-trait = "0.1.41" ballista-core = { path = "../core", version = "0.10.0" } chrono = { version = "0.4", default-features = false } configure_me = "0.4.0" dashmap = "5.4.0" -datafusion = "15.0.0" -datafusion-proto = "15.0.0" +datafusion = "16.0.0" +datafusion-proto = "16.0.0" futures = "0.3" hyper = "0.14.4" log = "0.4" diff --git a/ballista/scheduler/Cargo.toml b/ballista/scheduler/Cargo.toml index db8d22a0b..30863c2a9 100644 --- a/ballista/scheduler/Cargo.toml +++ b/ballista/scheduler/Cargo.toml @@ -43,7 +43,7 @@ sled = ["sled_package", "tokio-stream"] [dependencies] anyhow = "1" -arrow-flight = { version = "28.0.0", features = ["flight-sql-experimental"] } +arrow-flight = { version = "29.0.0", features = ["flight-sql-experimental"] } async-recursion = "1.0.0" async-trait = "0.1.41" ballista-core = { path = "../core", version = "0.10.0" } @@ -51,8 +51,9 @@ base64 = { version = "0.13", default-features = false } clap = { version = "3", features = ["derive", "cargo"] } configure_me = "0.4.0" dashmap = "5.4.0" -datafusion = "15.0.0" -datafusion-proto = "15.0.0" +datafusion = "16.0.0" +datafusion-expr = "16.0.0" +datafusion-proto = "16.0.0" etcd-client = { version = "0.10", optional = true } flatbuffers = { version = "22.9.29" } futures = "0.3" @@ -62,7 +63,7 @@ http-body = "0.4" hyper = "0.14.4" itertools = "0.10.3" log = "0.4" -object_store = "0.5.0" +object_store = "0.5.2" once_cell = { version = "1.16.0", optional = true } parking_lot = "0.12" parse_arg = "0.1.3" diff --git a/ballista/scheduler/src/flight_sql.rs b/ballista/scheduler/src/flight_sql.rs index a5a613c44..7067a1894 100644 --- a/ballista/scheduler/src/flight_sql.rs +++ b/ballista/scheduler/src/flight_sql.rs @@ -180,7 +180,7 @@ impl FlightSqlServiceImpl { let plan = ctx .sql(query) .await - .and_then(|df| df.to_logical_plan()) + .and_then(|df| df.into_optimized_plan()) .map_err(|e| Status::internal(format!("Error building plan: {}", e)))?; Ok(plan) } diff --git a/ballista/scheduler/src/planner.rs b/ballista/scheduler/src/planner.rs index 4265d5bd1..e31ad1c45 100644 --- a/ballista/scheduler/src/planner.rs +++ b/ballista/scheduler/src/planner.rs @@ -360,9 +360,9 @@ mod test { ) .await?; - let plan = df.to_logical_plan()?; - let plan = ctx.optimize(&plan)?; - let plan = ctx.create_physical_plan(&plan).await?; + let plan = df.into_optimized_plan()?; + let plan = ctx.state().optimize(&plan)?; + let plan = ctx.state().create_physical_plan(&plan).await?; let mut planner = DistributedPlanner::new(); let job_uuid = Uuid::new_v4(); @@ -472,9 +472,9 @@ order by ) .await?; - let plan = df.to_logical_plan()?; - let plan = ctx.optimize(&plan)?; - let plan = ctx.create_physical_plan(&plan).await?; + let plan = df.into_optimized_plan()?; + let plan = ctx.state().optimize(&plan)?; + let plan = ctx.state().create_physical_plan(&plan).await?; let mut planner = DistributedPlanner::new(); let job_uuid = Uuid::new_v4(); @@ -618,9 +618,9 @@ order by ) .await?; - let plan = df.to_logical_plan()?; - let plan = ctx.optimize(&plan)?; - let plan = ctx.create_physical_plan(&plan).await?; + let plan = df.into_optimized_plan()?; + let plan = ctx.state().optimize(&plan)?; + let plan = ctx.state().create_physical_plan(&plan).await?; let mut planner = DistributedPlanner::new(); let job_uuid = Uuid::new_v4(); diff --git a/ballista/scheduler/src/scheduler_server/grpc.rs b/ballista/scheduler/src/scheduler_server/grpc.rs index ad04efb21..0863bf735 100644 --- a/ballista/scheduler/src/scheduler_server/grpc.rs +++ b/ballista/scheduler/src/scheduler_server/grpc.rs @@ -44,7 +44,7 @@ use std::ops::Deref; use std::sync::Arc; use crate::scheduler_server::event::QueryStageSchedulerEvent; -use datafusion::prelude::SessionConfig; +use datafusion::prelude::SessionContext; use std::time::{SystemTime, UNIX_EPOCH}; use tonic::{Request, Response, Status}; @@ -294,9 +294,8 @@ impl SchedulerGrpc let GetFileMetadataParams { path, file_type } = request.into_inner(); // Here, we use the default config, since we don't know the session id - let config = SessionConfig::default().config_options(); let file_format: Arc = match file_type.as_str() { - "parquet" => Ok(Arc::new(ParquetFormat::new(config))), + "parquet" => Ok(Arc::new(ParquetFormat::new())), // TODO implement for CSV _ => Err(tonic::Status::unimplemented( "get_file_metadata unsupported file type", @@ -320,8 +319,10 @@ impl SchedulerGrpc tonic::Status::internal(msg) })?; + let session = SessionContext::new(); + let ctx = session.state(); let schema = file_format - .infer_schema(&obj_store, &file_metas) + .infer_schema(&ctx, &obj_store, &file_metas) .await .map_err(|e| { let msg = format!("Error inferring schema: {}", e); @@ -408,7 +409,7 @@ impl SchedulerGrpc Query::Sql(sql) => session_ctx .sql(&sql) .await - .and_then(|df| df.to_logical_plan()) + .and_then(|df| df.into_optimized_plan()) .map_err(|e| { let msg = format!("Error parsing SQL: {}", e); error!("{}", msg); diff --git a/ballista/scheduler/src/scheduler_server/mod.rs b/ballista/scheduler/src/scheduler_server/mod.rs index f39e0d1b4..5631a9f09 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -613,7 +613,10 @@ mod test { ctx.register_table("explode", Arc::new(ExplodingTableProvider))?; - let plan = ctx.sql("SELECT * FROM explode").await?.to_logical_plan()?; + let plan = ctx + .sql("SELECT * FROM explode") + .await? + .into_optimized_plan()?; // This should fail when we try and create the physical plan let status = test.run("job", "", &plan).await?; diff --git a/ballista/scheduler/src/state/execution_graph.rs b/ballista/scheduler/src/state/execution_graph.rs index 976ec5f51..7e4531af6 100644 --- a/ballista/scheduler/src/state/execution_graph.rs +++ b/ballista/scheduler/src/state/execution_graph.rs @@ -1590,10 +1590,11 @@ mod test { use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::logical_expr::JoinType; - use datafusion::logical_expr::{col, count, sum, Expr}; + use datafusion::logical_expr::{col, count, sum}; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion::test_util::scan_empty; + use datafusion_expr::Expr; use crate::scheduler_server::event::QueryStageSchedulerEvent; use ballista_core::error::Result; @@ -2773,9 +2774,13 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = ctx.state().optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = ctx + .state() + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2801,9 +2806,13 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = ctx.state().optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = ctx + .state() + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2826,9 +2835,13 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = ctx.state().optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = ctx + .state() + .create_physical_plan(&optimized_plan) + .await + .unwrap(); ExecutionGraph::new("localhost:50050", "job", "", "session", plan, 0).unwrap() } @@ -2849,14 +2862,14 @@ mod test { .build() .unwrap(); - let sort_expr = Expr::Sort { + let sort_expr = Expr::Sort(datafusion_expr::expr::Sort { expr: Box::new(col("id")), asc: false, nulls_first: false, - }; + }); let logical_plan = left_plan - .join(&right_plan, JoinType::Inner, (vec!["id"], vec!["id"]), None) + .join(right_plan, JoinType::Inner, (vec!["id"], vec!["id"]), None) .unwrap() .aggregate(vec![col("id")], vec![sum(col("gmv"))]) .unwrap() @@ -2865,9 +2878,13 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = ctx.state().optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = ctx + .state() + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2887,12 +2904,16 @@ mod test { .sql("SELECT 1 as NUMBER union all SELECT 1 as NUMBER;") .await .unwrap() - .to_logical_plan() + .into_optimized_plan() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = ctx.state().optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = ctx + .state() + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2912,12 +2933,16 @@ mod test { .sql("SELECT 1 as NUMBER union SELECT 1 as NUMBER;") .await .unwrap() - .to_logical_plan() + .into_optimized_plan() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = ctx.state().optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = ctx + .state() + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); diff --git a/ballista/scheduler/src/state/execution_graph/execution_stage.rs b/ballista/scheduler/src/state/execution_graph/execution_stage.rs index b187d8e91..1a5d92586 100644 --- a/ballista/scheduler/src/state/execution_graph/execution_stage.rs +++ b/ballista/scheduler/src/state/execution_graph/execution_stage.rs @@ -364,7 +364,8 @@ impl UnresolvedStage { // Optimize join order based on new resolved statistics let optimize_join = JoinSelection::new(); - let plan = optimize_join.optimize(plan, &SessionConfig::new())?; + let plan = + optimize_join.optimize(plan, SessionConfig::new().config_options())?; Ok(ResolvedStage::new( self.stage_id, diff --git a/ballista/scheduler/src/state/execution_graph_dot.rs b/ballista/scheduler/src/state/execution_graph_dot.rs index 6e65612fc..6955a82e2 100644 --- a/ballista/scheduler/src/state/execution_graph_dot.rs +++ b/ballista/scheduler/src/state/execution_graph_dot.rs @@ -646,8 +646,8 @@ filter_expr="] let df = ctx .sql("SELECT * FROM foo JOIN bar ON foo.a = bar.a JOIN baz on bar.b = baz.b") .await?; - let plan = df.to_logical_plan()?; - let plan = ctx.create_physical_plan(&plan).await?; + let plan = df.into_optimized_plan()?; + let plan = ctx.state().create_physical_plan(&plan).await?; ExecutionGraph::new("scheduler_id", "job_id", "job_name", "session_id", plan, 0) } @@ -665,8 +665,8 @@ filter_expr="] let df = ctx .sql("SELECT * FROM foo JOIN bar ON foo.a = bar.a JOIN baz on bar.a = baz.a") .await?; - let plan = df.to_logical_plan()?; - let plan = ctx.create_physical_plan(&plan).await?; + let plan = df.into_optimized_plan()?; + let plan = ctx.state().create_physical_plan(&plan).await?; ExecutionGraph::new("scheduler_id", "job_id", "job_name", "session_id", plan, 0) } } diff --git a/ballista/scheduler/src/state/mod.rs b/ballista/scheduler/src/state/mod.rs index 99791457b..974d4b2ec 100644 --- a/ballista/scheduler/src/state/mod.rs +++ b/ballista/scheduler/src/state/mod.rs @@ -338,7 +338,7 @@ impl SchedulerState= log::Level::Debug { // optimizing the plan here is redundant because the physical planner will do this again // but it is helpful to see what the optimized plan will be - let optimized_plan = session_ctx.optimize(plan)?; + let optimized_plan = session_ctx.state().optimize(plan)?; debug!("Optimized plan: {}", optimized_plan.display_indent()); } @@ -391,7 +391,7 @@ impl SchedulerState, - ballista_config: &BallistaConfig, -) -> Arc { - { - let mut mut_state = session_ctx.state.write(); - // TODO Currently we have to start from default session config due to the interface not support update - let config = SessionConfig::default() - .with_target_partitions(ballista_config.default_shuffle_partitions()) - .with_batch_size(ballista_config.default_batch_size()) - .with_repartition_joins(ballista_config.repartition_joins()) - .with_repartition_aggregations(ballista_config.repartition_aggregations()) - .with_repartition_windows(ballista_config.repartition_windows()) - .with_parquet_pruning(ballista_config.parquet_pruning()); - let config = propagate_ballista_configs(config, ballista_config); - mut_state.config = config; - } - session_ctx -} - fn propagate_ballista_configs( config: SessionConfig, ballista_config: &BallistaConfig, diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index 27121d219..dd350b8c8 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -34,8 +34,8 @@ snmalloc = ["snmalloc-rs"] [dependencies] ballista = { path = "../ballista/client", version = "0.10.0" } -datafusion = "15.0.0" -datafusion-proto = "15.0.0" +datafusion = "16.0.0" +datafusion-proto = "16.0.0" env_logger = "0.10" futures = "0.3" mimalloc = { version = "0.1", optional = true, default-features = false } diff --git a/benchmarks/src/bin/nyctaxi.rs b/benchmarks/src/bin/nyctaxi.rs index e22c71e5e..917022cfa 100644 --- a/benchmarks/src/bin/nyctaxi.rs +++ b/benchmarks/src/bin/nyctaxi.rs @@ -119,12 +119,12 @@ async fn datafusion_sql_benchmarks( } async fn execute_sql(ctx: &SessionContext, sql: &str, debug: bool) -> Result<()> { - let plan = ctx.create_logical_plan(sql)?; - let plan = ctx.optimize(&plan)?; + let plan = ctx.state().create_logical_plan(sql).await?; + let plan = ctx.state().optimize(&plan)?; if debug { println!("Optimized logical plan:\n{:?}", plan); } - let physical_plan = ctx.create_physical_plan(&plan).await?; + let physical_plan = ctx.state().create_physical_plan(&plan).await?; let task_ctx = ctx.task_ctx(); let result = collect(physical_plan, task_ctx).await?; if debug { diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index aabfea6ff..0a86728bd 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -293,7 +293,7 @@ async fn benchmark_datafusion(opt: DataFusionBenchmarkOpt) -> Result Result = Vec::with_capacity(1); for i in 0..opt.iterations { let start = Instant::now(); - let plans = create_logical_plans(&ctx, opt.query)?; + let plans = create_logical_plans(&ctx, opt.query).await?; for plan in plans { result = execute_query(&ctx, &plan, opt.debug).await?; } @@ -394,9 +394,8 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { .await .map_err(|e| DataFusionError::Plan(format!("{:?}", e))) .unwrap(); - let plan = df.to_logical_plan()?; if opt.debug { - println!("=== Optimized logical plan ===\n{:?}\n", plan); + println!("=== Optimized logical plan ===\n{:?}\n", df.logical_plan()); } batches = df .collect() @@ -676,11 +675,17 @@ fn get_query_sql(query: usize) -> Result> { } /// Create a logical plan for each query in the specified query file -fn create_logical_plans(ctx: &SessionContext, query: usize) -> Result> { +async fn create_logical_plans( + ctx: &SessionContext, + query: usize, +) -> Result> { let sql = get_query_sql(query)?; - sql.iter() - .map(|sql| ctx.create_logical_plan(sql.as_str())) - .collect::>>() + let mut plans = vec![]; + for sql in sql.iter() { + let plan = ctx.state().create_logical_plan(sql.as_str()).await?; + plans.push(plan); + } + Ok(plans) } async fn execute_query( @@ -691,11 +696,11 @@ async fn execute_query( if debug { println!("=== Logical plan ===\n{:?}\n", plan); } - let plan = ctx.optimize(plan)?; + let plan = ctx.state().optimize(plan)?; if debug { println!("=== Optimized logical plan ===\n{:?}\n", plan); } - let physical_plan = ctx.create_physical_plan(&plan).await?; + let physical_plan = ctx.state().create_physical_plan(&plan).await?; if debug { println!( "=== Physical plan ===\n{}\n", @@ -740,9 +745,9 @@ async fn convert_tbl(opt: ConvertOpt) -> Result<()> { } // create the physical plan - let csv = csv.to_logical_plan()?; - let csv = ctx.optimize(&csv)?; - let csv = ctx.create_physical_plan(&csv).await?; + let csv = csv.into_optimized_plan()?; + let csv = ctx.state().optimize(&csv)?; + let csv = ctx.state().create_physical_plan(&csv).await?; let output_path = output_root_path.join(table); let output_path = output_path.to_str().unwrap().to_owned(); @@ -816,8 +821,7 @@ async fn get_table( } "parquet" => { let path = format!("{}/{}", path, table); - let format = ParquetFormat::new(ctx.config_options()) - .with_enable_pruning(Some(true)); + let format = ParquetFormat::new().with_enable_pruning(Some(true)); (Arc::new(format), path, DEFAULT_PARQUET_EXTENSION) } @@ -834,6 +838,7 @@ async fn get_table( collect_stat: true, table_partition_cols: vec![], file_sort_order: None, + infinite_source: false, }; let url = ListingTableUrl::parse(path)?; @@ -1502,7 +1507,7 @@ mod tests { ctx.register_table(table, Arc::new(provider))?; } - let plans = create_logical_plans(&ctx, n)?; + let plans = create_logical_plans(&ctx, n).await?; for plan in plans { execute_query(&ctx, &plan, false).await?; } @@ -1579,10 +1584,10 @@ mod tests { } // test logical plan round trip - let plans = create_logical_plans(&ctx, n)?; + let plans = create_logical_plans(&ctx, n).await?; for plan in plans { // test optimized logical plan round trip - let plan = ctx.optimize(&plan)?; + let plan = ctx.state().optimize(&plan)?; let proto: datafusion_proto::protobuf::LogicalPlanNode = datafusion_proto::protobuf::LogicalPlanNode::try_from_logical_plan( &plan, @@ -1600,7 +1605,7 @@ mod tests { // test physical plan roundtrip if env::var("TPCH_DATA").is_ok() { - let physical_plan = ctx.create_physical_plan(&plan).await?; + let physical_plan = ctx.state().create_physical_plan(&plan).await?; let proto: protobuf::PhysicalPlanNode = protobuf::PhysicalPlanNode::try_from_physical_plan( physical_plan.clone(), diff --git a/examples/Cargo.toml b/examples/Cargo.toml index 06aa536d6..9b3c65231 100644 --- a/examples/Cargo.toml +++ b/examples/Cargo.toml @@ -35,7 +35,7 @@ required-features = ["ballista/standalone"] [dependencies] ballista = { path = "../ballista/client", version = "0.10.0" } -datafusion = "15.0.0" +datafusion = "16.0.0" futures = "0.3" num_cpus = "1.13.0" prost = "0.11" diff --git a/python/Cargo.toml b/python/Cargo.toml index 5879d7443..9bc34137d 100644 --- a/python/Cargo.toml +++ b/python/Cargo.toml @@ -36,7 +36,7 @@ default = ["mimalloc"] [dependencies] async-trait = "0.1" ballista = { path = "../ballista/client", version = "0.10.0" } -datafusion = { version = "15.0.0", features = ["pyarrow"] } +datafusion = { version = "16.0.0", features = ["pyarrow"] } futures = "0.3" mimalloc = { version = "*", optional = true, default-features = false } pyo3 = { version = "~0.17.1", features = ["extension-module", "abi3", "abi3-py37"] } From 22315501b4eaaef7fd725efe8390f2bf4f99698c Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 12:58:30 -0700 Subject: [PATCH 02/13] lint --- ballista/client/src/context.rs | 1 + .../scheduler/src/state/execution_graph/execution_stage.rs | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ballista/client/src/context.rs b/ballista/client/src/context.rs index cd4c94a41..698f3f4dd 100644 --- a/ballista/client/src/context.rs +++ b/ballista/client/src/context.rs @@ -597,6 +597,7 @@ mod tests { collect_stat: x.collect_stat, target_partitions: x.target_partitions, file_sort_order: None, + infinite_source: false, }; let table_paths = listing_table diff --git a/ballista/scheduler/src/state/execution_graph/execution_stage.rs b/ballista/scheduler/src/state/execution_graph/execution_stage.rs index 1a5d92586..ed98286db 100644 --- a/ballista/scheduler/src/state/execution_graph/execution_stage.rs +++ b/ballista/scheduler/src/state/execution_graph/execution_stage.rs @@ -364,8 +364,7 @@ impl UnresolvedStage { // Optimize join order based on new resolved statistics let optimize_join = JoinSelection::new(); - let plan = - optimize_join.optimize(plan, SessionConfig::new().config_options())?; + let plan = optimize_join.optimize(plan, SessionConfig::new().config_options())?; Ok(ResolvedStage::new( self.stage_id, From 3ac91524764d284f51d1e499858e13b11146bd68 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 13:50:53 -0700 Subject: [PATCH 03/13] Update test fixtures --- .../src/state/execution_graph_dot.rs | 147 ++++++++---------- 1 file changed, 68 insertions(+), 79 deletions(-) diff --git a/ballista/scheduler/src/state/execution_graph_dot.rs b/ballista/scheduler/src/state/execution_graph_dot.rs index 6955a82e2..13a5af012 100644 --- a/ballista/scheduler/src/state/execution_graph_dot.rs +++ b/ballista/scheduler/src/state/execution_graph_dot.rs @@ -433,7 +433,8 @@ mod tests { let dot = ExecutionGraphDot::generate(Arc::new(graph)) .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; - let expected = r#"digraph G { + let expected = r#" +digraph G { subgraph cluster0 { label = "Stage 1 [Resolved]"; stage_1_0 [shape=box, label="ShuffleWriter [0 partitions]"] @@ -449,15 +450,15 @@ mod tests { subgraph cluster2 { label = "Stage 3 [Unresolved]"; stage_3_0 [shape=box, label="ShuffleWriter [48 partitions]"] - stage_3_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_3_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_3_0_0_0 [shape=box, label="HashJoin join_expr=a@0 = a@0 filter_expr="] - stage_3_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_3_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_3_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=1]"] stage_3_0_0_0_0_0 -> stage_3_0_0_0_0 stage_3_0_0_0_0 -> stage_3_0_0_0 - stage_3_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_3_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_3_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=2]"] stage_3_0_0_0_1_0 -> stage_3_0_0_0_1 stage_3_0_0_0_1 -> stage_3_0_0_0 @@ -474,15 +475,15 @@ filter_expr="] label = "Stage 5 [Unresolved]"; stage_5_0 [shape=box, label="ShuffleWriter [48 partitions]"] stage_5_0_0 [shape=box, label="Projection: a@0, b@1, a@2, b@3, a@4, b@5"] - stage_5_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_5_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_5_0_0_0_0 [shape=box, label="HashJoin join_expr=b@3 = b@1 filter_expr="] - stage_5_0_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_5_0_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_5_0_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=3]"] stage_5_0_0_0_0_0_0 -> stage_5_0_0_0_0_0 stage_5_0_0_0_0_0 -> stage_5_0_0_0_0 - stage_5_0_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_5_0_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_5_0_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=4]"] stage_5_0_0_0_0_1_0 -> stage_5_0_0_0_0_1 stage_5_0_0_0_0_1 -> stage_5_0_0_0_0 @@ -495,8 +496,8 @@ filter_expr="] stage_3_0 -> stage_5_0_0_0_0_0_0 stage_4_0 -> stage_5_0_0_0_0_1_0 } -"#; - assert_eq!(expected, &dot); + "#.trim(); + assert_eq!(dot.trim(), expected); Ok(()) } @@ -506,25 +507,26 @@ filter_expr="] let dot = ExecutionGraphDot::generate_for_query_stage(Arc::new(graph), 3) .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; - let expected = r#"digraph G { + let expected = r#" +digraph G { stage_3_0 [shape=box, label="ShuffleWriter [48 partitions]"] - stage_3_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_3_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_3_0_0_0 [shape=box, label="HashJoin join_expr=a@0 = a@0 filter_expr="] - stage_3_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_3_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_3_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=1]"] stage_3_0_0_0_0_0 -> stage_3_0_0_0_0 stage_3_0_0_0_0 -> stage_3_0_0_0 - stage_3_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=4096]"] + stage_3_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=8192]"] stage_3_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=2]"] stage_3_0_0_0_1_0 -> stage_3_0_0_0_1 stage_3_0_0_0_1 -> stage_3_0_0_0 stage_3_0_0_0 -> stage_3_0_0 stage_3_0_0 -> stage_3_0 } -"#; - assert_eq!(expected, &dot); + "#.trim(); + assert_eq!(dot.trim(), expected); Ok(()) } @@ -534,7 +536,8 @@ filter_expr="] let dot = ExecutionGraphDot::generate(Arc::new(graph)) .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; - let expected = r#"digraph G { + let expected = r#" +digraph G { subgraph cluster0 { label = "Stage 1 [Resolved]"; stage_1_0 [shape=box, label="ShuffleWriter [0 partitions]"] @@ -548,47 +551,56 @@ filter_expr="] stage_2_0_0 -> stage_2_0 } subgraph cluster2 { - label = "Stage 3 [Resolved]"; - stage_3_0 [shape=box, label="ShuffleWriter [0 partitions]"] - stage_3_0_0 [shape=box, label="MemoryExec"] + label = "Stage 3 [Unresolved]"; + stage_3_0 [shape=box, label="ShuffleWriter [48 partitions]"] + stage_3_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_3_0_0_0 [shape=box, label="HashJoin +join_expr=a@0 = a@0 +filter_expr="] + stage_3_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_3_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=1]"] + stage_3_0_0_0_0_0 -> stage_3_0_0_0_0 + stage_3_0_0_0_0 -> stage_3_0_0_0 + stage_3_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_3_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=2]"] + stage_3_0_0_0_1_0 -> stage_3_0_0_0_1 + stage_3_0_0_0_1 -> stage_3_0_0_0 + stage_3_0_0_0 -> stage_3_0_0 stage_3_0_0 -> stage_3_0 } subgraph cluster3 { - label = "Stage 4 [Unresolved]"; - stage_4_0 [shape=box, label="ShuffleWriter [48 partitions]"] - stage_4_0_0 [shape=box, label="Projection: a@0, a@1, a@2"] - stage_4_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0 [shape=box, label="HashJoin + label = "Stage 4 [Resolved]"; + stage_4_0 [shape=box, label="ShuffleWriter [0 partitions]"] + stage_4_0_0 [shape=box, label="MemoryExec"] + stage_4_0_0 -> stage_4_0 + } + subgraph cluster4 { + label = "Stage 5 [Unresolved]"; + stage_5_0 [shape=box, label="ShuffleWriter [48 partitions]"] + stage_5_0_0 [shape=box, label="Projection: a@0, a@1, a@2"] + stage_5_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_5_0_0_0_0 [shape=box, label="HashJoin join_expr=a@1 = a@0 filter_expr="] - stage_4_0_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_0_0 [shape=box, label="HashJoin -join_expr=a@0 = a@0 -filter_expr="] - stage_4_0_0_0_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=1]"] - stage_4_0_0_0_0_0_0_0_0 -> stage_4_0_0_0_0_0_0_0 - stage_4_0_0_0_0_0_0_0 -> stage_4_0_0_0_0_0_0 - stage_4_0_0_0_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=2]"] - stage_4_0_0_0_0_0_0_1_0 -> stage_4_0_0_0_0_0_0_1 - stage_4_0_0_0_0_0_0_1 -> stage_4_0_0_0_0_0_0 - stage_4_0_0_0_0_0_0 -> stage_4_0_0_0_0_0 - stage_4_0_0_0_0_0 -> stage_4_0_0_0_0 - stage_4_0_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=3]"] - stage_4_0_0_0_0_1_0 -> stage_4_0_0_0_0_1 - stage_4_0_0_0_0_1 -> stage_4_0_0_0_0 - stage_4_0_0_0_0 -> stage_4_0_0_0 - stage_4_0_0_0 -> stage_4_0_0 - stage_4_0_0 -> stage_4_0 + stage_5_0_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_5_0_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=3]"] + stage_5_0_0_0_0_0_0 -> stage_5_0_0_0_0_0 + stage_5_0_0_0_0_0 -> stage_5_0_0_0_0 + stage_5_0_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_5_0_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=4]"] + stage_5_0_0_0_0_1_0 -> stage_5_0_0_0_0_1 + stage_5_0_0_0_0_1 -> stage_5_0_0_0_0 + stage_5_0_0_0_0 -> stage_5_0_0_0 + stage_5_0_0_0 -> stage_5_0_0 + stage_5_0_0 -> stage_5_0 } - stage_1_0 -> stage_4_0_0_0_0_0_0_0_0 - stage_2_0 -> stage_4_0_0_0_0_0_0_1_0 - stage_3_0 -> stage_4_0_0_0_0_1_0 + stage_1_0 -> stage_3_0_0_0_0_0 + stage_2_0 -> stage_3_0_0_0_1_0 + stage_3_0 -> stage_5_0_0_0_0_0_0 + stage_4_0 -> stage_5_0_0_0_0_1_0 } -"#; - assert_eq!(expected, &dot); +"#.trim(); + assert_eq!(dot.trim(), expected); Ok(()) } @@ -598,37 +610,14 @@ filter_expr="] let dot = ExecutionGraphDot::generate_for_query_stage(Arc::new(graph), 4) .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; - let expected = r#"digraph G { - stage_4_0 [shape=box, label="ShuffleWriter [48 partitions]"] - stage_4_0_0 [shape=box, label="Projection: a@0, a@1, a@2"] - stage_4_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0 [shape=box, label="HashJoin -join_expr=a@1 = a@0 -filter_expr="] - stage_4_0_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_0_0 [shape=box, label="HashJoin -join_expr=a@0 = a@0 -filter_expr="] - stage_4_0_0_0_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=1]"] - stage_4_0_0_0_0_0_0_0_0 -> stage_4_0_0_0_0_0_0_0 - stage_4_0_0_0_0_0_0_0 -> stage_4_0_0_0_0_0_0 - stage_4_0_0_0_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=2]"] - stage_4_0_0_0_0_0_0_1_0 -> stage_4_0_0_0_0_0_0_1 - stage_4_0_0_0_0_0_0_1 -> stage_4_0_0_0_0_0_0 - stage_4_0_0_0_0_0_0 -> stage_4_0_0_0_0_0 - stage_4_0_0_0_0_0 -> stage_4_0_0_0_0 - stage_4_0_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=4096]"] - stage_4_0_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=3]"] - stage_4_0_0_0_0_1_0 -> stage_4_0_0_0_0_1 - stage_4_0_0_0_0_1 -> stage_4_0_0_0_0 - stage_4_0_0_0_0 -> stage_4_0_0_0 - stage_4_0_0_0 -> stage_4_0_0 + let expected = r#" +digraph G { + stage_4_0 [shape=box, label="ShuffleWriter [0 partitions]"] + stage_4_0_0 [shape=box, label="MemoryExec"] stage_4_0_0 -> stage_4_0 } -"#; - assert_eq!(expected, &dot); +"#.trim(); + assert_eq!(dot.trim(), expected); Ok(()) } From d32f368132e82e61adf24e35bb69519767becd79 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 13:52:22 -0700 Subject: [PATCH 04/13] lint --- ballista/scheduler/src/state/execution_graph_dot.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/ballista/scheduler/src/state/execution_graph_dot.rs b/ballista/scheduler/src/state/execution_graph_dot.rs index 13a5af012..185d0d578 100644 --- a/ballista/scheduler/src/state/execution_graph_dot.rs +++ b/ballista/scheduler/src/state/execution_graph_dot.rs @@ -496,7 +496,8 @@ filter_expr="] stage_3_0 -> stage_5_0_0_0_0_0_0 stage_4_0 -> stage_5_0_0_0_0_1_0 } - "#.trim(); + "# + .trim(); assert_eq!(dot.trim(), expected); Ok(()) } @@ -525,7 +526,8 @@ filter_expr="] stage_3_0_0_0 -> stage_3_0_0 stage_3_0_0 -> stage_3_0 } - "#.trim(); + "# + .trim(); assert_eq!(dot.trim(), expected); Ok(()) } @@ -599,7 +601,8 @@ filter_expr="] stage_3_0 -> stage_5_0_0_0_0_0_0 stage_4_0 -> stage_5_0_0_0_0_1_0 } -"#.trim(); +"# + .trim(); assert_eq!(dot.trim(), expected); Ok(()) } @@ -616,7 +619,8 @@ digraph G { stage_4_0_0 [shape=box, label="MemoryExec"] stage_4_0_0 -> stage_4_0 } -"#.trim(); +"# + .trim(); assert_eq!(dot.trim(), expected); Ok(()) } From b1251f66eeb29e373ad48276d0130b19471a8fa3 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 14:50:32 -0700 Subject: [PATCH 05/13] New clippy --- ballista-cli/src/command.rs | 6 +- ballista-cli/src/exec.rs | 12 +-- ballista-cli/src/main.rs | 10 +- ballista/client/src/context.rs | 14 ++- ballista/core/build.rs | 4 +- ballista/core/src/client.rs | 24 +++-- ballista/core/src/config.rs | 13 ++- ballista/core/src/error.rs | 35 ++++--- ballista/core/src/event_loop.rs | 7 +- .../src/execution_plans/distributed_query.rs | 26 +++--- .../src/execution_plans/shuffle_reader.rs | 16 +--- .../src/execution_plans/shuffle_writer.rs | 13 ++- ballista/core/src/lib.rs | 2 +- ballista/core/src/plugin/plugin_manager.rs | 2 +- ballista/core/src/plugin/udf.rs | 6 +- ballista/core/src/serde/mod.rs | 24 ++--- .../src/serde/physical_plan/from_proto.rs | 10 +- ballista/core/src/serde/physical_plan/mod.rs | 41 +++----- .../core/src/serde/physical_plan/to_proto.rs | 6 +- ballista/core/src/serde/scheduler/to_proto.rs | 3 +- ballista/core/src/utils.rs | 11 +-- ballista/executor/build.rs | 2 +- ballista/executor/src/collect.rs | 2 +- ballista/executor/src/execution_loop.rs | 3 +- ballista/executor/src/executor_process.rs | 9 +- ballista/executor/src/executor_server.rs | 12 +-- ballista/executor/src/flight_service.rs | 11 +-- ballista/scheduler/build.rs | 4 +- ballista/scheduler/src/bin/main.rs | 7 +- ballista/scheduler/src/display.rs | 2 +- ballista/scheduler/src/flight_sql.rs | 66 ++++++------- ballista/scheduler/src/metrics/prometheus.rs | 16 ++-- ballista/scheduler/src/planner.rs | 7 +- .../scheduler/src/scheduler_server/grpc.rs | 49 +++++----- .../scheduler/src/scheduler_server/mod.rs | 17 ++-- .../scheduler_server/query_stage_scheduler.rs | 6 +- .../scheduler/src/state/backend/cluster.rs | 33 ++----- ballista/scheduler/src/state/backend/etcd.rs | 14 +-- .../scheduler/src/state/backend/memory.rs | 14 +-- ballista/scheduler/src/state/backend/sled.rs | 38 ++++---- .../scheduler/src/state/execution_graph.rs | 42 ++++----- .../state/execution_graph/execution_stage.rs | 2 +- .../src/state/execution_graph_dot.rs | 46 +++++---- .../scheduler/src/state/executor_manager.rs | 19 ++-- ballista/scheduler/src/state/mod.rs | 9 +- ballista/scheduler/src/state/task_manager.rs | 5 +- ballista/scheduler/src/test_utils.rs | 16 ++-- benchmarks/src/bin/nyctaxi.rs | 8 +- benchmarks/src/bin/tpch.rs | 93 ++++++++----------- 49 files changed, 361 insertions(+), 476 deletions(-) diff --git a/ballista-cli/src/command.rs b/ballista-cli/src/command.rs index 9998d0c2d..8708c7349 100644 --- a/ballista-cli/src/command.rs +++ b/ballista-cli/src/command.rs @@ -67,7 +67,7 @@ impl Command { .map_err(BallistaError::DataFusionError) } Self::DescribeTable(name) => { - let df = ctx.sql(&format!("SHOW COLUMNS FROM {}", name)).await?; + let df = ctx.sql(&format!("SHOW COLUMNS FROM {name}")).await?; let batches = df.collect().await?; print_options .print_batches(&batches, now) @@ -97,10 +97,10 @@ impl Command { Self::SearchFunctions(function) => { if let Ok(func) = function.parse::() { let details = func.function_details()?; - println!("{}", details); + println!("{details}"); Ok(()) } else { - let msg = format!("{} is not a supported function", function); + let msg = format!("{function} is not a supported function"); Err(BallistaError::NotImplemented(msg)) } } diff --git a/ballista-cli/src/exec.rs b/ballista-cli/src/exec.rs index a01ceadf9..4311cf2da 100644 --- a/ballista-cli/src/exec.rs +++ b/ballista-cli/src/exec.rs @@ -51,7 +51,7 @@ pub async fn exec_from_lines( if line.ends_with(';') { match exec_and_print(ctx, print_options, query).await { Ok(_) => {} - Err(err) => println!("{:?}", err), + Err(err) => println!("{err:?}"), } query = "".to_owned(); } else { @@ -68,7 +68,7 @@ pub async fn exec_from_lines( if !query.is_empty() { match exec_and_print(ctx, print_options, query).await { Ok(_) => {} - Err(err) => println!("{:?}", err), + Err(err) => println!("{err:?}"), } } } @@ -110,7 +110,7 @@ pub async fn exec_from_repl(ctx: &BallistaContext, print_options: &mut PrintOpti if let Err(e) = command.execute(&mut print_options).await { - eprintln!("{}", e) + eprintln!("{e}") } } else { eprintln!( @@ -124,7 +124,7 @@ pub async fn exec_from_repl(ctx: &BallistaContext, print_options: &mut PrintOpti } _ => { if let Err(e) = cmd.execute(ctx, &mut print_options).await { - eprintln!("{}", e) + eprintln!("{e}") } } } @@ -136,7 +136,7 @@ pub async fn exec_from_repl(ctx: &BallistaContext, print_options: &mut PrintOpti rl.add_history_entry(line.trim_end()); match exec_and_print(ctx, &print_options, line).await { Ok(_) => {} - Err(err) => eprintln!("{:?}", err), + Err(err) => eprintln!("{err:?}"), } } Err(ReadlineError::Interrupted) => { @@ -148,7 +148,7 @@ pub async fn exec_from_repl(ctx: &BallistaContext, print_options: &mut PrintOpti break; } Err(err) => { - eprintln!("Unknown error happened {:?}", err); + eprintln!("Unknown error happened {err:?}"); break; } } diff --git a/ballista-cli/src/main.rs b/ballista-cli/src/main.rs index 5cfd019f2..3f8f9ba87 100644 --- a/ballista-cli/src/main.rs +++ b/ballista-cli/src/main.rs @@ -96,7 +96,7 @@ pub async fn main() -> Result<()> { let args = Args::parse(); if !args.quiet { - println!("Ballista CLI v{}", BALLISTA_CLI_VERSION); + println!("Ballista CLI v{BALLISTA_CLI_VERSION}"); } if let Some(ref path) = args.data_path { @@ -166,7 +166,7 @@ fn is_valid_file(dir: &str) -> std::result::Result<(), String> { if Path::new(dir).is_file() { Ok(()) } else { - Err(format!("Invalid file '{}'", dir)) + Err(format!("Invalid file '{dir}'")) } } @@ -174,20 +174,20 @@ fn is_valid_data_dir(dir: &str) -> std::result::Result<(), String> { if Path::new(dir).is_dir() { Ok(()) } else { - Err(format!("Invalid data directory '{}'", dir)) + Err(format!("Invalid data directory '{dir}'")) } } fn is_valid_batch_size(size: &str) -> std::result::Result<(), String> { match size.parse::() { Ok(size) if size > 0 => Ok(()), - _ => Err(format!("Invalid batch size '{}'", size)), + _ => Err(format!("Invalid batch size '{size}'")), } } fn is_valid_concurrent_tasks_size(size: &str) -> std::result::Result<(), String> { match size.parse::() { Ok(size) if size > 0 => Ok(()), - _ => Err(format!("Invalid concurrent_tasks size '{}'", size)), + _ => Err(format!("Invalid concurrent_tasks size '{size}'")), } } diff --git a/ballista/client/src/context.rs b/ballista/client/src/context.rs index 698f3f4dd..35e4860bd 100644 --- a/ballista/client/src/context.rs +++ b/ballista/client/src/context.rs @@ -95,7 +95,7 @@ impl BallistaContext { ); let connection = create_grpc_client_connection(scheduler_url.clone()) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; let mut scheduler = SchedulerGrpcClient::new(connection); let remote_session_id = scheduler @@ -112,7 +112,7 @@ impl BallistaContext { optional_session_id: None, }) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))? .into_inner() .session_id; @@ -171,7 +171,7 @@ impl BallistaContext { optional_session_id: None, }) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))? .into_inner() .session_id; @@ -258,7 +258,7 @@ impl BallistaContext { options: CsvReadOptions<'_>, ) -> Result<()> { let df = self.read_csv(path, options).await.map_err(|e| { - DataFusionError::Context(format!("Can't read CSV: {}", path), Box::new(e)) + DataFusionError::Context(format!("Can't read CSV: {path}"), Box::new(e)) })?; let plan = df.into_optimized_plan()?; match plan { @@ -427,14 +427,12 @@ impl BallistaContext { Ok(DataFrame::new(ctx.state(), plan)) } _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported file type {:?}.", - file_type + "Unsupported file type {file_type:?}." ))), }, (true, true) => Ok(DataFrame::new(ctx.state(), plan)), (false, true) => Err(DataFusionError::Execution(format!( - "Table '{:?}' already exists", - name + "Table '{name:?}' already exists" ))), } } diff --git a/ballista/core/build.rs b/ballista/core/build.rs index 0fdbdc38f..a13da7777 100644 --- a/ballista/core/build.rs +++ b/ballista/core/build.rs @@ -26,7 +26,7 @@ fn main() -> Result<(), String> { println!("cargo:rerun-if-env-changed=FORCE_REBUILD"); let version = rustc_version::version().unwrap(); - println!("cargo:rustc-env=RUSTC_VERSION={}", version); + println!("cargo:rustc-env=RUSTC_VERSION={version}"); // TODO: undo when resolved: https://github.com/intellij-rust/intellij-rust/issues/9402 #[cfg(feature = "docsrs")] @@ -42,7 +42,7 @@ fn main() -> Result<(), String> { tonic_build::configure() .extern_path(".datafusion", "::datafusion_proto::protobuf") .compile(&["proto/ballista.proto"], &["proto"]) - .map_err(|e| format!("protobuf compilation failed: {}", e))?; + .map_err(|e| format!("protobuf compilation failed: {e}"))?; let generated_source_path = out.join("ballista.protobuf.rs"); let code = std::fs::read_to_string(generated_source_path).unwrap(); let mut file = std::fs::OpenOptions::new() diff --git a/ballista/core/src/client.rs b/ballista/core/src/client.rs index 460254c42..465a83ec5 100644 --- a/ballista/core/src/client.rs +++ b/ballista/core/src/client.rs @@ -56,17 +56,15 @@ impl BallistaClient { /// Create a new BallistaClient to connect to the executor listening on the specified /// host and port pub async fn try_new(host: &str, port: u16) -> Result { - let addr = format!("http://{}:{}", host, port); + let addr = format!("http://{host}:{port}"); debug!("BallistaClient connecting to {}", addr); - let connection = - create_grpc_client_connection(addr.clone()) - .await - .map_err(|e| { - BallistaError::GrpcConnectionError(format!( - "Error connecting to Ballista scheduler or executor at {}: {:?}", - addr, e - )) - })?; + let connection = create_grpc_client_connection(addr.clone()).await.map_err( + |e| { + BallistaError::GrpcConnectionError(format!( + "Error connecting to Ballista scheduler or executor at {addr}: {e:?}" + )) + }, + )?; let flight_client = FlightServiceClient::new(connection); debug!("BallistaClient connected OK"); @@ -115,7 +113,7 @@ impl BallistaClient { serialized_action .encode(&mut buf) - .map_err(|e| BallistaError::GrpcActionError(format!("{:?}", e)))?; + .map_err(|e| BallistaError::GrpcActionError(format!("{e:?}")))?; let request = tonic::Request::new(Ticket { ticket: buf }); @@ -123,14 +121,14 @@ impl BallistaClient { .flight_client .do_get(request) .await - .map_err(|e| BallistaError::GrpcActionError(format!("{:?}", e)))? + .map_err(|e| BallistaError::GrpcActionError(format!("{e:?}")))? .into_inner(); // the schema should be the first message returned, else client should error match stream .message() .await - .map_err(|e| BallistaError::GrpcActionError(format!("{:?}", e)))? + .map_err(|e| BallistaError::GrpcActionError(format!("{e:?}")))? { Some(flight_data) => { // convert FlightData to a stream diff --git a/ballista/core/src/config.rs b/ballista/core/src/config.rs index c3981bef7..6999dd4f2 100644 --- a/ballista/core/src/config.rs +++ b/ballista/core/src/config.rs @@ -116,15 +116,14 @@ impl BallistaConfig { for (name, entry) in &supported_entries { if let Some(v) = settings.get(name) { // validate that we can parse the user-supplied value - Self::parse_value(v.as_str(), entry._data_type.clone()).map_err(|e| BallistaError::General(format!("Failed to parse user-supplied value '{}' for configuration setting '{}': {}", name, v, e)))?; + Self::parse_value(v.as_str(), entry._data_type.clone()).map_err(|e| BallistaError::General(format!("Failed to parse user-supplied value '{name}' for configuration setting '{v}': {e}")))?; } else if let Some(v) = entry.default_value.clone() { - Self::parse_value(v.as_str(), entry._data_type.clone()).map_err(|e| BallistaError::General(format!("Failed to parse default value '{}' for configuration setting '{}': {}", name, v, e)))?; + Self::parse_value(v.as_str(), entry._data_type.clone()).map_err(|e| BallistaError::General(format!("Failed to parse default value '{name}' for configuration setting '{v}': {e}")))?; } else if entry.default_value.is_none() { // optional config } else { return Err(BallistaError::General(format!( - "No value specified for mandatory configuration setting '{}'", - name + "No value specified for mandatory configuration setting '{name}'" ))); } } @@ -137,18 +136,18 @@ impl BallistaConfig { DataType::UInt16 => { val.to_string() .parse::() - .map_err(|e| format!("{:?}", e))?; + .map_err(|e| format!("{e:?}"))?; } DataType::Boolean => { val.to_string() .parse::() - .map_err(|e| format!("{:?}", e))?; + .map_err(|e| format!("{e:?}"))?; } DataType::Utf8 => { val.to_string(); } _ => { - return Err(format!("not support data type: {}", data_type)); + return Err(format!("not support data type: {data_type}")); } } diff --git a/ballista/core/src/error.rs b/ballista/core/src/error.rs index 77a7228c9..f94b4d006 100644 --- a/ballista/core/src/error.rs +++ b/ballista/core/src/error.rs @@ -180,15 +180,15 @@ impl Display for BallistaError { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { BallistaError::NotImplemented(ref desc) => { - write!(f, "Not implemented: {}", desc) + write!(f, "Not implemented: {desc}") } - BallistaError::General(ref desc) => write!(f, "General error: {}", desc), - BallistaError::ArrowError(ref desc) => write!(f, "Arrow error: {}", desc), + BallistaError::General(ref desc) => write!(f, "General error: {desc}"), + BallistaError::ArrowError(ref desc) => write!(f, "Arrow error: {desc}"), BallistaError::DataFusionError(ref desc) => { - write!(f, "DataFusion error: {:?}", desc) + write!(f, "DataFusion error: {desc:?}") } - BallistaError::SqlError(ref desc) => write!(f, "SQL error: {:?}", desc), - BallistaError::IoError(ref desc) => write!(f, "IO error: {}", desc), + BallistaError::SqlError(ref desc) => write!(f, "SQL error: {desc:?}"), + BallistaError::IoError(ref desc) => write!(f, "IO error: {desc}"), // BallistaError::ReqwestError(ref desc) => write!(f, "Reqwest error: {}", desc), // BallistaError::HttpError(ref desc) => write!(f, "HTTP error: {}", desc), // BallistaError::KubeAPIError(ref desc) => write!(f, "Kube API error: {}", desc), @@ -198,24 +198,23 @@ impl Display for BallistaError { // BallistaError::KubeAPIResponseError(ref desc) => { // write!(f, "KubeAPI response error: {}", desc) // } - BallistaError::TonicError(desc) => write!(f, "Tonic error: {}", desc), - BallistaError::GrpcError(desc) => write!(f, "Grpc error: {}", desc), + BallistaError::TonicError(desc) => write!(f, "Tonic error: {desc}"), + BallistaError::GrpcError(desc) => write!(f, "Grpc error: {desc}"), BallistaError::GrpcConnectionError(desc) => { - write!(f, "Grpc connection error: {}", desc) + write!(f, "Grpc connection error: {desc}") } BallistaError::Internal(desc) => { - write!(f, "Internal Ballista error: {}", desc) + write!(f, "Internal Ballista error: {desc}") } - BallistaError::TokioError(desc) => write!(f, "Tokio join error: {}", desc), + BallistaError::TokioError(desc) => write!(f, "Tokio join error: {desc}"), BallistaError::GrpcActionError(desc) => { - write!(f, "Grpc Execute Action error: {}", desc) + write!(f, "Grpc Execute Action error: {desc}") } BallistaError::FetchFailed(executor_id, map_stage, map_partition, desc) => { write!( f, - "Shuffle fetch partition error from Executor {}, map_stage {}, \ - map_partition {}, error desc: {}", - executor_id, map_stage, map_partition, desc + "Shuffle fetch partition error from Executor {executor_id}, map_stage {map_stage}, \ + map_partition {map_partition}, error desc: {desc}" ) } BallistaError::Cancelled => write!(f, "Task cancelled"), @@ -248,7 +247,7 @@ impl From for FailedTask { } BallistaError::IoError(io) => { FailedTask { - error: format!("Task failed due to Ballista IO error: {:?}", io), + error: format!("Task failed due to Ballista IO error: {io:?}"), // IO error is considered to be temporary and retryable retryable: true, count_to_failures: true, @@ -257,7 +256,7 @@ impl From for FailedTask { } BallistaError::DataFusionError(DataFusionError::IoError(io)) => { FailedTask { - error: format!("Task failed due to DataFusion IO error: {:?}", io), + error: format!("Task failed due to DataFusion IO error: {io:?}"), // IO error is considered to be temporary and retryable retryable: true, count_to_failures: true, @@ -265,7 +264,7 @@ impl From for FailedTask { } } other => FailedTask { - error: format!("Task failed due to runtime execution error: {:?}", other), + error: format!("Task failed due to runtime execution error: {other:?}"), retryable: false, count_to_failures: false, failed_reason: Some(FailedReason::ExecutionError(ExecutionError {})), diff --git a/ballista/core/src/event_loop.rs b/ballista/core/src/event_loop.rs index a803bf890..05b6bb413 100644 --- a/ballista/core/src/event_loop.rs +++ b/ballista/core/src/event_loop.rs @@ -134,8 +134,9 @@ impl EventSender { } pub async fn post_event(&self, event: E) -> Result<()> { - self.tx_event.send(event).await.map_err(|e| { - BallistaError::General(format!("Fail to send event due to {}", e)) - }) + self.tx_event + .send(event) + .await + .map_err(|e| BallistaError::General(format!("Fail to send event due to {e}"))) } } diff --git a/ballista/core/src/execution_plans/distributed_query.rs b/ballista/core/src/execution_plans/distributed_query.rs index fdd450e5c..75348692b 100644 --- a/ballista/core/src/execution_plans/distributed_query.rs +++ b/ballista/core/src/execution_plans/distributed_query.rs @@ -162,17 +162,15 @@ impl ExecutionPlan for DistributedQueryExec { assert_eq!(0, partition); let mut buf: Vec = vec![]; - let plan_message = - T::try_from_logical_plan(&self.plan, self.extension_codec.as_ref()).map_err( - |e| { - DataFusionError::Internal(format!( - "failed to serialize logical plan: {:?}", - e - )) - }, - )?; + let plan_message = T::try_from_logical_plan( + &self.plan, + self.extension_codec.as_ref(), + ) + .map_err(|e| { + DataFusionError::Internal(format!("failed to serialize logical plan: {e:?}")) + })?; plan_message.try_encode(&mut buf).map_err(|e| { - DataFusionError::Execution(format!("failed to encode logical plan: {:?}", e)) + DataFusionError::Execution(format!("failed to encode logical plan: {e:?}")) })?; let query = ExecuteQueryParams { @@ -234,14 +232,14 @@ async fn execute_query( // TODO reuse the scheduler to avoid connecting to the Ballista scheduler again and again let connection = create_grpc_client_connection(scheduler_url) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; let mut scheduler = SchedulerGrpcClient::new(connection); let query_result = scheduler .execute_query(query) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))? .into_inner(); assert_eq!( @@ -258,7 +256,7 @@ async fn execute_query( job_id: job_id.clone(), }) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))? .into_inner(); let status = status.and_then(|s| s.status); let wait_future = tokio::time::sleep(Duration::from_millis(100)); @@ -317,7 +315,7 @@ async fn fetch_partition( let port = metadata.port as u16; let mut ballista_client = BallistaClient::try_new(host, port) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; ballista_client .fetch_partition( &metadata.id, diff --git a/ballista/core/src/execution_plans/shuffle_reader.rs b/ballista/core/src/execution_plans/shuffle_reader.rs index edc44d99c..73733bcb9 100644 --- a/ballista/core/src/execution_plans/shuffle_reader.rs +++ b/ballista/core/src/execution_plans/shuffle_reader.rs @@ -403,16 +403,10 @@ fn fetch_partition_local_inner( path: &str, ) -> result::Result, BallistaError> { let file = File::open(path).map_err(|e| { - BallistaError::General(format!( - "Failed to open partition file at {}: {:?}", - path, e - )) + BallistaError::General(format!("Failed to open partition file at {path}: {e:?}")) })?; FileReader::try_new(file, None).map_err(|e| { - BallistaError::General(format!( - "Failed to new arrow FileReader at {}: {:?}", - path, e - )) + BallistaError::General(format!("Failed to new arrow FileReader at {path}: {e:?}")) }) } @@ -585,7 +579,7 @@ mod tests { let batches = utils::collect_stream(&mut stream) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e))) + .map_err(|e| DataFusionError::Execution(format!("{e:?}"))) .unwrap(); let path = batches[0].columns()[1] @@ -602,7 +596,7 @@ mod tests { let result = utils::collect_stream(&mut stream) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e))) + .map_err(|e| DataFusionError::Execution(format!("{e:?}"))) .unwrap(); assert_eq!(result.len(), 2); @@ -652,7 +646,7 @@ mod tests { partition_id, }, executor_meta: ExecutorMetadata { - id: format!("exec{}", partition_id), + id: format!("exec{partition_id}"), host: "localhost".to_string(), port: 50051, grpc_port: 50052, diff --git a/ballista/core/src/execution_plans/shuffle_writer.rs b/ballista/core/src/execution_plans/shuffle_writer.rs index da8c32b39..54db511a1 100644 --- a/ballista/core/src/execution_plans/shuffle_writer.rs +++ b/ballista/core/src/execution_plans/shuffle_writer.rs @@ -159,7 +159,7 @@ impl ShuffleWriterExec { match output_partitioning { None => { let timer = write_metrics.write_time.timer(); - path.push(&format!("{}", input_partition)); + path.push(&format!("{input_partition}")); std::fs::create_dir_all(&path)?; path.push("data.arrow"); let path = path.to_str().unwrap(); @@ -172,7 +172,7 @@ impl ShuffleWriterExec { &write_metrics.write_time, ) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; write_metrics .input_rows @@ -230,12 +230,11 @@ impl ShuffleWriterExec { } None => { let mut path = path.clone(); - path.push(&format!("{}", output_partition)); + path.push(&format!("{output_partition}")); std::fs::create_dir_all(&path)?; path.push(format!( - "data-{}.arrow", - input_partition + "data-{input_partition}.arrow" )); debug!("Writing results to {:?}", path); @@ -457,7 +456,7 @@ mod tests { let mut stream = query_stage.execute(0, task_ctx)?; let batches = utils::collect_stream(&mut stream) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; assert_eq!(1, batches.len()); let batch = &batches[0]; assert_eq!(3, batch.num_columns()); @@ -514,7 +513,7 @@ mod tests { let mut stream = query_stage.execute(0, task_ctx)?; let batches = utils::collect_stream(&mut stream) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; assert_eq!(1, batches.len()); let batch = &batches[0]; assert_eq!(3, batch.num_columns()); diff --git a/ballista/core/src/lib.rs b/ballista/core/src/lib.rs index 34f4699e1..93f738e9c 100644 --- a/ballista/core/src/lib.rs +++ b/ballista/core/src/lib.rs @@ -19,7 +19,7 @@ pub const BALLISTA_VERSION: &str = env!("CARGO_PKG_VERSION"); pub fn print_version() { - println!("Ballista version: {}", BALLISTA_VERSION) + println!("Ballista version: {BALLISTA_VERSION}") } pub mod client; diff --git a/ballista/core/src/plugin/plugin_manager.rs b/ballista/core/src/plugin/plugin_manager.rs index e238383b4..6c19f0542 100644 --- a/ballista/core/src/plugin/plugin_manager.rs +++ b/ballista/core/src/plugin/plugin_manager.rs @@ -69,7 +69,7 @@ impl GlobalPluginManager { let library = Library::new(plugin_file.path()).map_err(|e| { BallistaError::IoError(io::Error::new( io::ErrorKind::Other, - format!("load library error: {}", e), + format!("load library error: {e}"), )) })?; diff --git a/ballista/core/src/plugin/udf.rs b/ballista/core/src/plugin/udf.rs index ea82742fb..88adb8da5 100644 --- a/ballista/core/src/plugin/udf.rs +++ b/ballista/core/src/plugin/udf.rs @@ -60,7 +60,7 @@ impl PluginRegistrar for UDFPluginManager { library.get(b"registrar_udf_plugin\0").map_err(|e| { BallistaError::IoError(io::Error::new( io::ErrorKind::Other, - format!("not found fn registrar_udf_plugin in the library: {}", e), + format!("not found fn registrar_udf_plugin in the library: {e}"), )) })?; @@ -73,7 +73,7 @@ impl PluginRegistrar for UDFPluginManager { if self.scalar_udfs.contains_key(udf_name) { Err(BallistaError::IoError(io::Error::new( io::ErrorKind::Other, - format!("udf name: {} already exists", udf_name), + format!("udf name: {udf_name} already exists"), ))) } else { let scalar_udf = udf_plugin.get_scalar_udf_by_name(udf_name)?; @@ -91,7 +91,7 @@ impl PluginRegistrar for UDFPluginManager { if self.aggregate_udfs.contains_key(udaf_name) { Err(BallistaError::IoError(io::Error::new( io::ErrorKind::Other, - format!("udaf name: {} already exists", udaf_name), + format!("udaf name: {udaf_name} already exists"), ))) } else { let aggregate_udf = diff --git a/ballista/core/src/serde/mod.rs b/ballista/core/src/serde/mod.rs index 14333f2dc..cdf25b19e 100644 --- a/ballista/core/src/serde/mod.rs +++ b/ballista/core/src/serde/mod.rs @@ -57,7 +57,7 @@ pub fn decode_protobuf(bytes: &[u8]) -> Result { let mut buf = Cursor::new(bytes); protobuf::Action::decode(&mut buf) - .map_err(|e| BallistaError::Internal(format!("{:?}", e))) + .map_err(|e| BallistaError::Internal(format!("{e:?}"))) .and_then(|node| node.try_into()) } @@ -504,8 +504,7 @@ mod tests { if let Some((input, _)) = inputs.split_first() { let proto = TopKPlanProto::decode(buf).map_err(|e| { DataFusionError::Internal(format!( - "failed to decode logical plan: {:?}", - e + "failed to decode logical plan: {e:?}" )) })?; @@ -540,8 +539,7 @@ mod tests { proto.encode(buf).map_err(|e| { DataFusionError::Internal(format!( - "failed to encode logical plan: {:?}", - e + "failed to encode logical plan: {e:?}" )) })?; @@ -580,8 +578,7 @@ mod tests { if let Some((input, _)) = inputs.split_first() { let proto = TopKExecProto::decode(buf).map_err(|e| { BallistaError::Internal(format!( - "failed to decode execution plan: {:?}", - e + "failed to decode execution plan: {e:?}" )) })?; Ok(Arc::new(TopKExec::new(proto.k as usize, input.clone()))) @@ -600,8 +597,7 @@ mod tests { proto.encode(buf).map_err(|e| { BallistaError::Internal(format!( - "failed to encode execution plan: {:?}", - e + "failed to encode execution plan: {e:?}" )) })?; @@ -640,10 +636,7 @@ mod tests { let proto = LogicalPlanNode::try_from_logical_plan(&topk_plan, &extension_codec)?; let logical_round_trip = proto.try_into_logical_plan(&ctx, &extension_codec)?; - assert_eq!( - format!("{:?}", topk_plan), - format!("{:?}", logical_round_trip) - ); + assert_eq!(format!("{topk_plan:?}"), format!("{logical_round_trip:?}")); let proto = PhysicalPlanNode::try_from_physical_plan( topk_exec.clone(), @@ -652,10 +645,7 @@ mod tests { let physical_round_trip = proto.try_into_physical_plan(&ctx, runtime.deref(), &extension_codec)?; - assert_eq!( - format!("{:?}", topk_exec), - format!("{:?}", physical_round_trip) - ); + assert_eq!(format!("{topk_exec:?}"), format!("{physical_round_trip:?}")); Ok(()) } diff --git a/ballista/core/src/serde/physical_plan/from_proto.rs b/ballista/core/src/serde/physical_plan/from_proto.rs index 11cd5b725..8738221e2 100644 --- a/ballista/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/core/src/serde/physical_plan/from_proto.rs @@ -229,7 +229,7 @@ fn parse_required_physical_box_expr( .map(|e| parse_physical_expr(e.as_ref(), registry, input_schema)) .transpose()? .ok_or_else(|| { - BallistaError::General(format!("Missing required field {:?}", field)) + BallistaError::General(format!("Missing required field {field:?}")) }) } @@ -243,7 +243,7 @@ fn parse_required_physical_expr( .map(|e| parse_physical_expr(e, registry, input_schema)) .transpose()? .ok_or_else(|| { - BallistaError::General(format!("Missing required field {:?}", field)) + BallistaError::General(format!("Missing required field {field:?}")) }) } @@ -258,8 +258,7 @@ impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFun let f = datafusion_proto::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}" )) })?; @@ -269,8 +268,7 @@ impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFun let f = datafusion_proto::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/ballista/core/src/serde/physical_plan/mod.rs b/ballista/core/src/serde/physical_plan/mod.rs index 2e1027404..2bd43fdea 100644 --- a/ballista/core/src/serde/physical_plan/mod.rs +++ b/ballista/core/src/serde/physical_plan/mod.rs @@ -82,7 +82,7 @@ impl AsExecutionPlan for PhysicalPlanNode { Self: Sized, { PhysicalPlanNode::decode(buf).map_err(|e| { - BallistaError::Internal(format!("failed to decode physical plan: {:?}", e)) + BallistaError::Internal(format!("failed to decode physical plan: {e:?}")) }) } @@ -92,7 +92,7 @@ impl AsExecutionPlan for PhysicalPlanNode { Self: Sized, { self.encode(buf).map_err(|e| { - BallistaError::Internal(format!("failed to encode physical plan: {:?}", e)) + BallistaError::Internal(format!("failed to encode physical plan: {e:?}")) }) } @@ -105,8 +105,7 @@ impl AsExecutionPlan for PhysicalPlanNode { ) -> Result, BallistaError> { 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 { @@ -617,8 +616,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 { @@ -627,8 +625,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(); @@ -641,8 +638,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) } else { Err(BallistaError::General(format!( - "physical_plan::from_proto() {:?}", - self + "physical_plan::from_proto() {self:?}" ))) } }) @@ -663,8 +659,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 { @@ -673,8 +668,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(); @@ -687,8 +681,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) } else { Err(BallistaError::General(format!( - "physical_plan::from_proto() {:?}", - self + "physical_plan::from_proto() {self:?}" ))) } }) @@ -1143,8 +1136,7 @@ impl AsExecutionPlan for PhysicalPlanNode { None => None, other => { return Err(BallistaError::General(format!( - "physical_plan::to_proto() invalid partitioning for ShuffleWriterExec: {:?}", - other + "physical_plan::to_proto() invalid partitioning for ShuffleWriterExec: {other:?}" ))) } }; @@ -1230,8 +1222,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) } Err(e) => Err(BallistaError::Internal(format!( - "Unsupported plan and extension codec failed with [{}]. Plan: {:?}", - e, plan_clone + "Unsupported plan and extension codec failed with [{e}]. Plan: {plan_clone:?}" ))), } } @@ -1357,10 +1348,7 @@ mod roundtrip_tests { codec.physical_extension_codec(), ) .expect("from proto"); - assert_eq!( - format!("{:?}", exec_plan), - format!("{:?}", result_exec_plan) - ); + assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); Ok(()) } @@ -1384,10 +1372,7 @@ mod roundtrip_tests { codec.physical_extension_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/ballista/core/src/serde/physical_plan/to_proto.rs b/ballista/core/src/serde/physical_plan/to_proto.rs index c303ba453..0764835b0 100644 --- a/ballista/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/core/src/serde/physical_plan/to_proto.rs @@ -144,8 +144,7 @@ impl TryInto for Arc { Ok(AggregateFunction::ApproxMedian.into()) } else { Err(BallistaError::NotImplemented(format!( - "Aggregate function not supported: {:?}", - self + "Aggregate function not supported: {self:?}" ))) }?; let expressions: Vec = self @@ -348,8 +347,7 @@ impl TryFrom> for protobuf::PhysicalExprNode { }) } else { Err(BallistaError::General(format!( - "physical_plan::to_proto() unsupported expression {:?}", - value + "physical_plan::to_proto() unsupported expression {value:?}" ))) } } diff --git a/ballista/core/src/serde/scheduler/to_proto.rs b/ballista/core/src/serde/scheduler/to_proto.rs index d453548ea..a7b2be866 100644 --- a/ballista/core/src/serde/scheduler/to_proto.rs +++ b/ballista/core/src/serde/scheduler/to_proto.rs @@ -111,8 +111,7 @@ pub fn hash_partitioning_to_proto( } None => Ok(None), other => Err(BallistaError::General(format!( - "scheduler::to_proto() invalid partitioning for ExecutePartition: {:?}", - other + "scheduler::to_proto() invalid partitioning for ExecutePartition: {other:?}" ))), } } diff --git a/ballista/core/src/utils.rs b/ballista/core/src/utils.rs index 489155d91..83b75151a 100644 --- a/ballista/core/src/utils.rs +++ b/ballista/core/src/utils.rs @@ -135,8 +135,7 @@ impl ObjectStoreProvider for FeatureBasedObjectStoreProvider { } Err(DataFusionError::Execution(format!( - "No object store available for {}", - url + "No object store available for {url}" ))) } } @@ -267,7 +266,7 @@ fn build_exec_plan_diagram( { "CoalescePartitionsExec" } else { - println!("Unknown: {:?}", plan); + println!("Unknown: {plan:?}"); "Unknown" }; @@ -277,8 +276,7 @@ fn build_exec_plan_diagram( if draw_entity { writeln!( w, - "\t\tstage_{}_exec_{} [shape=box, label=\"{}\"];", - stage_id, node_id, operator_str + "\t\tstage_{stage_id}_exec_{node_id} [shape=box, label=\"{operator_str}\"];" )?; } for child in plan.children() { @@ -297,8 +295,7 @@ fn build_exec_plan_diagram( if draw_entity { writeln!( w, - "\t\tstage_{}_exec_{} -> stage_{}_exec_{};", - stage_id, child_id, stage_id, node_id + "\t\tstage_{stage_id}_exec_{child_id} -> stage_{stage_id}_exec_{node_id};" )?; } } diff --git a/ballista/executor/build.rs b/ballista/executor/build.rs index 1c9e32b0b..7d2b9b87b 100644 --- a/ballista/executor/build.rs +++ b/ballista/executor/build.rs @@ -20,5 +20,5 @@ extern crate configure_me_codegen; fn main() -> Result<(), String> { println!("cargo:rerun-if-changed=executor_config_spec.toml"); configure_me_codegen::build_script_auto() - .map_err(|e| format!("configure_me code generation failed: {}", e)) + .map_err(|e| format!("configure_me code generation failed: {e}")) } diff --git a/ballista/executor/src/collect.rs b/ballista/executor/src/collect.rs index 54e97550a..2985e914f 100644 --- a/ballista/executor/src/collect.rs +++ b/ballista/executor/src/collect.rs @@ -87,7 +87,7 @@ impl ExecutionPlan for CollectExec { let streams = (0..num_partitions) .map(|i| self.plan.execute(i, context.clone())) .collect::>>() - .map_err(|e| DataFusionError::Execution(format!("BallistaError: {:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("BallistaError: {e:?}")))?; Ok(Box::pin(MergedRecordBatchStream { schema: self.schema(), diff --git a/ballista/executor/src/execution_loop.rs b/ballista/executor/src/execution_loop.rs index 4efbbe18c..67780c444 100644 --- a/ballista/executor/src/execution_loop.rs +++ b/ballista/executor/src/execution_loop.rs @@ -164,8 +164,7 @@ async fn run_received_task Result<()> { let addr = format!("{}:{}", opt.bind_host, opt.port); let addr = addr .parse() - .with_context(|| format!("Could not parse address: {}", addr))?; + .with_context(|| format!("Could not parse address: {addr}"))?; let scheduler_host = opt.scheduler_host; let scheduler_port = opt.scheduler_port; - let scheduler_url = format!("http://{}:{}", scheduler_host, scheduler_port); + let scheduler_url = format!("http://{scheduler_host}:{scheduler_port}"); let work_dir = opt.work_dir.unwrap_or( TempDir::new()? @@ -214,8 +214,7 @@ pub async fn start_executor_process(opt: ExecutorProcessConfig) -> Result<()> { match x { Some(conn) => Ok(conn), _ => Err(BallistaError::General(format!( - "Timed out attempting to connect to scheduler at {}", - scheduler_url + "Timed out attempting to connect to scheduler at {scheduler_url}" )) .into()), } @@ -301,7 +300,7 @@ pub async fn start_executor_process(opt: ExecutorProcessConfig) -> Result<()> { // until the `shutdown` signal is received or a stop request is coming. let (notify_scheduler, stop_reason) = tokio::select! { service_val = check_services(&mut service_handlers) => { - let msg = format!("executor services stopped with reason {:?}", service_val); + let msg = format!("executor services stopped with reason {service_val:?}"); info!("{:?}", msg); (true, msg) }, diff --git a/ballista/executor/src/executor_server.rs b/ballista/executor/src/executor_server.rs index de4696d29..9f0a44a61 100644 --- a/ballista/executor/src/executor_server.rs +++ b/ballista/executor/src/executor_server.rs @@ -224,7 +224,7 @@ impl ExecutorServer ExecutorGrpc scheduler_id: scheduler_id.clone(), task: task .try_into() - .map_err(|e| Status::invalid_argument(format!("{}", e)))?, + .map_err(|e| Status::invalid_argument(format!("{e}")))?, }) .await .unwrap(); @@ -650,7 +650,7 @@ impl ExecutorGrpc for multi_task in multi_tasks { let multi_task: Vec = multi_task .try_into() - .map_err(|e| Status::invalid_argument(format!("{}", e)))?; + .map_err(|e| Status::invalid_argument(format!("{e}")))?; for task in multi_task { task_sender .send(CuratorTaskDefinition { @@ -731,16 +731,14 @@ impl ExecutorGrpc Ok(Response::new(RemoveJobDataResult {})) } else { Err(Status::invalid_argument(format!( - "Path {:?} is not for a directory!!!", - path + "Path {path:?} is not for a directory!!!" ))) }; } if !is_subdirectory(path.as_path(), work_dir.as_path()) { return Err(Status::invalid_argument(format!( - "Path {:?} is not a subdirectory of {:?}!!!", - path, work_dir + "Path {path:?} is not a subdirectory of {work_dir:?}!!!" ))); } diff --git a/ballista/executor/src/flight_service.rs b/ballista/executor/src/flight_service.rs index d12686edc..c2310b321 100644 --- a/ballista/executor/src/flight_service.rs +++ b/ballista/executor/src/flight_service.rs @@ -94,8 +94,7 @@ impl FlightService for BallistaFlightService { let file = File::open(path) .map_err(|e| { BallistaError::General(format!( - "Failed to open partition file at {}: {:?}", - path, e + "Failed to open partition file at {path}: {e:?}" )) }) .map_err(|e| from_ballista_err(&e))?; @@ -147,7 +146,7 @@ impl FlightService for BallistaFlightService { }; let result = Ok(result); let output = futures::stream::iter(vec![result]); - let str = format!("Bearer {}", token); + let str = format!("Bearer {token}"); let mut resp: Response< Pin> + Sync + Send>>, > = Response::new(Box::pin(output)); @@ -257,13 +256,13 @@ async fn send_response( ) -> Result<(), Status> { tx.send(data) .await - .map_err(|e| Status::internal(format!("{:?}", e))) + .map_err(|e| Status::internal(format!("{e:?}"))) } fn from_arrow_err(e: &ArrowError) -> Status { - Status::internal(format!("ArrowError: {:?}", e)) + Status::internal(format!("ArrowError: {e:?}")) } fn from_ballista_err(e: &ballista_core::error::BallistaError) -> Status { - Status::internal(format!("Ballista Error: {:?}", e)) + Status::internal(format!("Ballista Error: {e:?}")) } diff --git a/ballista/scheduler/build.rs b/ballista/scheduler/build.rs index e90bd495a..d0c8c270f 100644 --- a/ballista/scheduler/build.rs +++ b/ballista/scheduler/build.rs @@ -20,10 +20,10 @@ extern crate configure_me_codegen; fn main() -> Result<(), String> { println!("cargo:rerun-if-changed=scheduler_config_spec.toml"); configure_me_codegen::build_script_auto() - .map_err(|e| format!("configure_me code generation failed: {}", e))?; + .map_err(|e| format!("configure_me code generation failed: {e}"))?; println!("cargo:rerun-if-changed=proto/keda.proto"); tonic_build::configure() .compile(&["proto/keda.proto"], &["proto"]) - .map_err(|e| format!("protobuf compilation failed: {}", e)) + .map_err(|e| format!("protobuf compilation failed: {e}")) } diff --git a/ballista/scheduler/src/bin/main.rs b/ballista/scheduler/src/bin/main.rs index 2ad29bd23..62bfc0839 100644 --- a/ballista/scheduler/src/bin/main.rs +++ b/ballista/scheduler/src/bin/main.rs @@ -78,9 +78,8 @@ async fn main() -> Result<()> { let port = opt.bind_port; let log_dir = opt.log_dir; let print_thread_info = opt.print_thread_info; - let log_file_name_prefix = - format!("scheduler_{}_{}_{}", namespace, external_host, port); - let scheduler_name = format!("{}:{}", external_host, port); + let log_file_name_prefix = format!("scheduler_{namespace}_{external_host}_{port}"); + let scheduler_name = format!("{external_host}:{port}"); let rust_log = env::var(EnvFilter::DEFAULT_ENV); let log_filter = EnvFilter::new(rust_log.unwrap_or(special_mod_log_level)); @@ -118,7 +117,7 @@ async fn main() -> Result<()> { .init(); } - let addr = format!("{}:{}", bind_host, port); + let addr = format!("{bind_host}:{port}"); let addr = addr.parse()?; let config = SchedulerConfig { diff --git a/ballista/scheduler/src/display.rs b/ballista/scheduler/src/display.rs index 6f1de1209..9026e0f08 100644 --- a/ballista/scheduler/src/display.rs +++ b/ballista/scheduler/src/display.rs @@ -134,7 +134,7 @@ impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { .aggregate_by_name() .sorted_for_display() .timestamps_removed(); - write!(self.f, ", metrics=[{}]", metrics)?; + write!(self.f, ", metrics=[{metrics}]")?; } else { write!(self.f, ", metrics=[]")?; } diff --git a/ballista/scheduler/src/flight_sql.rs b/ballista/scheduler/src/flight_sql.rs index 7067a1894..b389b6d2d 100644 --- a/ballista/scheduler/src/flight_sql.rs +++ b/ballista/scheduler/src/flight_sql.rs @@ -131,7 +131,7 @@ impl FlightSqlServiceImpl { let config_builder = BallistaConfig::builder(); let config = config_builder .build() - .map_err(|e| Status::internal(format!("Error building config: {}", e)))?; + .map_err(|e| Status::internal(format!("Error building config: {e}")))?; let ctx = self .server .state @@ -139,7 +139,7 @@ impl FlightSqlServiceImpl { .create_session(&config) .await .map_err(|e| { - Status::internal(format!("Failed to create SessionContext: {:?}", e)) + Status::internal(format!("Failed to create SessionContext: {e:?}")) })?; let handle = Uuid::new_v4(); self.contexts.insert(handle, ctx); @@ -153,7 +153,7 @@ impl FlightSqlServiceImpl { .ok_or_else(|| Status::internal("No authorization header!"))?; let str = auth .to_str() - .map_err(|e| Status::internal(format!("Error parsing header: {}", e)))?; + .map_err(|e| Status::internal(format!("Error parsing header: {e}")))?; let authorization = str.to_string(); let bearer = "Bearer "; if !authorization.starts_with(bearer) { @@ -162,13 +162,12 @@ impl FlightSqlServiceImpl { let auth = authorization[bearer.len()..].to_string(); let handle = Uuid::from_str(auth.as_str()) - .map_err(|e| Status::internal(format!("Error locking contexts: {}", e)))?; + .map_err(|e| Status::internal(format!("Error locking contexts: {e}")))?; if let Some(context) = self.contexts.get(&handle) { Ok(context.clone()) } else { Err(Status::internal(format!( - "Context handle not found: {}", - handle + "Context handle not found: {handle}" )))? } } @@ -181,7 +180,7 @@ impl FlightSqlServiceImpl { .sql(query) .await .and_then(|df| df.into_optimized_plan()) - .map_err(|e| Status::internal(format!("Error building plan: {}", e)))?; + .map_err(|e| Status::internal(format!("Error building plan: {e}")))?; Ok(plan) } @@ -193,14 +192,14 @@ impl FlightSqlServiceImpl { .get_job_status(job_id) .await .map_err(|e| { - let msg = format!("Error getting status for job {}: {:?}", job_id, e); + let msg = format!("Error getting status for job {job_id}: {e:?}"); error!("{}", msg); Status::internal(msg) })?; let status: JobStatus = match status { Some(status) => status, None => { - let msg = format!("Error getting status for job {}!", job_id); + let msg = format!("Error getting status for job {job_id}!"); error!("{}", msg); Err(Status::internal(msg))? } @@ -208,7 +207,7 @@ impl FlightSqlServiceImpl { let status: job_status::Status = match status.status { Some(status) => status, None => { - let msg = format!("Error getting status for job {}!", job_id); + let msg = format!("Error getting status for job {job_id}!"); error!("{}", msg); Err(Status::internal(msg))? } @@ -288,7 +287,7 @@ impl FlightSqlServiceImpl { } let authority = format!("{}:{}", &host, &port); let loc = Location { - uri: format!("grpc+tcp://{}", authority), + uri: format!("grpc+tcp://{authority}"), }; let buf = fetch.as_any().encode_to_vec(); let ticket = Ticket { ticket: buf }; @@ -317,7 +316,7 @@ impl FlightSqlServiceImpl { }; let authority = format!("{}:{}", &host, &port); // TODO: use advertise host let loc = Location { - uri: format!("grpc+tcp://{}", authority), + uri: format!("grpc+tcp://{authority}"), }; let buf = fetch.as_any().encode_to_vec(); let ticket = Ticket { ticket: buf }; @@ -340,8 +339,7 @@ impl FlightSqlServiceImpl { Ok(plan.clone()) } else { Err(Status::internal(format!( - "Statement handle not found: {}", - handle + "Statement handle not found: {handle}" )))? } } @@ -364,7 +362,7 @@ impl FlightSqlServiceImpl { let encoded_data = data_gen.schema_to_bytes(pair.0, pair.1); let mut schema_bytes = vec![]; arrow::ipc::writer::write_message(&mut schema_bytes, encoded_data, pair.1) - .map_err(|e| Status::internal(format!("Error encoding schema: {}", e)))?; + .map_err(|e| Status::internal(format!("Error encoding schema: {e}")))?; Ok(schema_bytes) } @@ -374,13 +372,12 @@ impl FlightSqlServiceImpl { plan: &LogicalPlan, ) -> Result { let job_id = self.server.state.task_manager.generate_job_id(); - let job_name = format!("Flight SQL job {}", job_id); + let job_name = format!("Flight SQL job {job_id}"); self.server .submit_job(&job_id, &job_name, ctx, plan) .await .map_err(|e| { - let msg = - format!("Failed to send JobQueued event for {}: {:?}", job_id, e); + let msg = format!("Failed to send JobQueued event for {job_id}: {e:?}"); error!("{}", msg); Status::internal(msg) })?; @@ -503,8 +500,7 @@ impl FlightSqlService for FlightSqlServiceImpl { .map_err(|_| Status::invalid_argument("authorization not parsable"))?; if !authorization.starts_with(basic) { Err(Status::invalid_argument(format!( - "Auth type not implemented: {}", - authorization + "Auth type not implemented: {authorization}" )))?; } let base64 = &authorization[basic.len()..]; @@ -530,7 +526,7 @@ impl FlightSqlService for FlightSqlServiceImpl { }; let result = Ok(result); let output = futures::stream::iter(vec![result]); - let str = format!("Bearer {}", token); + let str = format!("Bearer {token}"); let mut resp: Response> + Send>>> = Response::new(Box::pin(output)); let md = MetadataValue::try_from(str) @@ -555,7 +551,7 @@ impl FlightSqlService for FlightSqlServiceImpl { let action: protobuf::Action = message .unpack() - .map_err(|e| Status::internal(format!("{:?}", e)))? + .map_err(|e| Status::internal(format!("{e:?}")))? .ok_or_else(|| Status::internal("Expected an Action but got None!"))?; let fp = match &action.action_type { Some(FetchPartition(fp)) => fp.clone(), @@ -586,15 +582,13 @@ impl FlightSqlService for FlightSqlServiceImpl { // Proxy the flight let addr = format!("http://{}:{}", fp.host, fp.port); debug!("Scheduler proxying flight for to {}", addr); - let connection = - create_grpc_client_connection(addr.clone()) - .await - .map_err(|e| { - Status::internal(format!( - "Error connecting to Ballista scheduler or executor at {}: {:?}", - addr, e - )) - })?; + let connection = create_grpc_client_connection(addr.clone()).await.map_err( + |e| { + Status::internal(format!( + "Error connecting to Ballista scheduler or executor at {addr}: {e:?}" + )) + }, + )?; let mut flight_client = FlightServiceClient::new(connection); let buf = action.encode_to_vec(); let request = Request::new(Ticket { ticket: buf }); @@ -602,7 +596,7 @@ impl FlightSqlService for FlightSqlServiceImpl { let stream = flight_client .do_get(request) .await - .map_err(|e| Status::internal(format!("{:?}", e)))? + .map_err(|e| Status::internal(format!("{e:?}")))? .into_inner(); Ok(Response::new(Box::pin(stream))) } @@ -630,7 +624,7 @@ impl FlightSqlService for FlightSqlServiceImpl { debug!("get_flight_info_prepared_statement"); let ctx = self.get_ctx(&request)?; let handle = Uuid::from_slice(handle.prepared_statement_handle.as_slice()) - .map_err(|e| Status::internal(format!("Error decoding handle: {}", e)))?; + .map_err(|e| Status::internal(format!("Error decoding handle: {e}")))?; let plan = self.get_plan(&handle)?; let resp = self.execute_plan(ctx, &plan).await?; @@ -664,7 +658,7 @@ impl FlightSqlService for FlightSqlServiceImpl { let ctx = self.get_ctx(&request)?; let data = self .tables(ctx) - .map_err(|e| Status::internal(format!("Error getting tables: {}", e)))?; + .map_err(|e| Status::internal(format!("Error getting tables: {e}")))?; let resp = self.batch_to_schema_resp(&data, "get_flight_info_tables")?; Ok(resp) } @@ -676,7 +670,7 @@ impl FlightSqlService for FlightSqlServiceImpl { ) -> Result, Status> { debug!("get_flight_info_table_types"); let data = FlightSqlServiceImpl::table_types() - .map_err(|e| Status::internal(format!("Error getting table types: {}", e)))?; + .map_err(|e| Status::internal(format!("Error getting table types: {e}")))?; let resp = self.batch_to_schema_resp(&data, "get_flight_info_table_types")?; Ok(resp) } @@ -852,7 +846,7 @@ impl FlightSqlService for FlightSqlServiceImpl { debug!("do_put_prepared_statement_update"); let ctx = self.get_ctx(&request)?; let handle = Uuid::from_slice(handle.prepared_statement_handle.as_slice()) - .map_err(|e| Status::internal(format!("Error decoding handle: {}", e)))?; + .map_err(|e| Status::internal(format!("Error decoding handle: {e}")))?; let plan = self.get_plan(&handle)?; let _ = self.execute_plan(ctx, &plan).await?; debug!("Sending -1 rows affected"); diff --git a/ballista/scheduler/src/metrics/prometheus.rs b/ballista/scheduler/src/metrics/prometheus.rs index 81efb07db..9c26145b2 100644 --- a/ballista/scheduler/src/metrics/prometheus.rs +++ b/ballista/scheduler/src/metrics/prometheus.rs @@ -57,7 +57,7 @@ impl PrometheusMetricsCollector { registry ) .map_err(|e| { - BallistaError::Internal(format!("Error registering metric: {:?}", e)) + BallistaError::Internal(format!("Error registering metric: {e:?}")) })?; let planning_time = register_histogram_with_registry!( @@ -67,7 +67,7 @@ impl PrometheusMetricsCollector { registry ) .map_err(|e| { - BallistaError::Internal(format!("Error registering metric: {:?}", e)) + BallistaError::Internal(format!("Error registering metric: {e:?}")) })?; let failed = register_counter_with_registry!( @@ -76,7 +76,7 @@ impl PrometheusMetricsCollector { registry ) .map_err(|e| { - BallistaError::Internal(format!("Error registering metric: {:?}", e)) + BallistaError::Internal(format!("Error registering metric: {e:?}")) })?; let cancelled = register_counter_with_registry!( @@ -85,7 +85,7 @@ impl PrometheusMetricsCollector { registry ) .map_err(|e| { - BallistaError::Internal(format!("Error registering metric: {:?}", e)) + BallistaError::Internal(format!("Error registering metric: {e:?}")) })?; let completed = register_counter_with_registry!( @@ -94,7 +94,7 @@ impl PrometheusMetricsCollector { registry ) .map_err(|e| { - BallistaError::Internal(format!("Error registering metric: {:?}", e)) + BallistaError::Internal(format!("Error registering metric: {e:?}")) })?; let submitted = register_counter_with_registry!( @@ -103,7 +103,7 @@ impl PrometheusMetricsCollector { registry ) .map_err(|e| { - BallistaError::Internal(format!("Error registering metric: {:?}", e)) + BallistaError::Internal(format!("Error registering metric: {e:?}")) })?; let pending_queue_size = register_gauge_with_registry!( @@ -112,7 +112,7 @@ impl PrometheusMetricsCollector { registry ) .map_err(|e| { - BallistaError::Internal(format!("Error registering metric: {:?}", e)) + BallistaError::Internal(format!("Error registering metric: {e:?}")) })?; Ok(Self { @@ -168,7 +168,7 @@ impl SchedulerMetricsCollector for PrometheusMetricsCollector { let metric_families = prometheus::gather(); let mut buffer = vec![]; encoder.encode(&metric_families, &mut buffer).map_err(|e| { - BallistaError::Internal(format!("Error encoding prometheus metrics: {:?}", e)) + BallistaError::Internal(format!("Error encoding prometheus metrics: {e:?}")) })?; Ok(Some((buffer, encoder.format_type().to_owned()))) diff --git a/ballista/scheduler/src/planner.rs b/ballista/scheduler/src/planner.rs index e31ad1c45..42c3aa3b6 100644 --- a/ballista/scheduler/src/planner.rs +++ b/ballista/scheduler/src/planner.rs @@ -185,8 +185,7 @@ impl DistributedPlanner { execution_plan.as_any().downcast_ref::() { Err(BallistaError::NotImplemented(format!( - "WindowAggExec with window {:?}", - window + "WindowAggExec with window {window:?}" ))) } else { Ok(( @@ -633,8 +632,8 @@ order by let partial_hash_serde = downcast_exec!(partial_hash_serde, AggregateExec); assert_eq!( - format!("{:?}", partial_hash), - format!("{:?}", partial_hash_serde) + format!("{partial_hash:?}"), + format!("{partial_hash_serde:?}") ); Ok(()) diff --git a/ballista/scheduler/src/scheduler_server/grpc.rs b/ballista/scheduler/src/scheduler_server/grpc.rs index 0863bf735..825f18c88 100644 --- a/ballista/scheduler/src/scheduler_server/grpc.rs +++ b/ballista/scheduler/src/scheduler_server/grpc.rs @@ -112,7 +112,7 @@ impl SchedulerGrpc .save_executor_metadata(metadata.clone()) .await .map_err(|e| { - let msg = format!("Could not save executor metadata: {}", e); + let msg = format!("Could not save executor metadata: {e}"); error!("{}", msg); Status::internal(msg) })?; @@ -122,7 +122,7 @@ impl SchedulerGrpc .save_executor_heartbeat(executor_heartbeat) .await .map_err(|e| { - let msg = format!("Could not save executor heartbeat: {}", e); + let msg = format!("Could not save executor heartbeat: {e}"); error!("{}", msg); Status::internal(msg) })?; @@ -213,7 +213,7 @@ impl SchedulerGrpc } .await .map_err(|e| { - let msg = format!("Fail to do executor registration due to: {}", e); + let msg = format!("Fail to do executor registration due to: {e}"); error!("{}", msg); Status::internal(msg) })?; @@ -249,7 +249,7 @@ impl SchedulerGrpc .save_executor_heartbeat(executor_heartbeat) .await .map_err(|e| { - let msg = format!("Could not save executor heartbeat: {}", e); + let msg = format!("Could not save executor heartbeat: {e}"); error!("{}", msg); Status::internal(msg) })?; @@ -307,14 +307,14 @@ impl SchedulerGrpc .list(Some(&path)) .await .map_err(|e| { - let msg = format!("Error listing files: {}", e); + let msg = format!("Error listing files: {e}"); error!("{}", msg); tonic::Status::internal(msg) })? .try_collect() .await .map_err(|e| { - let msg = format!("Error listing files: {}", e); + let msg = format!("Error listing files: {e}"); error!("{}", msg); tonic::Status::internal(msg) })?; @@ -325,14 +325,14 @@ impl SchedulerGrpc .infer_schema(&ctx, &obj_store, &file_metas) .await .map_err(|e| { - let msg = format!("Error inferring schema: {}", e); + let msg = format!("Error inferring schema: {e}"); error!("{}", msg); tonic::Status::internal(msg) })?; Ok(Response::new(GetFileMetadataResult { schema: Some(schema.as_ref().try_into().map_err(|e| { - let msg = format!("Error inferring schema: {}", e); + let msg = format!("Error inferring schema: {e}"); error!("{}", msg); tonic::Status::internal(msg) })?), @@ -356,7 +356,7 @@ impl SchedulerGrpc config_builder = config_builder.set(&kv_pair.key, &kv_pair.value); } let config = config_builder.build().map_err(|e| { - let msg = format!("Could not parse configs: {}", e); + let msg = format!("Could not parse configs: {e}"); error!("{}", msg); Status::internal(msg) })?; @@ -370,8 +370,7 @@ impl SchedulerGrpc .await .map_err(|e| { Status::internal(format!( - "Failed to load SessionContext for session ID {}: {:?}", - session_id, e + "Failed to load SessionContext for session ID {session_id}: {e:?}" )) })?; (session_id, ctx) @@ -384,8 +383,7 @@ impl SchedulerGrpc .await .map_err(|e| { Status::internal(format!( - "Failed to create SessionContext: {:?}", - e + "Failed to create SessionContext: {e:?}" )) })?; @@ -402,7 +400,7 @@ impl SchedulerGrpc ) }) .map_err(|e| { - let msg = format!("Could not parse logical plan protobuf: {}", e); + let msg = format!("Could not parse logical plan protobuf: {e}"); error!("{}", msg); Status::internal(msg) })?, @@ -411,7 +409,7 @@ impl SchedulerGrpc .await .and_then(|df| df.into_optimized_plan()) .map_err(|e| { - let msg = format!("Error parsing SQL: {}", e); + let msg = format!("Error parsing SQL: {e}"); error!("{}", msg); Status::internal(msg) })?, @@ -430,7 +428,7 @@ impl SchedulerGrpc .await .map_err(|e| { let msg = - format!("Failed to send JobQueued event for {}: {:?}", job_id, e); + format!("Failed to send JobQueued event for {job_id}: {e:?}"); error!("{}", msg); Status::internal(msg) @@ -449,7 +447,7 @@ impl SchedulerGrpc config_builder = config_builder.set(&kv_pair.key, &kv_pair.value); } let config = config_builder.build().map_err(|e| { - let msg = format!("Could not parse configs: {}", e); + let msg = format!("Could not parse configs: {e}"); error!("{}", msg); Status::internal(msg) })?; @@ -460,8 +458,7 @@ impl SchedulerGrpc .await .map_err(|e| { Status::internal(format!( - "Failed to create new SessionContext: {:?}", - e + "Failed to create new SessionContext: {e:?}" )) })?; @@ -483,7 +480,7 @@ impl SchedulerGrpc match self.state.task_manager.get_job_status(&job_id).await { Ok(status) => Ok(Response::new(GetJobStatusResult { status })), Err(e) => { - let msg = format!("Error getting status for job {}: {:?}", job_id, e); + let msg = format!("Error getting status for job {job_id}: {e:?}"); error!("{}", msg); Err(Status::internal(msg)) } @@ -505,14 +502,14 @@ impl SchedulerGrpc let executor_manager = self.state.executor_manager.clone(); let event_sender = self.query_stage_event_loop.get_sender().map_err(|e| { - let msg = format!("Get query stage event loop error due to {:?}", e); + let msg = format!("Get query stage event loop error due to {e:?}"); error!("{}", msg); Status::internal(msg) })?; Self::remove_executor(executor_manager, event_sender, &executor_id, Some(reason)) .await .map_err(|e| { - let msg = format!("Error to remove executor in Scheduler due to {:?}", e); + let msg = format!("Error to remove executor in Scheduler due to {e:?}"); error!("{}", msg); Status::internal(msg) })?; @@ -530,14 +527,14 @@ impl SchedulerGrpc self.query_stage_event_loop .get_sender() .map_err(|e| { - let msg = format!("Get query stage event loop error due to {:?}", e); + let msg = format!("Get query stage event loop error due to {e:?}"); error!("{}", msg); Status::internal(msg) })? .post_event(QueryStageSchedulerEvent::JobCancel(job_id)) .await .map_err(|e| { - let msg = format!("Post to query stage event loop error due to {:?}", e); + let msg = format!("Post to query stage event loop error due to {e:?}"); error!("{}", msg); Status::internal(msg) })?; @@ -554,14 +551,14 @@ impl SchedulerGrpc self.query_stage_event_loop .get_sender() .map_err(|e| { - let msg = format!("Get query stage event loop error due to {:?}", e); + let msg = format!("Get query stage event loop error due to {e:?}"); error!("{}", msg); Status::internal(msg) })? .post_event(QueryStageSchedulerEvent::JobDataClean(job_id)) .await .map_err(|e| { - let msg = format!("Post to query stage event loop error due to {:?}", e); + let msg = format!("Post to query stage event loop error due to {e:?}"); error!("{}", msg); Status::internal(msg) })?; diff --git a/ballista/scheduler/src/scheduler_server/mod.rs b/ballista/scheduler/src/scheduler_server/mod.rs index 5631a9f09..6835fcdcf 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -215,8 +215,7 @@ impl SchedulerServer SchedulerServer .submit_job(&job_id, &job_name, session_ctx, &plan, queued_at) .await { - let fail_message = - format!("Error planning job {}: {:?}", job_id, e); + let fail_message = format!("Error planning job {job_id}: {e:?}"); error!("{}", &fail_message); QueryStageSchedulerEvent::JobPlanningFailed { job_id, @@ -287,8 +286,7 @@ impl } Err(e) => { let msg = format!( - "TaskManager error to handle Executor {} lost: {}", - executor_id, e + "TaskManager error to handle Executor {executor_id} lost: {e}" ); error!("{}", msg); } diff --git a/ballista/scheduler/src/state/backend/cluster.rs b/ballista/scheduler/src/state/backend/cluster.rs index 2370c4925..5fc26bda8 100644 --- a/ballista/scheduler/src/state/backend/cluster.rs +++ b/ballista/scheduler/src/state/backend/cluster.rs @@ -293,10 +293,7 @@ impl ClusterState for DefaultClusterState { let current_ts = SystemTime::now() .duration_since(UNIX_EPOCH) .map_err(|e| { - BallistaError::Internal(format!( - "Error getting current timestamp: {:?}", - e - )) + BallistaError::Internal(format!("Error getting current timestamp: {e:?}")) })? .as_secs(); @@ -378,10 +375,7 @@ impl ClusterState for DefaultClusterState { let current_ts = SystemTime::now() .duration_since(UNIX_EPOCH) .map_err(|e| { - BallistaError::Internal(format!( - "Error getting current timestamp: {:?}", - e - )) + BallistaError::Internal(format!("Error getting current timestamp: {e:?}")) })? .as_secs(); @@ -460,8 +454,7 @@ fn get_alive_executors( .checked_sub(Duration::from_secs(last_seen_threshold)) .ok_or_else(|| { BallistaError::Internal(format!( - "Error getting alive executors, invalid last_seen_threshold of {}", - last_seen_threshold + "Error getting alive executors, invalid last_seen_threshold of {last_seen_threshold}" )) })? .as_secs(); @@ -618,7 +611,7 @@ mod tests { false }; - assert!(received, "Did not receive heartbeat for executor {}", i); + assert!(received, "Did not receive heartbeat for executor {i}"); } Ok(()) @@ -654,14 +647,9 @@ mod tests { assert_eq!( hb.executor_id, i.to_string(), - "Expected heartbeat in map for {}", - i - ); - assert_eq!( - hb.timestamp, i, - "Expected timestamp to be correct for {}", - i + "Expected heartbeat in map for {i}" ); + assert_eq!(hb.timestamp, i, "Expected timestamp to be correct for {i}"); } else { panic!("Expected heartbeat for executor {}", i); } @@ -692,14 +680,9 @@ mod tests { assert_eq!( hb.executor_id, i.to_string(), - "Expected heartbeat in map for {}", - i - ); - assert_eq!( - hb.timestamp, i, - "Expected timestamp to be correct for {}", - i + "Expected heartbeat in map for {i}" ); + assert_eq!(hb.timestamp, i, "Expected timestamp to be correct for {i}"); } else { panic!("Expected heartbeat for executor {}", i); } diff --git a/ballista/scheduler/src/state/backend/etcd.rs b/ballista/scheduler/src/state/backend/etcd.rs index 5fc83aaa5..631acb2c2 100644 --- a/ballista/scheduler/src/state/backend/etcd.rs +++ b/ballista/scheduler/src/state/backend/etcd.rs @@ -57,7 +57,7 @@ impl StateBackendClient for EtcdClient { .clone() .get(key, None) .await - .map_err(|e| ballista_error(&format!("etcd error {:?}", e)))? + .map_err(|e| ballista_error(&format!("etcd error {e:?}")))? .kvs() .get(0) .map(|kv| kv.value().to_owned()) @@ -76,7 +76,7 @@ impl StateBackendClient for EtcdClient { .clone() .get(prefix, Some(GetOptions::new().with_prefix())) .await - .map_err(|e| ballista_error(&format!("etcd error {:?}", e)))? + .map_err(|e| ballista_error(&format!("etcd error {e:?}")))? .kvs() .iter() .map(|kv| (kv.key_str().unwrap().to_owned(), kv.value().to_owned())) @@ -101,7 +101,7 @@ impl StateBackendClient for EtcdClient { .clone() .get(prefix, Some(options)) .await - .map_err(|e| ballista_error(&format!("etcd error {:?}", e)))? + .map_err(|e| ballista_error(&format!("etcd error {e:?}")))? .kvs() .iter() .map(|kv| (kv.key_str().unwrap().to_owned(), kv.value().to_owned())) @@ -118,7 +118,7 @@ impl StateBackendClient for EtcdClient { .clone() .get(prefix.clone(), Some(options)) .await - .map_err(|e| ballista_error(&format!("etcd error {:?}", e)))? + .map_err(|e| ballista_error(&format!("etcd error {e:?}")))? .kvs() .iter() .map(|kv| { @@ -139,7 +139,7 @@ impl StateBackendClient for EtcdClient { .await .map_err(|e| { warn!("etcd put failed: {}", e); - ballista_error(&format!("etcd put failed: {}", e)) + ballista_error(&format!("etcd put failed: {e}")) }) .map(|_| ()) } @@ -163,7 +163,7 @@ impl StateBackendClient for EtcdClient { .await .map_err(|e| { error!("etcd operation failed: {}", e); - ballista_error(&format!("etcd operation failed: {}", e)) + ballista_error(&format!("etcd operation failed: {e}")) }) .map(|_| ()) } @@ -181,7 +181,7 @@ impl StateBackendClient for EtcdClient { let current_value = etcd .get(from_key.as_str(), None) .await - .map_err(|e| ballista_error(&format!("etcd error {:?}", e)))? + .map_err(|e| ballista_error(&format!("etcd error {e:?}")))? .kvs() .get(0) .map(|kv| kv.value().to_owned()); diff --git a/ballista/scheduler/src/state/backend/memory.rs b/ballista/scheduler/src/state/backend/memory.rs index 6cbedb843..ddb2a450d 100644 --- a/ballista/scheduler/src/state/backend/memory.rs +++ b/ballista/scheduler/src/state/backend/memory.rs @@ -46,11 +46,11 @@ impl MemoryBackendClient { } fn get_space_key(keyspace: &Keyspace) -> String { - format!("/{:?}", keyspace) + format!("/{keyspace:?}") } fn get_flat_key(keyspace: &Keyspace, key: &str) -> String { - format!("/{:?}/{}", keyspace, key) + format!("/{keyspace:?}/{key}") } } @@ -140,7 +140,7 @@ impl StateBackendClient for MemoryBackendClient { .insert(key.clone(), value.clone()); // Notify subscribers - let full_key = format!("{}/{}", space_key, key); + let full_key = format!("{space_key}/{key}"); if let Some(res) = self.subscribers.reserve(&full_key) { let event = WatchEvent::Put(full_key, value); res.complete(&event); @@ -215,7 +215,7 @@ impl StateBackendClient for MemoryBackendClient { } async fn watch(&self, keyspace: Keyspace, prefix: String) -> Result> { - let prefix = format!("/{:?}/{}", keyspace, prefix); + let prefix = format!("/{keyspace:?}/{prefix}"); Ok(Box::new(MemoryWatch { subscriber: self.subscribers.register(prefix.as_bytes()), @@ -227,7 +227,7 @@ impl StateBackendClient for MemoryBackendClient { if let Some(mut space_state) = self.states.get_mut(&space_key) { if space_state.value_mut().remove(key).is_some() { // Notify subscribers - let full_key = format!("{}/{}", space_key, key); + let full_key = format!("{space_key}/{key}"); if let Some(res) = self.subscribers.reserve(&full_key) { let event = WatchEvent::Delete(full_key); res.complete(&event); @@ -345,10 +345,10 @@ mod tests { let key = "key"; let value = "value".as_bytes(); client - .put(Keyspace::Slots, format!("{}/1", key), value.to_vec()) + .put(Keyspace::Slots, format!("{key}/1"), value.to_vec()) .await?; client - .put(Keyspace::Slots, format!("{}/2", key), value.to_vec()) + .put(Keyspace::Slots, format!("{key}/2"), value.to_vec()) .await?; assert_eq!( client.get_from_prefix(Keyspace::Slots, key).await?, diff --git a/ballista/scheduler/src/state/backend/sled.rs b/ballista/scheduler/src/state/backend/sled.rs index e42da4ca8..66e896bcd 100644 --- a/ballista/scheduler/src/state/backend/sled.rs +++ b/ballista/scheduler/src/state/backend/sled.rs @@ -60,18 +60,18 @@ impl SledClient { fn sled_to_ballista_error(e: sled::Error) -> BallistaError { match e { sled::Error::Io(io) => BallistaError::IoError(io), - _ => BallistaError::General(format!("{}", e)), + _ => BallistaError::General(format!("{e}")), } } #[tonic::async_trait] impl StateBackendClient for SledClient { async fn get(&self, keyspace: Keyspace, key: &str) -> Result> { - let key = format!("/{:?}/{}", keyspace, key); + let key = format!("/{keyspace:?}/{key}"); Ok(self .db .get(key) - .map_err(|e| ballista_error(&format!("sled error {:?}", e)))? + .map_err(|e| ballista_error(&format!("sled error {e:?}")))? .map(|v| v.to_vec()) .unwrap_or_default()) } @@ -81,7 +81,7 @@ impl StateBackendClient for SledClient { keyspace: Keyspace, prefix: &str, ) -> Result)>> { - let prefix = format!("/{:?}/{}", keyspace, prefix); + let prefix = format!("/{keyspace:?}/{prefix}"); Ok(self .db .scan_prefix(prefix) @@ -94,7 +94,7 @@ impl StateBackendClient for SledClient { }) }) .collect::, _>>() - .map_err(|e| ballista_error(&format!("sled error {:?}", e)))?) + .map_err(|e| ballista_error(&format!("sled error {e:?}")))?) } async fn scan( @@ -102,7 +102,7 @@ impl StateBackendClient for SledClient { keyspace: Keyspace, limit: Option, ) -> Result)>> { - let prefix = format!("/{:?}/", keyspace); + let prefix = format!("/{keyspace:?}/"); if let Some(limit) = limit { Ok(self .db @@ -117,7 +117,7 @@ impl StateBackendClient for SledClient { }) }) .collect::, _>>() - .map_err(|e| ballista_error(&format!("sled error {:?}", e)))?) + .map_err(|e| ballista_error(&format!("sled error {e:?}")))?) } else { Ok(self .db @@ -131,12 +131,12 @@ impl StateBackendClient for SledClient { }) }) .collect::, _>>() - .map_err(|e| ballista_error(&format!("sled error {:?}", e)))?) + .map_err(|e| ballista_error(&format!("sled error {e:?}")))?) } } async fn scan_keys(&self, keyspace: Keyspace) -> Result> { - let prefix = format!("/{:?}/", keyspace); + let prefix = format!("/{keyspace:?}/"); Ok(self .db .scan_prefix(prefix.clone()) @@ -150,11 +150,11 @@ impl StateBackendClient for SledClient { }) }) .collect::, _>>() - .map_err(|e| ballista_error(&format!("sled error {:?}", e)))?) + .map_err(|e| ballista_error(&format!("sled error {e:?}")))?) } async fn put(&self, keyspace: Keyspace, key: String, value: Vec) -> Result<()> { - let key = format!("/{:?}/{}", keyspace, key); + let key = format!("/{keyspace:?}/{key}"); self.db .insert(key, value) .map_err(|e| { @@ -187,13 +187,13 @@ impl StateBackendClient for SledClient { to_keyspace: Keyspace, key: &str, ) -> Result<()> { - let from_key = format!("/{:?}/{}", from_keyspace, key); - let to_key = format!("/{:?}/{}", to_keyspace, key); + let from_key = format!("/{from_keyspace:?}/{key}"); + let to_key = format!("/{to_keyspace:?}/{key}"); let current_value = self .db .get(from_key.as_str()) - .map_err(|e| ballista_error(&format!("sled error {:?}", e)))? + .map_err(|e| ballista_error(&format!("sled error {e:?}")))? .map(|v| v.to_vec()); if let Some(value) = current_value { @@ -215,7 +215,7 @@ impl StateBackendClient for SledClient { async fn lock(&self, keyspace: Keyspace, key: &str) -> Result> { let mut mlock = self.locks.lock().await; - let lock_key = format!("/{:?}/{}", keyspace, key); + let lock_key = format!("/{keyspace:?}/{key}"); if let Some(lock) = mlock.get(&lock_key) { Ok(Box::new(lock.clone().lock_owned().await)) } else { @@ -226,7 +226,7 @@ impl StateBackendClient for SledClient { } async fn watch(&self, keyspace: Keyspace, prefix: String) -> Result> { - let prefix = format!("/{:?}/{}", keyspace, prefix); + let prefix = format!("/{keyspace:?}/{prefix}"); Ok(Box::new(SledWatch { subscriber: self.db.watch_prefix(prefix), @@ -234,7 +234,7 @@ impl StateBackendClient for SledClient { } async fn delete(&self, keyspace: Keyspace, key: &str) -> Result<()> { - let key = format!("/{:?}/{}", keyspace, key); + let key = format!("/{keyspace:?}/{key}"); self.db.remove(key).map_err(|e| { warn!("sled delete failed: {:?}", e); ballista_error("sled delete failed") @@ -348,10 +348,10 @@ mod tests { let key = "key"; let value = "value".as_bytes(); client - .put(Keyspace::Slots, format!("{}/1", key), value.to_vec()) + .put(Keyspace::Slots, format!("{key}/1"), value.to_vec()) .await?; client - .put(Keyspace::Slots, format!("{}/2", key), value.to_vec()) + .put(Keyspace::Slots, format!("{key}/2"), value.to_vec()) .await?; assert_eq!( client.get_from_prefix(Keyspace::Slots, key).await?, diff --git a/ballista/scheduler/src/state/execution_graph.rs b/ballista/scheduler/src/state/execution_graph.rs index 7e4531af6..215a6c6bf 100644 --- a/ballista/scheduler/src/state/execution_graph.rs +++ b/ballista/scheduler/src/state/execution_graph.rs @@ -345,7 +345,7 @@ impl ExecutionGraph { if !failed_stages.is_empty() { let error_msg = format!( - "Stages was marked failed, ignore FetchPartitionError from task {}", task_identity); + "Stages was marked failed, ignore FetchPartitionError from task {task_identity}"); warn!("{}", error_msg); } else { // There are different removal strategies here. @@ -420,8 +420,7 @@ impl ExecutionGraph { } None => { let error_msg = format!( - "Task {} in Stage {} failed with unknown failure reasons, fail the stage", - partition_id, stage_id); + "Task {partition_id} in Stage {stage_id} failed with unknown failure reasons, fail the stage"); error!("{}", error_msg); failed_stages.insert(stage_id, error_msg); } @@ -570,8 +569,7 @@ impl ExecutionGraph { } } else { return Err(BallistaError::Internal(format!( - "Invalid stage ID {} for job {}", - stage_id, job_id + "Invalid stage ID {stage_id} for job {job_id}" ))); } } @@ -609,8 +607,7 @@ impl ExecutionGraph { } } else { return Err(BallistaError::Internal(format!( - "Invalid stage ID {} for job {}", - stage_id, job_id + "Invalid stage ID {stage_id} for job {job_id}" ))); } } @@ -635,8 +632,7 @@ impl ExecutionGraph { } } else { return Err(BallistaError::Internal(format!( - "Invalid stage ID {} for job {}", - stage_id, job_id + "Invalid stage ID {stage_id} for job {job_id}" ))); } } @@ -674,7 +670,7 @@ impl ExecutionGraph { // Fail the stage and also abort the job for (stage_id, err_msg) in &updated_stages.failed_stages { job_err_msg = - format!("Job failed due to stage {} failed: {}\n", stage_id, err_msg); + format!("Job failed due to stage {stage_id} failed: {err_msg}\n"); } let mut events = vec![]; @@ -755,14 +751,12 @@ impl ExecutionGraph { } } else { return Err(BallistaError::Internal(format!( - "Error updating job {}: The stage {} as the output link of stage {} should be unresolved", - job_id, link, stage_id + "Error updating job {job_id}: The stage {link} as the output link of stage {stage_id} should be unresolved" ))); } } else { return Err(BallistaError::Internal(format!( - "Error updating job {}: Invalid output link {} for stage {}", - job_id, stage_id, link + "Error updating job {job_id}: Invalid output link {stage_id} for stage {link}" ))); } } @@ -874,7 +868,7 @@ impl ExecutionGraph { .enumerate() .find(|(_partition, info)| info.is_none()) .ok_or_else(|| { - BallistaError::Internal(format!("Error getting next task for job {}: Stage {} is ready but has no pending tasks", job_id, stage_id)) + BallistaError::Internal(format!("Error getting next task for job {job_id}: Stage {stage_id} is ready but has no pending tasks")) })?; let partition = PartitionId { @@ -914,7 +908,7 @@ impl ExecutionGraph { output_partitioning: stage.output_partitioning.clone(), }) } else { - Err(BallistaError::General(format!("Stage {} is not a running stage", stage_id))) + Err(BallistaError::General(format!("Stage {stage_id} is not a running stage"))) } }).transpose()?; @@ -1411,7 +1405,7 @@ impl Debug for ExecutionGraph { let stages = self .stages .values() - .map(|stage| format!("{:?}", stage)) + .map(|stage| format!("{stage:?}")) .collect::>() .join(""); write!(f, "ExecutionGraph[job_id={}, session_id={}, available_tasks={}, is_successful={}]\n{}", @@ -1610,7 +1604,7 @@ mod test { async fn test_drain_tasks() -> Result<()> { let mut agg_graph = test_aggregation_plan(4).await; - println!("Graph: {:?}", agg_graph); + println!("Graph: {agg_graph:?}"); drain_tasks(&mut agg_graph)?; @@ -1632,7 +1626,7 @@ mod test { drain_tasks(&mut join_graph)?; - println!("{:?}", join_graph); + println!("{join_graph:?}"); assert!(join_graph.is_successful(), "Failed to complete join plan"); @@ -1640,7 +1634,7 @@ mod test { drain_tasks(&mut union_all_graph)?; - println!("{:?}", union_all_graph); + println!("{union_all_graph:?}"); assert!( union_all_graph.is_successful(), @@ -1651,7 +1645,7 @@ mod test { drain_tasks(&mut union_graph)?; - println!("{:?}", union_graph); + println!("{union_graph:?}"); assert!(union_graph.is_successful(), "Failed to complete union plan"); @@ -2891,7 +2885,7 @@ mod test { let graph = ExecutionGraph::new("localhost:50050", "job", "", "session", plan, 0) .unwrap(); - println!("{:?}", graph); + println!("{graph:?}"); graph } @@ -2920,7 +2914,7 @@ mod test { let graph = ExecutionGraph::new("localhost:50050", "job", "", "session", plan, 0) .unwrap(); - println!("{:?}", graph); + println!("{graph:?}"); graph } @@ -2949,7 +2943,7 @@ mod test { let graph = ExecutionGraph::new("localhost:50050", "job", "", "session", plan, 0) .unwrap(); - println!("{:?}", graph); + println!("{graph:?}"); graph } diff --git a/ballista/scheduler/src/state/execution_graph/execution_stage.rs b/ballista/scheduler/src/state/execution_graph/execution_stage.rs index ed98286db..7633ac157 100644 --- a/ballista/scheduler/src/state/execution_graph/execution_stage.rs +++ b/ballista/scheduler/src/state/execution_graph/execution_stage.rs @@ -952,7 +952,7 @@ impl SuccessfulStage { /// Returns the number of running tasks that were reset pub fn reset_tasks(&mut self, executor: &str) -> usize { let mut reset = 0; - let failure_reason = format!("Task failure due to Executor {} lost", executor); + let failure_reason = format!("Task failure due to Executor {executor} lost"); for task in self.task_infos.iter_mut() { match task { TaskInfo { diff --git a/ballista/scheduler/src/state/execution_graph_dot.rs b/ballista/scheduler/src/state/execution_graph_dot.rs index 185d0d578..a7fe8468d 100644 --- a/ballista/scheduler/src/state/execution_graph_dot.rs +++ b/ballista/scheduler/src/state/execution_graph_dot.rs @@ -64,7 +64,7 @@ impl ExecutionGraphDot { if let Some(stage) = graph.stages().get(&stage_id) { let mut dot = String::new(); writeln!(&mut dot, "digraph G {{")?; - let stage_name = format!("stage_{}", stage_id); + let stage_name = format!("stage_{stage_id}"); write_stage_plan(&mut dot, &stage_name, stage.plan(), 0)?; writeln!(&mut dot, "}}")?; Ok(dot) @@ -89,8 +89,8 @@ impl ExecutionGraphDot { #[allow(clippy::explicit_counter_loop)] for id in &stage_ids { let stage = stages.get(id).unwrap(); // safe unwrap - let stage_name = format!("stage_{}", id); - writeln!(&mut dot, "\tsubgraph cluster{} {{", cluster)?; + let stage_name = format!("stage_{id}"); + writeln!(&mut dot, "\tsubgraph cluster{cluster} {{")?; writeln!( &mut dot, "\t\tlabel = \"Stage {} [{}]\";", @@ -107,13 +107,13 @@ impl ExecutionGraphDot { let mut links = vec![]; for (reader_node, parent_stage_id) in &meta.readers { // shuffle write node is always node zero - let parent_shuffle_write_node = format!("stage_{}_0", parent_stage_id); - links.push(format!("{} -> {}", parent_shuffle_write_node, reader_node,)); + let parent_shuffle_write_node = format!("stage_{parent_stage_id}_0"); + links.push(format!("{parent_shuffle_write_node} -> {reader_node}",)); } // keep the order deterministic links.sort(); for link in links { - writeln!(&mut dot, "\t{}", link)?; + writeln!(&mut dot, "\t{link}")?; } } @@ -145,7 +145,7 @@ fn write_plan_recursive( i: usize, state: &mut StagePlanState, ) -> Result<(), fmt::Error> { - let node_name = format!("{}_{}", prefix, i); + let node_name = format!("{prefix}_{i}"); let display_name = get_operator_name(plan); if let Some(reader) = plan.as_any().downcast_ref::() { @@ -163,18 +163,14 @@ fn write_plan_recursive( let mut metrics_str = vec![]; if let Some(metrics) = plan.metrics() { if let Some(x) = metrics.output_rows() { - metrics_str.push(format!("output_rows={}", x)) + metrics_str.push(format!("output_rows={x}")) } if let Some(x) = metrics.elapsed_compute() { - metrics_str.push(format!("elapsed_compute={}", x)) + metrics_str.push(format!("elapsed_compute={x}")) } } if metrics_str.is_empty() { - writeln!( - f, - "\t\t{} [shape=box, label=\"{}\"]", - node_name, display_name - )?; + writeln!(f, "\t\t{node_name} [shape=box, label=\"{display_name}\"]")?; } else { writeln!( f, @@ -189,7 +185,7 @@ fn write_plan_recursive( for (j, child) in plan.children().into_iter().enumerate() { write_plan_recursive(f, &node_name, child.as_ref(), j, state)?; // write link from child to parent - writeln!(f, "\t\t{}_{} -> {}", node_name, j, node_name)?; + writeln!(f, "\t\t{node_name}_{j} -> {node_name}")?; } Ok(()) @@ -240,7 +236,7 @@ fn get_operator_name(plan: &dyn ExecutionPlan) -> String { let expr = exec .expr() .iter() - .map(|(e, _)| format!("{}", e)) + .map(|(e, _)| format!("{e}")) .collect::>() .join(", "); format!("Projection: {}", sanitize_dot_label(&expr)) @@ -264,7 +260,7 @@ fn get_operator_name(plan: &dyn ExecutionPlan) -> String { let group_exprs_with_alias = exec.group_expr().expr(); let group_expr = group_exprs_with_alias .iter() - .map(|(e, _)| format!("{}", e)) + .map(|(e, _)| format!("{e}")) .collect::>() .join(", "); let aggr_expr = exec @@ -296,7 +292,7 @@ aggr=[{}]", let join_expr = exec .on() .iter() - .map(|(l, r)| format!("{} = {}", l, r)) + .map(|(l, r)| format!("{l} = {r}")) .collect::>() .join(" AND "); let filter_expr = if let Some(f) = exec.filter() { @@ -335,7 +331,7 @@ filter_expr={}", ) } else if let Some(exec) = plan.as_any().downcast_ref::() { let parts = exec.output_partitioning().partition_count(); - format!("JSON [{} partitions]", parts) + format!("JSON [{parts} partitions]") } else if let Some(exec) = plan.as_any().downcast_ref::() { let parts = exec.output_partitioning().partition_count(); format!( @@ -370,7 +366,7 @@ filter_expr={}", fn format_partitioning(x: Partitioning) -> String { match x { Partitioning::UnknownPartitioning(n) | Partitioning::RoundRobinBatch(n) => { - format!("{} partitions", n) + format!("{n} partitions") } Partitioning::Hash(expr, n) => { format!("{} partitions, expr={}", n, format_expr_list(&expr)) @@ -379,7 +375,7 @@ fn format_partitioning(x: Partitioning) -> String { } fn format_expr_list(exprs: &[Arc]) -> String { - let expr_strings: Vec = exprs.iter().map(|e| format!("{}", e)).collect(); + let expr_strings: Vec = exprs.iter().map(|e| format!("{e}")).collect(); expr_strings.join(", ") } @@ -431,7 +427,7 @@ mod tests { async fn dot() -> Result<()> { let graph = test_graph().await?; let dot = ExecutionGraphDot::generate(Arc::new(graph)) - .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; + .map_err(|e| BallistaError::Internal(format!("{e:?}")))?; let expected = r#" digraph G { @@ -506,7 +502,7 @@ filter_expr="] async fn query_stage() -> Result<()> { let graph = test_graph().await?; let dot = ExecutionGraphDot::generate_for_query_stage(Arc::new(graph), 3) - .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; + .map_err(|e| BallistaError::Internal(format!("{e:?}")))?; let expected = r#" digraph G { @@ -536,7 +532,7 @@ filter_expr="] async fn dot_optimized() -> Result<()> { let graph = test_graph_optimized().await?; let dot = ExecutionGraphDot::generate(Arc::new(graph)) - .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; + .map_err(|e| BallistaError::Internal(format!("{e:?}")))?; let expected = r#" digraph G { @@ -611,7 +607,7 @@ filter_expr="] async fn query_stage_optimized() -> Result<()> { let graph = test_graph_optimized().await?; let dot = ExecutionGraphDot::generate_for_query_stage(Arc::new(graph), 4) - .map_err(|e| BallistaError::Internal(format!("{:?}", e)))?; + .map_err(|e| BallistaError::Internal(format!("{e:?}")))?; let expected = r#" digraph G { diff --git a/ballista/scheduler/src/state/executor_manager.rs b/ballista/scheduler/src/state/executor_manager.rs index a6234528c..8f0a810b4 100644 --- a/ballista/scheduler/src/state/executor_manager.rs +++ b/ballista/scheduler/src/state/executor_manager.rs @@ -163,7 +163,7 @@ impl ExecutorManager { } else { let alive_executors = self.get_alive_executors_within_one_minute(); - println!("Alive executors: {:?}", alive_executors); + println!("Alive executors: {alive_executors:?}"); self.cluster_state .reserve_slots(n, self.task_distribution, Some(alive_executors)) @@ -459,10 +459,7 @@ impl ExecutorManager { let current_ts = SystemTime::now() .duration_since(UNIX_EPOCH) .map_err(|e| { - BallistaError::Internal(format!( - "Error getting current timestamp: {:?}", - e - )) + BallistaError::Internal(format!("Error getting current timestamp: {e:?}")) })? .as_secs(); @@ -690,8 +687,7 @@ mod test { assert_eq!( reservations.len(), 40, - "Expected 40 reservations for policy {:?}", - slots_policy + "Expected 40 reservations for policy {slots_policy:?}" ); // Now cancel them @@ -703,8 +699,7 @@ mod test { assert_eq!( reservations.len(), 40, - "Expected 40 reservations for policy {:?}", - slots_policy + "Expected 40 reservations for policy {slots_policy:?}" ); Ok(()) @@ -857,8 +852,8 @@ mod test { for i in 0..total_executors { result.push(( ExecutorMetadata { - id: format!("executor-{}", i), - host: format!("host-{}", i), + id: format!("executor-{i}"), + host: format!("host-{i}"), port: 8080, grpc_port: 9090, specification: ExecutorSpecification { @@ -866,7 +861,7 @@ mod test { }, }, ExecutorData { - executor_id: format!("executor-{}", i), + executor_id: format!("executor-{i}"), total_task_slots: slots_per_executor, available_task_slots: slots_per_executor, }, diff --git a/ballista/scheduler/src/state/mod.rs b/ballista/scheduler/src/state/mod.rs index 974d4b2ec..3b0d1c529 100644 --- a/ballista/scheduler/src/state/mod.rs +++ b/ballista/scheduler/src/state/mod.rs @@ -377,8 +377,7 @@ impl SchedulerState TaskManager Ok(task_definition) } else { Err(BallistaError::General(format!( - "Cannot prepare task definition for job {} which is not in active cache", - job_id + "Cannot prepare task definition for job {job_id} which is not in active cache" ))) } } @@ -728,7 +727,7 @@ impl TaskManager }; Ok(multi_task_definition) } else { - Err(BallistaError::General(format!("Cannot prepare multi task definition for job {} which is not in active cache", job_id))) + Err(BallistaError::General(format!("Cannot prepare multi task definition for job {job_id} which is not in active cache"))) } } else { Err(BallistaError::General( diff --git a/ballista/scheduler/src/test_utils.rs b/ballista/scheduler/src/test_utils.rs index 3f98e3708..e844b57b7 100644 --- a/ballista/scheduler/src/test_utils.rs +++ b/ballista/scheduler/src/test_utils.rs @@ -126,7 +126,7 @@ pub async fn datafusion_test_context(path: &str) -> Result { .delimiter(b'|') .has_header(false) .file_extension(".tbl"); - let dir = format!("{}/{}", path, table); + let dir = format!("{path}/{table}"); ctx.register_csv(table, &dir, options).await?; } Ok(ctx) @@ -362,7 +362,7 @@ impl TaskLauncher for VirtualTaskLauncher { .send((executor.id.clone(), status)) .await .map_err(|e| { - BallistaError::Internal(format!("Error sending task status: {:?}", e)) + BallistaError::Internal(format!("Error sending task status: {e:?}")) }) } } @@ -397,7 +397,7 @@ impl SchedulerTest { let executors: HashMap = (0..num_executors) .into_iter() .map(|i| { - let id = format!("virtual-executor-{}", i); + let id = format!("virtual-executor-{i}"); let executor = VirtualExecutor { executor_id: id.clone(), task_slots: task_slots_per_executor, @@ -681,7 +681,7 @@ pub fn assert_submitted_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(found, "Expected submitted event for job {}", job_id); + assert!(found, "Expected submitted event for job {job_id}"); } pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) { @@ -690,7 +690,7 @@ pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(!found, "Expected no submitted event for job {}", job_id); + assert!(!found, "Expected no submitted event for job {job_id}"); } pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -699,7 +699,7 @@ pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Completed(_, _, _))); - assert!(found, "Expected completed event for job {}", job_id); + assert!(found, "Expected completed event for job {job_id}"); } pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { @@ -708,7 +708,7 @@ pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Cancelled(_))); - assert!(found, "Expected cancelled event for job {}", job_id); + assert!(found, "Expected cancelled event for job {job_id}"); } pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -717,5 +717,5 @@ pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Failed(_, _, _))); - assert!(found, "Expected failed event for job {}", job_id); + assert!(found, "Expected failed event for job {job_id}"); } diff --git a/benchmarks/src/bin/nyctaxi.rs b/benchmarks/src/bin/nyctaxi.rs index 917022cfa..630757f4c 100644 --- a/benchmarks/src/bin/nyctaxi.rs +++ b/benchmarks/src/bin/nyctaxi.rs @@ -67,7 +67,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) @@ -87,7 +87,7 @@ async fn main() -> Result<()> { .await? } other => { - println!("Invalid file format '{}'", other); + println!("Invalid file format '{other}'"); process::exit(-1); } } @@ -103,7 +103,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?; @@ -122,7 +122,7 @@ async fn execute_sql(ctx: &SessionContext, sql: &str, debug: bool) -> Result<()> let plan = ctx.state().create_logical_plan(sql).await?; let plan = ctx.state().optimize(&plan)?; if debug { - println!("Optimized logical plan:\n{:?}", plan); + println!("Optimized logical plan:\n{plan:?}"); } let physical_plan = ctx.state().create_physical_plan(&plan).await?; let task_ctx = ctx.task_ctx(); diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index 0a86728bd..ecbc932f9 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -283,7 +283,7 @@ async fn main() -> Result<()> { #[allow(clippy::await_holding_lock)] async fn benchmark_datafusion(opt: DataFusionBenchmarkOpt) -> Result> { - println!("Running benchmarks with the following options: {:?}", opt); + println!("Running benchmarks with the following options: {opt:?}"); let mut benchmark_run = BenchmarkRun::new(opt.query); let config = SessionConfig::new() .with_target_partitions(opt.partitions) @@ -304,7 +304,7 @@ async fn benchmark_datafusion(opt: DataFusionBenchmarkOpt) -> Result Result Result<()> { - println!("Running benchmarks with the following options: {:?}", opt); + println!("Running benchmarks with the following options: {opt:?}"); let mut benchmark_run = BenchmarkRun::new(opt.query); let config = BallistaConfig::builder() @@ -364,12 +364,12 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { ) .set(BALLISTA_DEFAULT_BATCH_SIZE, &format!("{}", opt.batch_size)) .build() - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; let ctx = BallistaContext::remote(opt.host.unwrap().as_str(), opt.port.unwrap(), &config) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; // register tables with Ballista context let path = opt.path.to_str().unwrap(); @@ -392,7 +392,7 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { let df = ctx .sql(sql) .await - .map_err(|e| DataFusionError::Plan(format!("{:?}", e))) + .map_err(|e| DataFusionError::Plan(format!("{e:?}"))) .unwrap(); if opt.debug { println!("=== Optimized logical plan ===\n{:?}\n", df.logical_plan()); @@ -400,7 +400,7 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { batches = df .collect() .await - .map_err(|e| DataFusionError::Plan(format!("{:?}", e))) + .map_err(|e| DataFusionError::Plan(format!("{e:?}"))) .unwrap(); } let elapsed = start.elapsed().as_secs_f64() * 1000.0; @@ -449,10 +449,7 @@ fn write_summary_json(benchmark_run: &mut BenchmarkRun, path: &Path) -> Result<( } async fn loadtest_ballista(opt: BallistaLoadtestOpt) -> Result<()> { - println!( - "Running loadtest_ballista with the following options: {:?}", - opt - ); + println!("Running loadtest_ballista with the following options: {opt:?}"); let config = BallistaConfig::builder() .set( @@ -461,7 +458,7 @@ async fn loadtest_ballista(opt: BallistaLoadtestOpt) -> Result<()> { ) .set(BALLISTA_DEFAULT_BATCH_SIZE, &format!("{}", opt.batch_size)) .build() - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?; let concurrency = opt.concurrency; let request_amount = opt.requests; @@ -475,7 +472,7 @@ async fn loadtest_ballista(opt: BallistaLoadtestOpt) -> Result<()> { &config, ) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?, + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?, ); } @@ -523,12 +520,12 @@ async fn loadtest_ballista(opt: BallistaLoadtestOpt) -> Result<()> { let df = client .sql(&sql) .await - .map_err(|e| DataFusionError::Plan(format!("{:?}", e))) + .map_err(|e| DataFusionError::Plan(format!("{e:?}"))) .unwrap(); let batches = df .collect() .await - .map_err(|e| DataFusionError::Plan(format!("{:?}", e))) + .map_err(|e| DataFusionError::Plan(format!("{e:?}"))) .unwrap(); let elapsed = start.elapsed().as_secs_f64() * 1000.0; println!( @@ -545,7 +542,7 @@ async fn loadtest_ballista(opt: BallistaLoadtestOpt) -> Result<()> { join_all(futures).await; let elapsed = total.elapsed().as_secs_f64() * 1000.0; println!("###############################"); - println!("load test took {:.1} ms", elapsed); + println!("load test took {elapsed:.1} ms"); Ok(()) } @@ -554,7 +551,7 @@ fn get_query_sql_by_path(query: usize, mut sql_path: String) -> Result { sql_path.pop(); } if query > 0 && query < 23 { - let filename = format!("{}/q{}.sql", sql_path, query); + let filename = format!("{sql_path}/q{query}.sql"); Ok(fs::read_to_string(filename).expect("failed to read query")) } else { Err(DataFusionError::Plan( @@ -582,13 +579,12 @@ async fn register_tables( .file_extension(".tbl"); if debug { println!( - "Registering table '{}' using TBL files at path {}", - table, path + "Registering table '{table}' using TBL files at path {path}" ); } ctx.register_csv(table, &path, options) .await - .map_err(|e| DataFusionError::Plan(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Plan(format!("{e:?}")))?; } "csv" => { let path = find_path(path, table, "csv")?; @@ -596,30 +592,27 @@ async fn register_tables( let options = CsvReadOptions::new().schema(&schema).has_header(true); if debug { println!( - "Registering table '{}' using CSV files at path {}", - table, path + "Registering table '{table}' using CSV files at path {path}" ); } ctx.register_csv(table, &path, options) .await - .map_err(|e| DataFusionError::Plan(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Plan(format!("{e:?}")))?; } "parquet" => { let path = find_path(path, table, "parquet")?; if debug { println!( - "Registering table '{}' using Parquet files at path {}", - table, path + "Registering table '{table}' using Parquet files at path {path}" ); } ctx.register_parquet(table, &path, ParquetReadOptions::default()) .await - .map_err(|e| DataFusionError::Plan(format!("{:?}", e)))?; + .map_err(|e| DataFusionError::Plan(format!("{e:?}")))?; } other => { return Err(DataFusionError::Plan(format!( - "Invalid file format '{}'", - other + "Invalid file format '{other}'" ))) } } @@ -628,16 +621,15 @@ async fn register_tables( } fn find_path(path: &str, table: &str, ext: &str) -> Result { - let path1 = format!("{}/{}.{}", path, table, ext); - let path2 = format!("{}/{}", path, table); + let path1 = format!("{path}/{table}.{ext}"); + let path2 = format!("{path}/{table}"); if Path::new(&path1).exists() { Ok(path1) } else if Path::new(&path2).exists() { Ok(path2) } else { Err(DataFusionError::Plan(format!( - "Could not find {} files at {} or {}", - ext, path1, path2 + "Could not find {ext} files at {path1} or {path2}" ))) } } @@ -646,8 +638,8 @@ fn find_path(path: &str, table: &str, ext: &str) -> Result { 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 { @@ -660,12 +652,11 @@ 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( @@ -694,11 +685,11 @@ async fn execute_query( debug: bool, ) -> Result> { if debug { - println!("=== Logical plan ===\n{:?}\n", plan); + println!("=== Logical plan ===\n{plan:?}\n"); } let plan = ctx.state().optimize(plan)?; if debug { - println!("=== Optimized logical plan ===\n{:?}\n", plan); + println!("=== Optimized logical plan ===\n{plan:?}\n"); } let physical_plan = ctx.state().create_physical_plan(&plan).await?; if debug { @@ -769,8 +760,7 @@ async fn convert_tbl(opt: ConvertOpt) -> Result<()> { "zstd" => Compression::ZSTD, other => { return Err(DataFusionError::NotImplemented(format!( - "Invalid compression format: {}", - other + "Invalid compression format: {other}" ))) } }; @@ -781,8 +771,7 @@ async fn convert_tbl(opt: ConvertOpt) -> Result<()> { } other => { return Err(DataFusionError::NotImplemented(format!( - "Invalid output format: {}", - other + "Invalid output format: {other}" ))) } } @@ -803,7 +792,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'|') @@ -812,7 +801,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); @@ -820,7 +809,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::new().with_enable_pruning(Some(true)); (Arc::new(format), path, DEFAULT_PARQUET_EXTENSION) @@ -1019,8 +1008,8 @@ async fn get_expected_results(n: usize, path: &str) -> Result> .schema(&schema) .delimiter(b'|') .file_extension(".out"); - let answer_path = format!("{}/answers/q{}.out", path, n); - println!("Looking for expected results at {}", answer_path); + let answer_path = format!("{path}/answers/q{n}.out"); + println!("Looking for expected results at {answer_path}"); let df = ctx.read_csv(&answer_path, options).await?; let df = df.select( get_answer_schema(n) @@ -1598,8 +1587,8 @@ mod tests { .try_into_logical_plan(&ctx, codec.logical_extension_codec()) .unwrap(); assert_eq!( - format!("{:?}", plan), - format!("{:?}", round_trip), + format!("{plan:?}"), + format!("{round_trip:?}"), "optimized logical plan round trip failed" ); @@ -1621,8 +1610,8 @@ mod tests { ) .unwrap(); assert_eq!( - format!("{:?}", physical_plan), - format!("{:?}", round_trip), + format!("{physical_plan:?}"), + format!("{round_trip:?}"), "physical plan round trip failed" ); } From 1070a7ef060cbc5442965fa6df19c241c594fe77 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 14:52:30 -0700 Subject: [PATCH 06/13] we lint nice, cause we lint twice --- ballista/core/src/client.rs | 11 ++++++----- ballista/scheduler/src/flight_sql.rs | 11 ++++++----- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/ballista/core/src/client.rs b/ballista/core/src/client.rs index 465a83ec5..a9e616e81 100644 --- a/ballista/core/src/client.rs +++ b/ballista/core/src/client.rs @@ -58,13 +58,14 @@ impl BallistaClient { pub async fn try_new(host: &str, port: u16) -> Result { let addr = format!("http://{host}:{port}"); debug!("BallistaClient connecting to {}", addr); - let connection = create_grpc_client_connection(addr.clone()).await.map_err( - |e| { - BallistaError::GrpcConnectionError(format!( + let connection = + create_grpc_client_connection(addr.clone()) + .await + .map_err(|e| { + BallistaError::GrpcConnectionError(format!( "Error connecting to Ballista scheduler or executor at {addr}: {e:?}" )) - }, - )?; + })?; let flight_client = FlightServiceClient::new(connection); debug!("BallistaClient connected OK"); diff --git a/ballista/scheduler/src/flight_sql.rs b/ballista/scheduler/src/flight_sql.rs index b389b6d2d..c8ea58a32 100644 --- a/ballista/scheduler/src/flight_sql.rs +++ b/ballista/scheduler/src/flight_sql.rs @@ -582,13 +582,14 @@ impl FlightSqlService for FlightSqlServiceImpl { // Proxy the flight let addr = format!("http://{}:{}", fp.host, fp.port); debug!("Scheduler proxying flight for to {}", addr); - let connection = create_grpc_client_connection(addr.clone()).await.map_err( - |e| { - Status::internal(format!( + let connection = + create_grpc_client_connection(addr.clone()) + .await + .map_err(|e| { + Status::internal(format!( "Error connecting to Ballista scheduler or executor at {addr}: {e:?}" )) - }, - )?; + })?; let mut flight_client = FlightServiceClient::new(connection); let buf = action.encode_to_vec(); let request = Request::new(Ticket { ticket: buf }); From 27ab3f2c230072883a776076acc85e8a0dd9e62b Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 15:10:15 -0700 Subject: [PATCH 07/13] restrain and overly-agressive clippy --fix --- .../scheduler/src/scheduler_server/mod.rs | 6 ++++-- .../scheduler/src/state/backend/cluster.rs | 20 ++++++++++++++----- ballista/scheduler/src/test_utils.rs | 10 +++++----- 3 files changed, 24 insertions(+), 12 deletions(-) diff --git a/ballista/scheduler/src/scheduler_server/mod.rs b/ballista/scheduler/src/scheduler_server/mod.rs index 6835fcdcf..ba0c29bc7 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -581,7 +581,8 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {status:?}" + "Expected job status to be failed but it was {:?}", + status ); assert_submitted_event("job", &metrics_collector); @@ -624,7 +625,8 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {status:?}" + "Expected job status to be failed but it was {:?}", + status ); assert_no_submitted_event("job", &metrics_collector); diff --git a/ballista/scheduler/src/state/backend/cluster.rs b/ballista/scheduler/src/state/backend/cluster.rs index 5fc26bda8..a8c4723b4 100644 --- a/ballista/scheduler/src/state/backend/cluster.rs +++ b/ballista/scheduler/src/state/backend/cluster.rs @@ -611,7 +611,7 @@ mod tests { false }; - assert!(received, "Did not receive heartbeat for executor {i}"); + assert!(received, "Did not receive heartbeat for executor {}", i); } Ok(()) @@ -647,9 +647,14 @@ mod tests { assert_eq!( hb.executor_id, i.to_string(), - "Expected heartbeat in map for {i}" + "Expected heartbeat in map for {}", + i + ); + assert_eq!( + hb.timestamp, i, + "Expected timestamp to be correct for {}", + i ); - assert_eq!(hb.timestamp, i, "Expected timestamp to be correct for {i}"); } else { panic!("Expected heartbeat for executor {}", i); } @@ -680,9 +685,14 @@ mod tests { assert_eq!( hb.executor_id, i.to_string(), - "Expected heartbeat in map for {i}" + "Expected heartbeat in map for {}", + i + ); + assert_eq!( + hb.timestamp, i, + "Expected timestamp to be correct for {}", + i ); - assert_eq!(hb.timestamp, i, "Expected timestamp to be correct for {i}"); } else { panic!("Expected heartbeat for executor {}", i); } diff --git a/ballista/scheduler/src/test_utils.rs b/ballista/scheduler/src/test_utils.rs index e844b57b7..20b7a9c70 100644 --- a/ballista/scheduler/src/test_utils.rs +++ b/ballista/scheduler/src/test_utils.rs @@ -681,7 +681,7 @@ pub fn assert_submitted_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(found, "Expected submitted event for job {job_id}"); + assert!(found, "Expected submitted event for job {}", job_id); } pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) { @@ -690,7 +690,7 @@ pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(!found, "Expected no submitted event for job {job_id}"); + assert!(!found, "Expected no submitted event for job {}", job_id); } pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -699,7 +699,7 @@ pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Completed(_, _, _))); - assert!(found, "Expected completed event for job {job_id}"); + assert!(found, "Expected completed event for job {}", job_id); } pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { @@ -708,7 +708,7 @@ pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Cancelled(_))); - assert!(found, "Expected cancelled event for job {job_id}"); + assert!(found, "Expected cancelled event for job {}", job_id); } pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -717,5 +717,5 @@ pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Failed(_, _, _))); - assert!(found, "Expected failed event for job {job_id}"); + assert!(found, "Expected failed event for job {}", job_id); } From 83ec9f13ae0c434bf717df6534ea746aa2e82858 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 16:01:55 -0700 Subject: [PATCH 08/13] Paste results back into test = passing test --- .../scheduler/src/state/execution_graph.rs | 8 ++++---- .../src/state/execution_graph_dot.rs | 20 +++++++++++++++---- 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/ballista/scheduler/src/state/execution_graph.rs b/ballista/scheduler/src/state/execution_graph.rs index 215a6c6bf..15c2cecbb 100644 --- a/ballista/scheduler/src/state/execution_graph.rs +++ b/ballista/scheduler/src/state/execution_graph.rs @@ -1686,13 +1686,13 @@ mod test { // With the improvement of https://github.com/apache/arrow-datafusion/pull/4122, // unnecessary RepartitionExec can be removed - assert_eq!(join_graph.stage_count(), 4); + assert_eq!(join_graph.stage_count(), 5); assert_eq!(join_graph.available_tasks(), 0); // Call revive to move the two leaf Resolved stages to Running join_graph.revive(); - assert_eq!(join_graph.stage_count(), 4); + assert_eq!(join_graph.stage_count(), 5); assert_eq!(join_graph.available_tasks(), 2); // Complete the first stage @@ -1737,13 +1737,13 @@ mod test { let executor2 = mock_executor("executor-id2".to_string()); let mut join_graph = test_join_plan(4).await; - assert_eq!(join_graph.stage_count(), 4); + assert_eq!(join_graph.stage_count(), 5); assert_eq!(join_graph.available_tasks(), 0); // Call revive to move the two leaf Resolved stages to Running join_graph.revive(); - assert_eq!(join_graph.stage_count(), 4); + assert_eq!(join_graph.stage_count(), 5); assert_eq!(join_graph.available_tasks(), 2); // Complete the first stage diff --git a/ballista/scheduler/src/state/execution_graph_dot.rs b/ballista/scheduler/src/state/execution_graph_dot.rs index a7fe8468d..6a438e4c2 100644 --- a/ballista/scheduler/src/state/execution_graph_dot.rs +++ b/ballista/scheduler/src/state/execution_graph_dot.rs @@ -606,14 +606,26 @@ filter_expr="] #[tokio::test] async fn query_stage_optimized() -> Result<()> { let graph = test_graph_optimized().await?; - let dot = ExecutionGraphDot::generate_for_query_stage(Arc::new(graph), 4) + let dot = ExecutionGraphDot::generate_for_query_stage(Arc::new(graph), 3) .map_err(|e| BallistaError::Internal(format!("{e:?}")))?; let expected = r#" digraph G { - stage_4_0 [shape=box, label="ShuffleWriter [0 partitions]"] - stage_4_0_0 [shape=box, label="MemoryExec"] - stage_4_0_0 -> stage_4_0 + stage_3_0 [shape=box, label="ShuffleWriter [48 partitions]"] + stage_3_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_3_0_0_0 [shape=box, label="HashJoin +join_expr=a@0 = a@0 +filter_expr="] + stage_3_0_0_0_0 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_3_0_0_0_0_0 [shape=box, label="UnresolvedShuffleExec [stage_id=1]"] + stage_3_0_0_0_0_0 -> stage_3_0_0_0_0 + stage_3_0_0_0_0 -> stage_3_0_0_0 + stage_3_0_0_0_1 [shape=box, label="CoalesceBatches [batchSize=8192]"] + stage_3_0_0_0_1_0 [shape=box, label="UnresolvedShuffleExec [stage_id=2]"] + stage_3_0_0_0_1_0 -> stage_3_0_0_0_1 + stage_3_0_0_0_1 -> stage_3_0_0_0 + stage_3_0_0_0 -> stage_3_0_0 + stage_3_0_0 -> stage_3_0 } "# .trim(); From 08eb10264cb4ee4b8053941bc12da1185a0244f7 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 16:04:25 -0700 Subject: [PATCH 09/13] Paste results back into test = passing test --- ballista/scheduler/src/planner.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ballista/scheduler/src/planner.rs b/ballista/scheduler/src/planner.rs index 42c3aa3b6..2fcf75c7b 100644 --- a/ballista/scheduler/src/planner.rs +++ b/ballista/scheduler/src/planner.rs @@ -628,7 +628,8 @@ order by let partial_hash = stages[0].children()[0].clone(); let partial_hash_serde = roundtrip_operator(partial_hash.clone())?; - let partial_hash = downcast_exec!(partial_hash, AggregateExec); + let partial_hash = downcast_exec!( + , AggregateExec); let partial_hash_serde = downcast_exec!(partial_hash_serde, AggregateExec); assert_eq!( From 046d543722c15d4b1629623cb03c2bfc99162531 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Thu, 26 Jan 2023 18:43:15 -0700 Subject: [PATCH 10/13] Fix last non-python test --- ballista/core/src/serde/physical_plan/from_proto.rs | 12 ++++++++++-- ballista/scheduler/src/planner.rs | 3 +-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/ballista/core/src/serde/physical_plan/from_proto.rs b/ballista/core/src/serde/physical_plan/from_proto.rs index 8738221e2..80ab8f300 100644 --- a/ballista/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/core/src/serde/physical_plan/from_proto.rs @@ -363,9 +363,17 @@ impl TryInto for &protobuf::Statistics { .iter() .map(|s| s.into()) .collect::>(); + let num_rows = match self.num_rows { + -1 => None, + x => Some(x as usize), + }; + let total_byte_size = match self.total_byte_size { + -1 => None, + x => Some(x as usize), + }; Ok(Statistics { - num_rows: Some(self.num_rows as usize), - total_byte_size: Some(self.total_byte_size as usize), + num_rows, + total_byte_size, // No column statistic (None) is encoded with empty array column_statistics: if column_statistics.is_empty() { None diff --git a/ballista/scheduler/src/planner.rs b/ballista/scheduler/src/planner.rs index 2fcf75c7b..42c3aa3b6 100644 --- a/ballista/scheduler/src/planner.rs +++ b/ballista/scheduler/src/planner.rs @@ -628,8 +628,7 @@ order by let partial_hash = stages[0].children()[0].clone(); let partial_hash_serde = roundtrip_operator(partial_hash.clone())?; - let partial_hash = downcast_exec!( - , AggregateExec); + let partial_hash = downcast_exec!(partial_hash, AggregateExec); let partial_hash_serde = downcast_exec!(partial_hash_serde, AggregateExec); assert_eq!( From 7631e85f5e955f867415ce27d5d4407a430f791e Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Fri, 27 Jan 2023 08:14:33 -0700 Subject: [PATCH 11/13] I guess clippy was right --- .../scheduler/src/scheduler_server/mod.rs | 6 ++---- .../scheduler/src/state/backend/cluster.rs | 20 +++++-------------- ballista/scheduler/src/test_utils.rs | 10 +++++----- 3 files changed, 12 insertions(+), 24 deletions(-) diff --git a/ballista/scheduler/src/scheduler_server/mod.rs b/ballista/scheduler/src/scheduler_server/mod.rs index ba0c29bc7..6835fcdcf 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -581,8 +581,7 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {:?}", - status + "Expected job status to be failed but it was {status:?}" ); assert_submitted_event("job", &metrics_collector); @@ -625,8 +624,7 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {:?}", - status + "Expected job status to be failed but it was {status:?}" ); assert_no_submitted_event("job", &metrics_collector); diff --git a/ballista/scheduler/src/state/backend/cluster.rs b/ballista/scheduler/src/state/backend/cluster.rs index a8c4723b4..5fc26bda8 100644 --- a/ballista/scheduler/src/state/backend/cluster.rs +++ b/ballista/scheduler/src/state/backend/cluster.rs @@ -611,7 +611,7 @@ mod tests { false }; - assert!(received, "Did not receive heartbeat for executor {}", i); + assert!(received, "Did not receive heartbeat for executor {i}"); } Ok(()) @@ -647,14 +647,9 @@ mod tests { assert_eq!( hb.executor_id, i.to_string(), - "Expected heartbeat in map for {}", - i - ); - assert_eq!( - hb.timestamp, i, - "Expected timestamp to be correct for {}", - i + "Expected heartbeat in map for {i}" ); + assert_eq!(hb.timestamp, i, "Expected timestamp to be correct for {i}"); } else { panic!("Expected heartbeat for executor {}", i); } @@ -685,14 +680,9 @@ mod tests { assert_eq!( hb.executor_id, i.to_string(), - "Expected heartbeat in map for {}", - i - ); - assert_eq!( - hb.timestamp, i, - "Expected timestamp to be correct for {}", - i + "Expected heartbeat in map for {i}" ); + assert_eq!(hb.timestamp, i, "Expected timestamp to be correct for {i}"); } else { panic!("Expected heartbeat for executor {}", i); } diff --git a/ballista/scheduler/src/test_utils.rs b/ballista/scheduler/src/test_utils.rs index 20b7a9c70..e844b57b7 100644 --- a/ballista/scheduler/src/test_utils.rs +++ b/ballista/scheduler/src/test_utils.rs @@ -681,7 +681,7 @@ pub fn assert_submitted_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(found, "Expected submitted event for job {}", job_id); + assert!(found, "Expected submitted event for job {job_id}"); } pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) { @@ -690,7 +690,7 @@ pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(!found, "Expected no submitted event for job {}", job_id); + assert!(!found, "Expected no submitted event for job {job_id}"); } pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -699,7 +699,7 @@ pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Completed(_, _, _))); - assert!(found, "Expected completed event for job {}", job_id); + assert!(found, "Expected completed event for job {job_id}"); } pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { @@ -708,7 +708,7 @@ pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Cancelled(_))); - assert!(found, "Expected cancelled event for job {}", job_id); + assert!(found, "Expected cancelled event for job {job_id}"); } pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -717,5 +717,5 @@ pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Failed(_, _, _))); - assert!(found, "Expected failed event for job {}", job_id); + assert!(found, "Expected failed event for job {job_id}"); } From 67d34a2434b54d3118856136b0dcc7bdbfdfe693 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Fri, 27 Jan 2023 08:59:32 -0700 Subject: [PATCH 12/13] okay, this clippy situation is nuts --- ballista/scheduler/src/scheduler_server/mod.rs | 6 ++++-- ballista/scheduler/src/state/backend/cluster.rs | 2 +- ballista/scheduler/src/test_utils.rs | 10 +++++----- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/ballista/scheduler/src/scheduler_server/mod.rs b/ballista/scheduler/src/scheduler_server/mod.rs index 6835fcdcf..ba0c29bc7 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -581,7 +581,8 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {status:?}" + "Expected job status to be failed but it was {:?}", + status ); assert_submitted_event("job", &metrics_collector); @@ -624,7 +625,8 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {status:?}" + "Expected job status to be failed but it was {:?}", + status ); assert_no_submitted_event("job", &metrics_collector); diff --git a/ballista/scheduler/src/state/backend/cluster.rs b/ballista/scheduler/src/state/backend/cluster.rs index 5fc26bda8..d30463f07 100644 --- a/ballista/scheduler/src/state/backend/cluster.rs +++ b/ballista/scheduler/src/state/backend/cluster.rs @@ -611,7 +611,7 @@ mod tests { false }; - assert!(received, "Did not receive heartbeat for executor {i}"); + assert!(received, "Did not receive heartbeat for executor {}", i); } Ok(()) diff --git a/ballista/scheduler/src/test_utils.rs b/ballista/scheduler/src/test_utils.rs index e844b57b7..20b7a9c70 100644 --- a/ballista/scheduler/src/test_utils.rs +++ b/ballista/scheduler/src/test_utils.rs @@ -681,7 +681,7 @@ pub fn assert_submitted_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(found, "Expected submitted event for job {job_id}"); + assert!(found, "Expected submitted event for job {}", job_id); } pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) { @@ -690,7 +690,7 @@ pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(!found, "Expected no submitted event for job {job_id}"); + assert!(!found, "Expected no submitted event for job {}", job_id); } pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -699,7 +699,7 @@ pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Completed(_, _, _))); - assert!(found, "Expected completed event for job {job_id}"); + assert!(found, "Expected completed event for job {}", job_id); } pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { @@ -708,7 +708,7 @@ pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Cancelled(_))); - assert!(found, "Expected cancelled event for job {job_id}"); + assert!(found, "Expected cancelled event for job {}", job_id); } pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -717,5 +717,5 @@ pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Failed(_, _, _))); - assert!(found, "Expected failed event for job {job_id}"); + assert!(found, "Expected failed event for job {}", job_id); } From d8e63ea4ad1c2bd4f8a286e053fa42ab05981293 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Fri, 27 Jan 2023 12:59:08 -0700 Subject: [PATCH 13/13] clippy --- ballista/scheduler/src/scheduler_server/mod.rs | 6 ++---- ballista/scheduler/src/state/backend/cluster.rs | 2 +- ballista/scheduler/src/test_utils.rs | 10 +++++----- 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/ballista/scheduler/src/scheduler_server/mod.rs b/ballista/scheduler/src/scheduler_server/mod.rs index ba0c29bc7..6835fcdcf 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -581,8 +581,7 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {:?}", - status + "Expected job status to be failed but it was {status:?}" ); assert_submitted_event("job", &metrics_collector); @@ -625,8 +624,7 @@ mod test { status: Some(job_status::Status::Failed(_)) } ), - "Expected job status to be failed but it was {:?}", - status + "Expected job status to be failed but it was {status:?}" ); assert_no_submitted_event("job", &metrics_collector); diff --git a/ballista/scheduler/src/state/backend/cluster.rs b/ballista/scheduler/src/state/backend/cluster.rs index d30463f07..5fc26bda8 100644 --- a/ballista/scheduler/src/state/backend/cluster.rs +++ b/ballista/scheduler/src/state/backend/cluster.rs @@ -611,7 +611,7 @@ mod tests { false }; - assert!(received, "Did not receive heartbeat for executor {}", i); + assert!(received, "Did not receive heartbeat for executor {i}"); } Ok(()) diff --git a/ballista/scheduler/src/test_utils.rs b/ballista/scheduler/src/test_utils.rs index 20b7a9c70..e844b57b7 100644 --- a/ballista/scheduler/src/test_utils.rs +++ b/ballista/scheduler/src/test_utils.rs @@ -681,7 +681,7 @@ pub fn assert_submitted_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(found, "Expected submitted event for job {}", job_id); + assert!(found, "Expected submitted event for job {job_id}"); } pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) { @@ -690,7 +690,7 @@ pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) .iter() .any(|ev| matches!(ev, MetricEvent::Submitted(_, _, _))); - assert!(!found, "Expected no submitted event for job {}", job_id); + assert!(!found, "Expected no submitted event for job {job_id}"); } pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -699,7 +699,7 @@ pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Completed(_, _, _))); - assert!(found, "Expected completed event for job {}", job_id); + assert!(found, "Expected completed event for job {job_id}"); } pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { @@ -708,7 +708,7 @@ pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Cancelled(_))); - assert!(found, "Expected cancelled event for job {}", job_id); + assert!(found, "Expected cancelled event for job {job_id}"); } pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -717,5 +717,5 @@ pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { .iter() .any(|ev| matches!(ev, MetricEvent::Failed(_, _, _))); - assert!(found, "Expected failed event for job {}", job_id); + assert!(found, "Expected failed event for job {job_id}"); }