From b383870924d7af73851e1a3087dc36640ba105d4 Mon Sep 17 00:00:00 2001 From: yangzhong Date: Tue, 17 Jan 2023 13:23:27 +0800 Subject: [PATCH 1/4] Update datafusion dependency to the latest version --- Cargo.toml | 4 +- ballista-cli/Cargo.toml | 4 +- ballista/client/Cargo.toml | 6 +- ballista/client/src/context.rs | 46 +- ballista/core/Cargo.toml | 8 +- ballista/core/build.rs | 4 +- ballista/core/proto/ballista.proto | 408 +--- ballista/core/proto/datafusion.proto | 849 ++++++-- 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 | 667 +------ ballista/core/src/serde/mod.rs | 741 ++----- .../src/serde/physical_plan/from_proto.rs | 418 ---- ballista/core/src/serde/physical_plan/mod.rs | 1707 ----------------- .../core/src/serde/physical_plan/to_proto.rs | 468 ----- .../core/src/serde/scheduler/from_proto.rs | 24 +- ballista/core/src/serde/scheduler/mod.rs | 4 +- ballista/core/src/serde/scheduler/to_proto.rs | 16 +- ballista/core/src/utils.rs | 4 +- ballista/executor/Cargo.toml | 8 +- ballista/executor/src/execution_loop.rs | 5 +- ballista/executor/src/executor_process.rs | 4 +- ballista/executor/src/executor_server.rs | 24 +- ballista/executor/src/standalone.rs | 3 +- ballista/scheduler/Cargo.toml | 6 +- ballista/scheduler/src/api/handlers.rs | 2 +- ballista/scheduler/src/api/mod.rs | 2 +- ballista/scheduler/src/flight_sql.rs | 5 +- ballista/scheduler/src/planner.rs | 36 +- ballista/scheduler/src/scheduler_process.rs | 6 +- .../src/scheduler_server/external_scaler.rs | 2 +- .../scheduler/src/scheduler_server/grpc.rs | 24 +- .../scheduler/src/scheduler_server/mod.rs | 13 +- .../scheduler_server/query_stage_scheduler.rs | 2 +- ballista/scheduler/src/standalone.rs | 2 +- .../scheduler/src/state/execution_graph.rs | 73 +- .../state/execution_graph/execution_stage.rs | 8 +- .../src/state/execution_graph_dot.rs | 28 +- ballista/scheduler/src/state/mod.rs | 12 +- .../scheduler/src/state/session_manager.rs | 21 - ballista/scheduler/src/state/task_manager.rs | 3 +- ballista/scheduler/src/test_utils.rs | 6 +- benchmarks/Cargo.toml | 4 +- benchmarks/src/bin/nyctaxi.rs | 7 +- benchmarks/src/bin/tpch.rs | 69 +- examples/Cargo.toml | 2 +- python/Cargo.toml | 2 +- 50 files changed, 1184 insertions(+), 4597 deletions(-) delete mode 100644 ballista/core/src/serde/physical_plan/from_proto.rs delete mode 100644 ballista/core/src/serde/physical_plan/mod.rs delete mode 100644 ballista/core/src/serde/physical_plan/to_proto.rs diff --git a/Cargo.toml b/Cargo.toml index 659c1d2ae..ab4fd3c7b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,13 +17,13 @@ [workspace] members = [ - "benchmarks", + "ballista-cli", "ballista/client", "ballista/core", "ballista/executor", "ballista/scheduler", + "benchmarks", "examples", - "ballista-cli", ] exclude = ["python"] diff --git a/ballista-cli/Cargo.toml b/ballista-cli/Cargo.toml index 8c295efca..a726b4fa6 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.1.0" +datafusion-cli = "16.1.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..d262e5bad 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.1.0" +datafusion-proto = "16.1.0" futures = "0.3" log = "0.4" parking_lot = "0.12" -sqlparser = "0.27" +sqlparser = "0.30.0" tempfile = "3" tokio = "1.0" diff --git a/ballista/client/src/context.rs b/ballista/client/src/context.rs index 51a6c3bc2..c935c5f8e 100644 --- a/ballista/client/src/context.rs +++ b/ballista/client/src/context.rs @@ -139,8 +139,8 @@ impl BallistaContext { config: &BallistaConfig, concurrent_tasks: usize, ) -> ballista_core::error::Result { - use ballista_core::serde::protobuf::PhysicalPlanNode; use ballista_core::serde::BallistaCodec; + use datafusion_proto::protobuf::PhysicalPlanNode; log::info!("Running in local mode. Scheduler will be run in-proc"); @@ -212,7 +212,7 @@ impl BallistaContext { &self, path: &str, options: AvroReadOptions<'_>, - ) -> Result> { + ) -> Result { let df = self.context.read_avro(path, options).await?; Ok(df) } @@ -223,7 +223,7 @@ impl BallistaContext { &self, path: &str, options: ParquetReadOptions<'_>, - ) -> Result> { + ) -> Result { let df = self.context.read_parquet(path, options).await?; Ok(df) } @@ -234,7 +234,7 @@ impl BallistaContext { &self, path: &str, options: CsvReadOptions<'_>, - ) -> Result> { + ) -> Result { let df = self.context.read_csv(path, options).await?; Ok(df) } @@ -262,7 +262,7 @@ impl BallistaContext { .map_err(|e| { DataFusionError::Context(format!("Can't read CSV: {}", path), Box::new(e)) })? - .to_logical_plan()?; + .into_optimized_plan()?; match plan { LogicalPlan::TableScan(TableScan { source, .. }) => { self.register_table(name, source_as_provider(&source)?) @@ -277,7 +277,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 +295,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)?) } @@ -332,7 +336,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 +365,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,7 +379,7 @@ impl BallistaContext { ref if_not_exists, .. }) => { - let table_exists = ctx.table_exist(name.as_str())?; + let table_exists = ctx.table_exist(name.as_table_reference())?; let schema: SchemaRef = Arc::new(schema.as_ref().to_owned().into()); let table_partition_cols = table_partition_cols .iter() @@ -397,37 +401,40 @@ impl BallistaContext { if !schema.fields().is_empty() { options = options.schema(&schema); } - self.register_csv(name, location, options).await?; - Ok(Arc::new(DataFrame::new(ctx.state.clone(), &plan))) + self.register_csv( + name.as_table_reference().table(), + location, + options, + ) + .await?; + Ok(DataFrame::new(ctx.state(), plan)) } "parquet" => { self.register_parquet( - name, + name.as_table_reference().table(), location, ParquetReadOptions::default() .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( - name, + name.as_table_reference().table(), location, AvroReadOptions::default() .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 @@ -593,6 +600,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/core/Cargo.toml b/ballista/core/Cargo.toml index 8aa4e14a1..27b28a5a0 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.1.0" datafusion-objectstore-hdfs = { version = "0.1.1", default-features = false, optional = true } -datafusion-proto = "15.0.0" +datafusion-proto = "16.1.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.0" sys-info = "0.9.0" tokio = "1.0" tokio-stream = { version = "0.1", features = ["net"] } 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/proto/ballista.proto b/ballista/core/proto/ballista.proto index 76d3521fa..ddf2c7af5 100644 --- a/ballista/core/proto/ballista.proto +++ b/ballista/core/proto/ballista.proto @@ -26,196 +26,24 @@ option java_outer_classname = "BallistaProto"; import "datafusion.proto"; -/////////////////////////////////////////////////////////////////////////////////////////////////// -// Ballista Logical Plan -/////////////////////////////////////////////////////////////////////////////////////////////////// - -message Statistics { - int64 num_rows = 1; - int64 total_byte_size = 2; - repeated ColumnStats column_stats = 3; - bool is_exact = 4; -} - -message FileRange { - int64 start = 1; - int64 end = 2; -} - -message PartitionedFile { - string path = 1; - uint64 size = 2; - uint64 last_modified_ns = 3; - repeated datafusion.ScalarValue partition_values = 4; - FileRange range = 5; -} - /////////////////////////////////////////////////////////////////////////////////////////////////// // Ballista Physical Plan /////////////////////////////////////////////////////////////////////////////////////////////////// - -// PhysicalPlanNode is a nested type -message PhysicalPlanNode { +message BallistaPhysicalPlanNode { oneof PhysicalPlanType { - ParquetScanExecNode parquet_scan = 1; - CsvScanExecNode csv_scan = 2; - EmptyExecNode empty = 3; - ProjectionExecNode projection = 4; - GlobalLimitExecNode global_limit = 6; - LocalLimitExecNode local_limit = 7; - AggregateExecNode aggregate = 8; - HashJoinExecNode hash_join = 9; - ShuffleReaderExecNode shuffle_reader = 10; - SortExecNode sort = 11; - CoalesceBatchesExecNode coalesce_batches = 12; - FilterExecNode filter = 13; - CoalescePartitionsExecNode merge = 14; - UnresolvedShuffleExecNode unresolved = 15; - RepartitionExecNode repartition = 16; - WindowAggExecNode window = 17; - ShuffleWriterExecNode shuffle_writer = 18; - CrossJoinExecNode cross_join = 19; - AvroScanExecNode avro_scan = 20; - PhysicalExtensionNode extension = 21; - UnionExecNode union = 22; - ExplainExecNode explain = 23; - SortPreservingMergeExecNode sort_preserving_merge = 24; + ShuffleWriterExecNode shuffle_writer = 1; + ShuffleReaderExecNode shuffle_reader = 2; + UnresolvedShuffleExecNode unresolved_shuffle = 3; } } -message PhysicalExtensionNode { - bytes node = 1; - repeated PhysicalPlanNode inputs = 2; -} - -// physical expressions -message PhysicalExprNode { - oneof ExprType { - // column references - PhysicalColumn column = 1; - - datafusion.ScalarValue literal = 2; - - // binary expressions - PhysicalBinaryExprNode binary_expr = 3; - - // aggregate expressions - PhysicalAggregateExprNode aggregate_expr = 4; - - // null checks - PhysicalIsNull is_null_expr = 5; - PhysicalIsNotNull is_not_null_expr = 6; - PhysicalNot not_expr = 7; - - PhysicalCaseNode case_ = 8; - PhysicalCastNode cast = 9; - PhysicalSortExprNode sort = 10; - PhysicalNegativeNode negative = 11; - PhysicalInListNode in_list = 12; - PhysicalScalarFunctionNode scalar_function = 13; - PhysicalTryCastNode try_cast = 14; - - // window expressions - PhysicalWindowExprNode window_expr = 15; - - PhysicalScalarUdfNode scalar_udf = 16; - - PhysicalDateTimeIntervalExprNode date_time_interval_expr = 17; - } -} - -message PhysicalScalarUdfNode { - string name = 1; - repeated PhysicalExprNode args = 2; - datafusion.ArrowType return_type = 4; -} - -message PhysicalAggregateExprNode { - datafusion.AggregateFunction aggr_function = 1; - repeated PhysicalExprNode expr = 2; - bool distinct = 3; -} - -message PhysicalWindowExprNode { - oneof window_function { - datafusion.AggregateFunction aggr_function = 1; - datafusion.BuiltInWindowFunction built_in_function = 2; - // udaf = 3 - } - PhysicalExprNode expr = 4; -} - -message PhysicalIsNull { - PhysicalExprNode expr = 1; -} - -message PhysicalIsNotNull { - PhysicalExprNode expr = 1; -} - -message PhysicalNot { - PhysicalExprNode expr = 1; -} - -message PhysicalAliasNode { - PhysicalExprNode expr = 1; - string alias = 2; -} - -message PhysicalBinaryExprNode { - PhysicalExprNode l = 1; - PhysicalExprNode r = 2; - string op = 3; -} - -message PhysicalDateTimeIntervalExprNode { - PhysicalExprNode l = 1; - PhysicalExprNode r = 2; - string op = 3; -} - -message PhysicalSortExprNode { - PhysicalExprNode expr = 1; - bool asc = 2; - bool nulls_first = 3; -} - -message PhysicalWhenThen { - PhysicalExprNode when_expr = 1; - PhysicalExprNode then_expr = 2; -} - -message PhysicalInListNode { - PhysicalExprNode expr = 1; - repeated PhysicalExprNode list = 2; - bool negated = 3; -} - -message PhysicalCaseNode { - PhysicalExprNode expr = 1; - repeated PhysicalWhenThen when_then_expr = 2; - PhysicalExprNode else_expr = 3; -} - -message PhysicalScalarFunctionNode { - string name = 1; - datafusion.ScalarFunction fun = 2; - repeated PhysicalExprNode args = 3; - datafusion.ArrowType return_type = 4; -} - -message PhysicalTryCastNode { - PhysicalExprNode expr = 1; - datafusion.ArrowType arrow_type = 2; -} - -message PhysicalCastNode { - PhysicalExprNode expr = 1; - datafusion.ArrowType arrow_type = 2; -} - -message PhysicalNegativeNode { - PhysicalExprNode expr = 1; +message ShuffleWriterExecNode { + //TODO it seems redundant to provide job and stage id here since we also have them + // in the TaskDefinition that wraps this plan + string job_id = 1; + uint32 stage_id = 2; + datafusion.PhysicalPlanNode input = 3; + datafusion.PhysicalHashRepartition output_partitioning = 4; } message UnresolvedShuffleExecNode { @@ -225,132 +53,6 @@ message UnresolvedShuffleExecNode { uint32 output_partition_count = 4; } -message FilterExecNode { - PhysicalPlanNode input = 1; - PhysicalExprNode expr = 2; -} - -message FileGroup { - repeated PartitionedFile files = 1; -} - -message ScanLimit { - // wrap into a message to make it optional - uint32 limit = 1; -} - -message FileScanExecConf { - repeated FileGroup file_groups = 1; - datafusion.Schema schema = 2; - repeated uint32 projection = 4; - ScanLimit limit = 5; - Statistics statistics = 6; - repeated string table_partition_cols = 7; - string object_store_url = 8; -} - -message ParquetScanExecNode { - FileScanExecConf base_conf = 1; - datafusion.LogicalExprNode pruning_predicate = 2; -} - -message CsvScanExecNode { - FileScanExecConf base_conf = 1; - bool has_header = 2; - string delimiter = 3; -} - -message AvroScanExecNode { - FileScanExecConf base_conf = 1; -} - -enum PartitionMode { - COLLECT_LEFT = 0; - PARTITIONED = 1; - AUTO = 2; -} - -message HashJoinExecNode { - PhysicalPlanNode left = 1; - PhysicalPlanNode right = 2; - repeated JoinOn on = 3; - datafusion.JoinType join_type = 4; - PartitionMode partition_mode = 6; - bool null_equals_null = 7; - JoinFilter filter = 8; -} - -message UnionExecNode { - repeated PhysicalPlanNode inputs = 1; -} - -message ExplainExecNode { - datafusion.Schema schema = 1; - repeated datafusion.StringifiedPlan stringified_plans = 2; - bool verbose = 3; -} - -message CrossJoinExecNode { - PhysicalPlanNode left = 1; - PhysicalPlanNode right = 2; -} - -message PhysicalColumn { - string name = 1; - uint32 index = 2; -} - -message JoinOn { - PhysicalColumn left = 1; - PhysicalColumn right = 2; -} - -message EmptyExecNode { - bool produce_one_row = 1; - datafusion.Schema schema = 2; -} - -message ProjectionExecNode { - PhysicalPlanNode input = 1; - repeated PhysicalExprNode expr = 2; - repeated string expr_name = 3; -} - -enum AggregateMode { - PARTIAL = 0; - FINAL = 1; - FINAL_PARTITIONED = 2; -} - -message WindowAggExecNode { - PhysicalPlanNode input = 1; - repeated PhysicalExprNode window_expr = 2; - repeated string window_expr_name = 3; - datafusion.Schema input_schema = 4; -} - -message AggregateExecNode { - repeated PhysicalExprNode group_expr = 1; - repeated PhysicalExprNode aggr_expr = 2; - AggregateMode mode = 3; - PhysicalPlanNode input = 4; - repeated string group_expr_name = 5; - repeated string aggr_expr_name = 6; - // we need the input schema to the partial aggregate to pass to the final aggregate - datafusion.Schema input_schema = 7; - repeated PhysicalExprNode null_expr = 8; - repeated bool groups = 9; -} - -message ShuffleWriterExecNode { - //TODO it seems redundant to provide job and stage id here since we also have them - // in the TaskDefinition that wraps this plan - string job_id = 1; - uint32 stage_id = 2; - PhysicalPlanNode input = 3; - PhysicalHashRepartition output_partitioning = 4; -} - message ShuffleReaderExecNode { repeated ShuffleReaderPartition partition = 1; datafusion.Schema schema = 2; @@ -361,70 +63,6 @@ message ShuffleReaderPartition { repeated PartitionLocation location = 1; } -message GlobalLimitExecNode { - PhysicalPlanNode input = 1; - // The number of rows to skip before fetch - uint32 skip = 2; - // Maximum number of rows to fetch; negative means no limit - int64 fetch = 3; -} - -message LocalLimitExecNode { - PhysicalPlanNode input = 1; - uint32 fetch = 2; -} - -message SortExecNode { - PhysicalPlanNode input = 1; - repeated PhysicalExprNode expr = 2; - // Maximum number of highest/lowest rows to fetch; negative means no limit - int64 fetch = 3; -} - -message SortPreservingMergeExecNode { - PhysicalPlanNode input = 1; - repeated PhysicalExprNode expr = 2; -} - -message CoalesceBatchesExecNode { - PhysicalPlanNode input = 1; - uint32 target_batch_size = 2; -} - -message CoalescePartitionsExecNode { - PhysicalPlanNode input = 1; -} - -message PhysicalHashRepartition { - repeated PhysicalExprNode hash_expr = 1; - uint64 partition_count = 2; -} - -message RepartitionExecNode{ - PhysicalPlanNode input = 1; - oneof partition_method { - uint64 round_robin = 2; - PhysicalHashRepartition hash = 3; - uint64 unknown = 4; - } -} - -message JoinFilter{ - PhysicalExprNode expression = 1; - repeated ColumnIndex column_indices = 2; - datafusion.Schema schema = 3; -} - -message ColumnIndex{ - uint32 index = 1; - JoinSide side = 2; -} - -enum JoinSide{ - LEFT_SIDE = 0; - RIGHT_SIDE = 1; -} - /////////////////////////////////////////////////////////////////////////////////////////////////// // Ballista Scheduling /////////////////////////////////////////////////////////////////////////////////////////////////// @@ -451,16 +89,16 @@ message StageAttempts { message ExecutionGraphStage { oneof StageType { - UnResolvedStage unresolved_stage = 1; - ResolvedStage resolved_stage = 2; - SuccessfulStage successful_stage = 3; - FailedStage failed_stage = 4; + UnResolvedStage unresolved_stage = 1; + ResolvedStage resolved_stage = 2; + SuccessfulStage successful_stage = 3; + FailedStage failed_stage = 4; } } message UnResolvedStage { uint32 stage_id = 1; - PhysicalHashRepartition output_partitioning = 2; + datafusion.PhysicalHashRepartition output_partitioning = 2; repeated uint32 output_links = 3; repeated GraphStageInput inputs = 4; bytes plan = 5; @@ -471,7 +109,7 @@ message UnResolvedStage { message ResolvedStage { uint32 stage_id = 1; uint32 partitions = 2; - PhysicalHashRepartition output_partitioning = 3; + datafusion.PhysicalHashRepartition output_partitioning = 3; repeated uint32 output_links = 4; repeated GraphStageInput inputs = 5; bytes plan = 6; @@ -482,7 +120,7 @@ message ResolvedStage { message SuccessfulStage { uint32 stage_id = 1; uint32 partitions = 2; - PhysicalHashRepartition output_partitioning = 3; + datafusion.PhysicalHashRepartition output_partitioning = 3; repeated uint32 output_links = 4; repeated GraphStageInput inputs = 5; bytes plan = 6; @@ -494,7 +132,7 @@ message SuccessfulStage { message FailedStage { uint32 stage_id = 1; uint32 partitions = 2; - PhysicalHashRepartition output_partitioning = 3; + datafusion.PhysicalHashRepartition output_partitioning = 3; repeated uint32 output_links = 4; bytes plan = 5; repeated TaskInfo task_infos = 6; @@ -554,11 +192,11 @@ message ExecutePartition { string job_id = 1; uint32 stage_id = 2; repeated uint32 partition_id = 3; - PhysicalPlanNode plan = 4; + datafusion.PhysicalPlanNode plan = 4; // The task could need to read partitions from other executors repeated PartitionLocation partition_location = 5; // Output partition for shuffle writer - PhysicalHashRepartition output_partitioning = 6; + datafusion.PhysicalHashRepartition output_partitioning = 6; } message FetchPartition { @@ -797,7 +435,7 @@ message TaskDefinition { uint32 partition_id = 6; bytes plan = 7; // Output partition for shuffle writer - PhysicalHashRepartition output_partitioning = 8; + datafusion.PhysicalHashRepartition output_partitioning = 8; string session_id = 9; uint64 launch_time = 10; repeated KeyValuePair props = 11; @@ -811,7 +449,7 @@ message MultiTaskDefinition { uint32 stage_attempt_num = 4; bytes plan = 5; // Output partition for shuffle writer - PhysicalHashRepartition output_partitioning = 6; + datafusion.PhysicalHashRepartition output_partitioning = 6; string session_id = 7; uint64 launch_time = 8; repeated KeyValuePair props = 9; diff --git a/ballista/core/proto/datafusion.proto b/ballista/core/proto/datafusion.proto index 4152802db..8a9a8c0f6 100644 --- a/ballista/core/proto/datafusion.proto +++ b/ballista/core/proto/datafusion.proto @@ -71,6 +71,7 @@ message LogicalPlanNode { DistinctNode distinct = 23; ViewTableScanNode view_scan = 24; CustomTableScanNode custom_scan = 25; + PrepareNode prepare = 26; } } @@ -89,7 +90,8 @@ message CsvFormat { } message ParquetFormat { - bool enable_pruning = 1; + // Used to be bool enable_pruning = 1; + reserved 1; } message AvroFormat {} @@ -99,8 +101,8 @@ message ListingTableScanNode { repeated string paths = 2; string file_extension = 3; ProjectionColumns projection = 4; - datafusion.Schema schema = 5; - repeated datafusion.LogicalExprNode filters = 6; + Schema schema = 5; + repeated LogicalExprNode filters = 6; repeated string table_partition_cols = 7; bool collect_stat = 8; uint32 target_partitions = 9; @@ -109,13 +111,13 @@ message ListingTableScanNode { ParquetFormat parquet = 11; AvroFormat avro = 12; } - repeated datafusion.LogicalExprNode file_sort_order = 13; + repeated LogicalExprNode file_sort_order = 13; } message ViewTableScanNode { string table_name = 1; LogicalPlanNode input = 2; - datafusion.Schema schema = 3; + Schema schema = 3; ProjectionColumns projection = 4; string definition = 5; } @@ -124,14 +126,14 @@ message ViewTableScanNode { message CustomTableScanNode { string table_name = 1; ProjectionColumns projection = 2; - datafusion.Schema schema = 3; - repeated datafusion.LogicalExprNode filters = 4; + Schema schema = 3; + repeated LogicalExprNode filters = 4; bytes custom_table_data = 5; } message ProjectionNode { LogicalPlanNode input = 1; - repeated datafusion.LogicalExprNode expr = 2; + repeated LogicalExprNode expr = 2; oneof optional_alias { string alias = 3; } @@ -139,12 +141,12 @@ message ProjectionNode { message SelectionNode { LogicalPlanNode input = 1; - datafusion.LogicalExprNode expr = 2; + LogicalExprNode expr = 2; } message SortNode { LogicalPlanNode input = 1; - repeated datafusion.LogicalExprNode expr = 2; + repeated LogicalExprNode expr = 2; // Maximum number of highest/lowest rows to fetch; negative means no limit int64 fetch = 3; } @@ -158,7 +160,7 @@ message RepartitionNode { } message HashRepartition { - repeated datafusion.LogicalExprNode hash_expr = 1; + repeated LogicalExprNode hash_expr = 1; uint64 partition_count = 2; } @@ -167,11 +169,12 @@ message EmptyRelationNode { } message CreateExternalTableNode { - string name = 1; + reserved 1; // was string name + OwnedTableReference name = 12; string location = 2; string file_type = 3; bool has_header = 4; - datafusion.DfSchema schema = 5; + DfSchema schema = 5; repeated string table_partition_cols = 6; bool if_not_exists = 7; string delimiter = 8; @@ -180,20 +183,27 @@ message CreateExternalTableNode { map options = 11; } +message PrepareNode { + string name = 1; + repeated ArrowType data_types = 2; + LogicalPlanNode input = 3; +} + message CreateCatalogSchemaNode { string schema_name = 1; bool if_not_exists = 2; - datafusion.DfSchema schema = 3; + DfSchema schema = 3; } message CreateCatalogNode { string catalog_name = 1; bool if_not_exists = 2; - datafusion.DfSchema schema = 3; + DfSchema schema = 3; } message CreateViewNode { - string name = 1; + reserved 1; // was string name + OwnedTableReference name = 5; LogicalPlanNode input = 2; bool or_replace = 3; string definition = 4; @@ -203,7 +213,7 @@ message CreateViewNode { // the list is flattened, and with the field n_cols it can be parsed and partitioned into rows message ValuesNode { uint64 n_cols = 1; - repeated datafusion.LogicalExprNode values_list = 2; + repeated LogicalExprNode values_list = 2; } message AnalyzeNode { @@ -218,13 +228,13 @@ message ExplainNode { message AggregateNode { LogicalPlanNode input = 1; - repeated datafusion.LogicalExprNode group_expr = 2; - repeated datafusion.LogicalExprNode aggr_expr = 3; + repeated LogicalExprNode group_expr = 2; + repeated LogicalExprNode aggr_expr = 3; } message WindowNode { LogicalPlanNode input = 1; - repeated datafusion.LogicalExprNode window_expr = 2; + repeated LogicalExprNode window_expr = 2; } enum JoinType { @@ -248,8 +258,8 @@ message JoinNode { LogicalPlanNode right = 2; JoinType join_type = 3; JoinConstraint join_constraint = 4; - repeated datafusion.Column left_join_column = 5; - repeated datafusion.Column right_join_column = 6; + repeated LogicalExprNode left_join_key = 5; + repeated LogicalExprNode right_join_key = 6; bool null_equals_null = 7; LogicalExprNode filter = 8; } @@ -276,7 +286,7 @@ message LimitNode { } message SelectionExecNode { - datafusion.LogicalExprNode expr = 1; + LogicalExprNode expr = 1; } message SubqueryAliasNode { @@ -343,9 +353,16 @@ message LogicalExprNode { ILikeNode ilike = 32; SimilarToNode similar_to = 33; + PlaceholderNode placeholder = 34; + } } +message PlaceholderNode { + string id = 1; + ArrowType data_type = 2; +} + message LogicalExprList { repeated LogicalExprNode expr = 1; } @@ -429,76 +446,79 @@ message InListNode { } enum ScalarFunction { - Abs=0; - Acos=1; - Asin=2; - Atan=3; - Ascii=4; - Ceil=5; - Cos=6; - Digest=7; - Exp=8; - Floor=9; - Ln=10; - Log=11; - Log10=12; - Log2=13; - Round=14; - Signum=15; - Sin=16; - Sqrt=17; - Tan=18; - Trunc=19; - Array=20; - RegexpMatch=21; - BitLength=22; - Btrim=23; - CharacterLength=24; - Chr=25; - Concat=26; - ConcatWithSeparator=27; - DatePart=28; - DateTrunc=29; - InitCap=30; - Left=31; - Lpad=32; - Lower=33; - Ltrim=34; - MD5=35; - NullIf=36; - OctetLength=37; - Random=38; - RegexpReplace=39; - Repeat=40; - Replace=41; - Reverse=42; - Right=43; - Rpad=44; - Rtrim=45; - SHA224=46; - SHA256=47; - SHA384=48; - SHA512=49; - SplitPart=50; - StartsWith=51; - Strpos=52; - Substr=53; - ToHex=54; - ToTimestamp=55; - ToTimestampMillis=56; - ToTimestampMicros=57; - ToTimestampSeconds=58; - Now=59; - Translate=60; - Trim=61; - Upper=62; - Coalesce=63; - Power=64; - StructFun=65; - FromUnixtime=66; - Atan2=67; - DateBin=68; - ArrowTypeof=69; + Abs = 0; + Acos = 1; + Asin = 2; + Atan = 3; + Ascii = 4; + Ceil = 5; + Cos = 6; + Digest = 7; + Exp = 8; + Floor = 9; + Ln = 10; + Log = 11; + Log10 = 12; + Log2 = 13; + Round = 14; + Signum = 15; + Sin = 16; + Sqrt = 17; + Tan = 18; + Trunc = 19; + Array = 20; + RegexpMatch = 21; + BitLength = 22; + Btrim = 23; + CharacterLength = 24; + Chr = 25; + Concat = 26; + ConcatWithSeparator = 27; + DatePart = 28; + DateTrunc = 29; + InitCap = 30; + Left = 31; + Lpad = 32; + Lower = 33; + Ltrim = 34; + MD5 = 35; + NullIf = 36; + OctetLength = 37; + Random = 38; + RegexpReplace = 39; + Repeat = 40; + Replace = 41; + Reverse = 42; + Right = 43; + Rpad = 44; + Rtrim = 45; + SHA224 = 46; + SHA256 = 47; + SHA384 = 48; + SHA512 = 49; + SplitPart = 50; + StartsWith = 51; + Strpos = 52; + Substr = 53; + ToHex = 54; + ToTimestamp = 55; + ToTimestampMillis = 56; + ToTimestampMicros = 57; + ToTimestampSeconds = 58; + Now = 59; + Translate = 60; + Trim = 61; + Upper = 62; + Coalesce = 63; + Power = 64; + StructFun = 65; + FromUnixtime = 66; + Atan2 = 67; + DateBin = 68; + ArrowTypeof = 69; + CurrentDate = 70; + CurrentTime = 71; + Uuid = 72; } message ScalarFunctionNode { @@ -514,18 +534,18 @@ enum AggregateFunction { COUNT = 4; APPROX_DISTINCT = 5; ARRAY_AGG = 6; - VARIANCE=7; - VARIANCE_POP=8; - COVARIANCE=9; - COVARIANCE_POP=10; - STDDEV=11; - STDDEV_POP=12; - CORRELATION=13; + VARIANCE = 7; + VARIANCE_POP = 8; + COVARIANCE = 9; + COVARIANCE_POP = 10; + STDDEV = 11; + STDDEV_POP = 12; + CORRELATION = 13; APPROX_PERCENTILE_CONT = 14; - APPROX_MEDIAN=15; + APPROX_MEDIAN = 15; APPROX_PERCENTILE_CONT_WITH_WEIGHT = 16; GROUPING = 17; - MEDIAN=18; + MEDIAN = 18; } message AggregateExprNode { @@ -570,9 +590,7 @@ message WindowExprNode { repeated LogicalExprNode partition_by = 5; repeated LogicalExprNode order_by = 6; // repeated LogicalExprNode filter = 7; - oneof window_frame { - WindowFrame frame = 8; - } + WindowFrame window_frame = 8; } message BetweenNode { @@ -662,7 +680,7 @@ message WindowFrameBound { /////////////////////////////////////////////////////////////////////////////////////////////////// message Schema { - repeated datafusion.Field columns = 1; + repeated Field columns = 1; } message Field { @@ -675,53 +693,54 @@ message Field { } message FixedSizeBinary{ - int32 length = 1; + int32 length = 1; } message Timestamp{ - TimeUnit time_unit = 1; - string timezone = 2; + TimeUnit time_unit = 1; + string timezone = 2; } enum DateUnit{ - Day = 0; - DateMillisecond = 1; + Day = 0; + DateMillisecond = 1; } enum TimeUnit{ - Second = 0; - Millisecond = 1; - Microsecond = 2; - Nanosecond = 3; + Second = 0; + Millisecond = 1; + Microsecond = 2; + Nanosecond = 3; } enum IntervalUnit{ - YearMonth = 0; - DayTime = 1; - MonthDayNano = 2; + YearMonth = 0; + DayTime = 1; + MonthDayNano = 2; } message Decimal{ - uint64 whole = 1; - uint64 fractional = 2; + reserved 1, 2; + uint32 precision = 3; + int32 scale = 4; } message List{ - Field field_type = 1; + Field field_type = 1; } message FixedSizeList{ - Field field_type = 1; - int32 list_size = 2; + Field field_type = 1; + int32 list_size = 2; } message Dictionary{ - ArrowType key = 1; - ArrowType value = 2; + ArrowType key = 1; + ArrowType value = 2; } message Struct{ - repeated Field sub_field_types = 1; + repeated Field sub_field_types = 1; } enum UnionMode{ @@ -730,17 +749,17 @@ enum UnionMode{ } message Union{ - repeated Field union_types = 1; - UnionMode union_mode = 2; - repeated int32 type_ids = 3; + repeated Field union_types = 1; + UnionMode union_mode = 2; + repeated int32 type_ids = 3; } message ScalarListValue{ - // encode null explicitly to distinguish a list with a null value - // from a list with no values) - bool is_null = 3; - Field field = 1; - repeated ScalarValue values = 2; + // encode null explicitly to distinguish a list with a null value + // from a list with no values) + bool is_null = 3; + Field field = 1; + repeated ScalarValue values = 2; } message ScalarTime32Value { @@ -759,8 +778,8 @@ message ScalarTime64Value { message ScalarTimestampValue { oneof value { - int64 time_microsecond_value = 1; - int64 time_nanosecond_value = 2; + int64 time_microsecond_value = 1; + int64 time_nanosecond_value = 2; int64 time_second_value = 3; int64 time_millisecond_value = 4; }; @@ -787,8 +806,8 @@ message StructValue { } message ScalarFixedSizeBinary{ - bytes values = 1; - int32 length = 2; + bytes values = 1; + int32 length = 2; } message ScalarValue{ @@ -796,42 +815,42 @@ message ScalarValue{ reserved 19; oneof value { - // was PrimitiveScalarType null_value = 19; - // Null value of any type - ArrowType null_value = 33; - - bool bool_value = 1; - string utf8_value = 2; - string large_utf8_value = 3; - int32 int8_value = 4; - int32 int16_value = 5; - int32 int32_value = 6; - int64 int64_value = 7; - uint32 uint8_value = 8; - uint32 uint16_value = 9; - uint32 uint32_value = 10; - uint64 uint64_value = 11; - float float32_value = 12; - double float64_value = 13; - // Literal Date32 value always has a unit of day - int32 date_32_value = 14; - ScalarTime32Value time32_value = 15; - ScalarListValue list_value = 17; - //WAS: ScalarType null_list_value = 18; - - Decimal128 decimal128_value = 20; - int64 date_64_value = 21; - int32 interval_yearmonth_value = 24; - int64 interval_daytime_value = 25; - ScalarTimestampValue timestamp_value = 26; - ScalarDictionaryValue dictionary_value = 27; - bytes binary_value = 28; - bytes large_binary_value = 29; - ScalarTime64Value time64_value = 30; - IntervalMonthDayNanoValue interval_month_day_nano = 31; - StructValue struct_value = 32; - ScalarFixedSizeBinary fixed_size_binary_value = 34; - } + // was PrimitiveScalarType null_value = 19; + // Null value of any type + ArrowType null_value = 33; + + bool bool_value = 1; + string utf8_value = 2; + string large_utf8_value = 3; + int32 int8_value = 4; + int32 int16_value = 5; + int32 int32_value = 6; + int64 int64_value = 7; + uint32 uint8_value = 8; + uint32 uint16_value = 9; + uint32 uint32_value = 10; + uint64 uint64_value = 11; + float float32_value = 12; + double float64_value = 13; + // Literal Date32 value always has a unit of day + int32 date_32_value = 14; + ScalarTime32Value time32_value = 15; + ScalarListValue list_value = 17; + //WAS: ScalarType null_list_value = 18; + + Decimal128 decimal128_value = 20; + int64 date_64_value = 21; + int32 interval_yearmonth_value = 24; + int64 interval_daytime_value = 25; + ScalarTimestampValue timestamp_value = 26; + ScalarDictionaryValue dictionary_value = 27; + bytes binary_value = 28; + bytes large_binary_value = 29; + ScalarTime64Value time64_value = 30; + IntervalMonthDayNanoValue interval_month_day_nano = 31; + StructValue struct_value = 32; + ScalarFixedSizeBinary fixed_size_binary_value = 34; + } } message Decimal128{ @@ -842,40 +861,40 @@ message Decimal128{ // Serialized data type message ArrowType{ - oneof arrow_type_enum { - EmptyMessage NONE = 1; // arrow::Type::NA - EmptyMessage BOOL = 2; // arrow::Type::BOOL - EmptyMessage UINT8 = 3; // arrow::Type::UINT8 - EmptyMessage INT8 = 4; // arrow::Type::INT8 - EmptyMessage UINT16 =5; // represents arrow::Type fields in src/arrow/type.h - EmptyMessage INT16 = 6; - EmptyMessage UINT32 =7; - EmptyMessage INT32 = 8; - EmptyMessage UINT64 =9; - EmptyMessage INT64 =10 ; - EmptyMessage FLOAT16 =11 ; - EmptyMessage FLOAT32 =12 ; - EmptyMessage FLOAT64 =13 ; - EmptyMessage UTF8 =14 ; - EmptyMessage LARGE_UTF8 = 32; - EmptyMessage BINARY =15 ; - int32 FIXED_SIZE_BINARY =16 ; - EmptyMessage LARGE_BINARY = 31; - EmptyMessage DATE32 =17 ; - EmptyMessage DATE64 =18 ; - TimeUnit DURATION = 19; - Timestamp TIMESTAMP =20 ; - TimeUnit TIME32 =21 ; - TimeUnit TIME64 =22 ; - IntervalUnit INTERVAL =23 ; - Decimal DECIMAL =24 ; - List LIST =25; - List LARGE_LIST = 26; - FixedSizeList FIXED_SIZE_LIST = 27; - Struct STRUCT =28; - Union UNION =29; - Dictionary DICTIONARY =30; - } + oneof arrow_type_enum { + EmptyMessage NONE = 1; // arrow::Type::NA + EmptyMessage BOOL = 2; // arrow::Type::BOOL + EmptyMessage UINT8 = 3; // arrow::Type::UINT8 + EmptyMessage INT8 = 4; // arrow::Type::INT8 + EmptyMessage UINT16 = 5; // represents arrow::Type fields in src/arrow/type.h + EmptyMessage INT16 = 6; + EmptyMessage UINT32 = 7; + EmptyMessage INT32 = 8; + EmptyMessage UINT64 = 9; + EmptyMessage INT64 = 10 ; + EmptyMessage FLOAT16 = 11 ; + EmptyMessage FLOAT32 = 12 ; + EmptyMessage FLOAT64 = 13 ; + EmptyMessage UTF8 = 14 ; + EmptyMessage LARGE_UTF8 = 32; + EmptyMessage BINARY = 15 ; + int32 FIXED_SIZE_BINARY = 16 ; + EmptyMessage LARGE_BINARY = 31; + EmptyMessage DATE32 = 17 ; + EmptyMessage DATE64 = 18 ; + TimeUnit DURATION = 19; + Timestamp TIMESTAMP = 20 ; + TimeUnit TIME32 = 21 ; + TimeUnit TIME64 = 22 ; + IntervalUnit INTERVAL = 23 ; + Decimal DECIMAL = 24 ; + List LIST = 25; + List LARGE_LIST = 26; + FixedSizeList FIXED_SIZE_LIST = 27; + Struct STRUCT = 28; + Union UNION = 29; + Dictionary DICTIONARY = 30; + } } //Useful for representing an empty enum variant in rust @@ -911,4 +930,418 @@ message PlanType { message StringifiedPlan { PlanType plan_type = 1; string plan = 2; +} + +message BareTableReference { + string table = 1; +} + +message PartialTableReference { + string schema = 1; + string table = 2; +} + +message FullTableReference { + string catalog = 1; + string schema = 2; + string table = 3; +} + +message OwnedTableReference { + oneof table_reference_enum { + BareTableReference bare = 1; + PartialTableReference partial = 2; + FullTableReference full = 3; + } +} + +///////////////////////////////////////////////////////////////////////////////////////////////// + +// PhysicalPlanNode is a nested type +message PhysicalPlanNode { + oneof PhysicalPlanType { + ParquetScanExecNode parquet_scan = 1; + CsvScanExecNode csv_scan = 2; + EmptyExecNode empty = 3; + ProjectionExecNode projection = 4; + GlobalLimitExecNode global_limit = 6; + LocalLimitExecNode local_limit = 7; + AggregateExecNode aggregate = 8; + HashJoinExecNode hash_join = 9; + SortExecNode sort = 10; + CoalesceBatchesExecNode coalesce_batches = 11; + FilterExecNode filter = 12; + CoalescePartitionsExecNode merge = 13; + RepartitionExecNode repartition = 14; + WindowAggExecNode window = 15; + CrossJoinExecNode cross_join = 16; + AvroScanExecNode avro_scan = 17; + PhysicalExtensionNode extension = 18; + UnionExecNode union = 19; + ExplainExecNode explain = 20; + SortPreservingMergeExecNode sort_preserving_merge = 21; + } +} + +message PhysicalExtensionNode { + bytes node = 1; + repeated PhysicalPlanNode inputs = 2; +} + +// physical expressions +message PhysicalExprNode { + oneof ExprType { + // column references + PhysicalColumn column = 1; + + ScalarValue literal = 2; + + // binary expressions + PhysicalBinaryExprNode binary_expr = 3; + + // aggregate expressions + PhysicalAggregateExprNode aggregate_expr = 4; + + // null checks + PhysicalIsNull is_null_expr = 5; + PhysicalIsNotNull is_not_null_expr = 6; + PhysicalNot not_expr = 7; + + PhysicalCaseNode case_ = 8; + PhysicalCastNode cast = 9; + PhysicalSortExprNode sort = 10; + PhysicalNegativeNode negative = 11; + PhysicalInListNode in_list = 12; + PhysicalScalarFunctionNode scalar_function = 13; + PhysicalTryCastNode try_cast = 14; + + // window expressions + PhysicalWindowExprNode window_expr = 15; + + PhysicalScalarUdfNode scalar_udf = 16; + + PhysicalDateTimeIntervalExprNode date_time_interval_expr = 17; + + PhysicalLikeExprNode like_expr = 18; + } +} + +message PhysicalScalarUdfNode { + string name = 1; + repeated PhysicalExprNode args = 2; + ArrowType return_type = 4; +} + +message PhysicalAggregateExprNode { + AggregateFunction aggr_function = 1; + repeated PhysicalExprNode expr = 2; + bool distinct = 3; +} + +message PhysicalWindowExprNode { + oneof window_function { + AggregateFunction aggr_function = 1; + BuiltInWindowFunction built_in_function = 2; + // udaf = 3 + } + PhysicalExprNode expr = 4; +} + +message PhysicalIsNull { + PhysicalExprNode expr = 1; +} + +message PhysicalIsNotNull { + PhysicalExprNode expr = 1; +} + +message PhysicalNot { + PhysicalExprNode expr = 1; +} + +message PhysicalAliasNode { + PhysicalExprNode expr = 1; + string alias = 2; +} + +message PhysicalBinaryExprNode { + PhysicalExprNode l = 1; + PhysicalExprNode r = 2; + string op = 3; +} + +message PhysicalDateTimeIntervalExprNode { + PhysicalExprNode l = 1; + PhysicalExprNode r = 2; + string op = 3; +} + +message PhysicalLikeExprNode { + bool negated = 1; + bool case_insensitive = 2; + PhysicalExprNode expr = 3; + PhysicalExprNode pattern = 4; +} + +message PhysicalSortExprNode { + PhysicalExprNode expr = 1; + bool asc = 2; + bool nulls_first = 3; +} + +message PhysicalWhenThen { + PhysicalExprNode when_expr = 1; + PhysicalExprNode then_expr = 2; +} + +message PhysicalInListNode { + PhysicalExprNode expr = 1; + repeated PhysicalExprNode list = 2; + bool negated = 3; +} + +message PhysicalCaseNode { + PhysicalExprNode expr = 1; + repeated PhysicalWhenThen when_then_expr = 2; + PhysicalExprNode else_expr = 3; +} + +message PhysicalScalarFunctionNode { + string name = 1; + ScalarFunction fun = 2; + repeated PhysicalExprNode args = 3; + ArrowType return_type = 4; +} + +message PhysicalTryCastNode { + PhysicalExprNode expr = 1; + ArrowType arrow_type = 2; +} + +message PhysicalCastNode { + PhysicalExprNode expr = 1; + ArrowType arrow_type = 2; +} + +message PhysicalNegativeNode { + PhysicalExprNode expr = 1; +} + +message FilterExecNode { + PhysicalPlanNode input = 1; + PhysicalExprNode expr = 2; +} + +message FileGroup { + repeated PartitionedFile files = 1; +} + +message ScanLimit { + // wrap into a message to make it optional + uint32 limit = 1; +} + +message FileScanExecConf { + // Was repeated ConfigOption options = 10; + reserved 10; + + repeated FileGroup file_groups = 1; + Schema schema = 2; + repeated uint32 projection = 4; + ScanLimit limit = 5; + Statistics statistics = 6; + repeated string table_partition_cols = 7; + string object_store_url = 8; + repeated PhysicalSortExprNode output_ordering = 9; +} + +message ParquetScanExecNode { + FileScanExecConf base_conf = 1; + LogicalExprNode pruning_predicate = 2; +} + +message CsvScanExecNode { + FileScanExecConf base_conf = 1; + bool has_header = 2; + string delimiter = 3; +} + +message AvroScanExecNode { + FileScanExecConf base_conf = 1; +} + +enum PartitionMode { + COLLECT_LEFT = 0; + PARTITIONED = 1; + AUTO = 2; +} + +message HashJoinExecNode { + PhysicalPlanNode left = 1; + PhysicalPlanNode right = 2; + repeated JoinOn on = 3; + JoinType join_type = 4; + PartitionMode partition_mode = 6; + bool null_equals_null = 7; + JoinFilter filter = 8; +} + +message UnionExecNode { + repeated PhysicalPlanNode inputs = 1; +} + +message ExplainExecNode { + Schema schema = 1; + repeated StringifiedPlan stringified_plans = 2; + bool verbose = 3; +} + +message CrossJoinExecNode { + PhysicalPlanNode left = 1; + PhysicalPlanNode right = 2; +} + +message PhysicalColumn { + string name = 1; + uint32 index = 2; +} + +message JoinOn { + PhysicalColumn left = 1; + PhysicalColumn right = 2; +} + +message EmptyExecNode { + bool produce_one_row = 1; + Schema schema = 2; +} + +message ProjectionExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode expr = 2; + repeated string expr_name = 3; +} + +enum AggregateMode { + PARTIAL = 0; + FINAL = 1; + FINAL_PARTITIONED = 2; +} + +message WindowAggExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode window_expr = 2; + repeated string window_expr_name = 3; + Schema input_schema = 4; +} + +message AggregateExecNode { + repeated PhysicalExprNode group_expr = 1; + repeated PhysicalExprNode aggr_expr = 2; + AggregateMode mode = 3; + PhysicalPlanNode input = 4; + repeated string group_expr_name = 5; + repeated string aggr_expr_name = 6; + // we need the input schema to the partial aggregate to pass to the final aggregate + Schema input_schema = 7; + repeated PhysicalExprNode null_expr = 8; + repeated bool groups = 9; +} + +message GlobalLimitExecNode { + PhysicalPlanNode input = 1; + // The number of rows to skip before fetch + uint32 skip = 2; + // Maximum number of rows to fetch; negative means no limit + int64 fetch = 3; +} + +message LocalLimitExecNode { + PhysicalPlanNode input = 1; + uint32 fetch = 2; +} + +message SortExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode expr = 2; + // Maximum number of highest/lowest rows to fetch; negative means no limit + int64 fetch = 3; +} + +message SortPreservingMergeExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode expr = 2; +} + +message CoalesceBatchesExecNode { + PhysicalPlanNode input = 1; + uint32 target_batch_size = 2; +} + +message CoalescePartitionsExecNode { + PhysicalPlanNode input = 1; +} + +message PhysicalHashRepartition { + repeated PhysicalExprNode hash_expr = 1; + uint64 partition_count = 2; +} + +message RepartitionExecNode{ + PhysicalPlanNode input = 1; + oneof partition_method { + uint64 round_robin = 2; + PhysicalHashRepartition hash = 3; + uint64 unknown = 4; + } +} + +message JoinFilter{ + PhysicalExprNode expression = 1; + repeated ColumnIndex column_indices = 2; + Schema schema = 3; +} + +message ColumnIndex{ + uint32 index = 1; + JoinSide side = 2; +} + +enum JoinSide{ + LEFT_SIDE = 0; + RIGHT_SIDE = 1; +} + +message PartitionedFile { + string path = 1; + uint64 size = 2; + uint64 last_modified_ns = 3; + repeated ScalarValue partition_values = 4; + FileRange range = 5; +} + +message FileRange { + int64 start = 1; + int64 end = 2; +} + +message PartitionStats { + int64 num_rows = 1; + int64 num_batches = 2; + int64 num_bytes = 3; + repeated ColumnStats column_stats = 4; +} + +message Statistics { + int64 num_rows = 1; + int64 total_byte_size = 2; + repeated ColumnStats column_stats = 3; + bool is_exact = 4; +} + +message ColumnStats { + ScalarValue min_value = 1; + ScalarValue max_value = 2; + uint32 null_count = 3; + uint32 distinct_count = 4; } \ No newline at end of file diff --git a/ballista/core/src/error.rs b/ballista/core/src/error.rs index 53d779e15..289b2492a 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::from_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::to_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..3fc305ee4 100644 --- a/ballista/core/src/serde/generated/ballista.rs +++ b/ballista/core/src/serde/generated/ballista.rs @@ -1,310 +1,40 @@ +/// ///////////////////////////////////////////////////////////////////////////////////////////////// +/// Ballista Physical Plan +/// ///////////////////////////////////////////////////////////////////////////////////////////////// #[derive(Clone, PartialEq, ::prost::Message)] -pub struct Statistics { - #[prost(int64, tag = "1")] - pub num_rows: i64, - #[prost(int64, tag = "2")] - pub total_byte_size: i64, - #[prost(message, repeated, tag = "3")] - pub column_stats: ::prost::alloc::vec::Vec, - #[prost(bool, tag = "4")] - pub is_exact: bool, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct FileRange { - #[prost(int64, tag = "1")] - pub start: i64, - #[prost(int64, tag = "2")] - pub end: i64, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PartitionedFile { - #[prost(string, tag = "1")] - pub path: ::prost::alloc::string::String, - #[prost(uint64, tag = "2")] - pub size: u64, - #[prost(uint64, tag = "3")] - pub last_modified_ns: u64, - #[prost(message, repeated, tag = "4")] - pub partition_values: ::prost::alloc::vec::Vec< - ::datafusion_proto::protobuf::ScalarValue, +pub struct BallistaPhysicalPlanNode { + #[prost(oneof = "ballista_physical_plan_node::PhysicalPlanType", tags = "1, 2, 3")] + pub physical_plan_type: ::core::option::Option< + ballista_physical_plan_node::PhysicalPlanType, >, - #[prost(message, optional, tag = "5")] - pub range: ::core::option::Option, -} -/// PhysicalPlanNode is a nested type -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalPlanNode { - #[prost( - oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24" - )] - pub physical_plan_type: ::core::option::Option, } -/// Nested message and enum types in `PhysicalPlanNode`. -pub mod physical_plan_node { +/// Nested message and enum types in `BallistaPhysicalPlanNode`. +pub mod ballista_physical_plan_node { #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum PhysicalPlanType { #[prost(message, tag = "1")] - ParquetScan(super::ParquetScanExecNode), + ShuffleWriter(super::ShuffleWriterExecNode), #[prost(message, tag = "2")] - CsvScan(super::CsvScanExecNode), - #[prost(message, tag = "3")] - Empty(super::EmptyExecNode), - #[prost(message, tag = "4")] - Projection(::prost::alloc::boxed::Box), - #[prost(message, tag = "6")] - GlobalLimit(::prost::alloc::boxed::Box), - #[prost(message, tag = "7")] - LocalLimit(::prost::alloc::boxed::Box), - #[prost(message, tag = "8")] - Aggregate(::prost::alloc::boxed::Box), - #[prost(message, tag = "9")] - HashJoin(::prost::alloc::boxed::Box), - #[prost(message, tag = "10")] ShuffleReader(super::ShuffleReaderExecNode), - #[prost(message, tag = "11")] - Sort(::prost::alloc::boxed::Box), - #[prost(message, tag = "12")] - CoalesceBatches(::prost::alloc::boxed::Box), - #[prost(message, tag = "13")] - Filter(::prost::alloc::boxed::Box), - #[prost(message, tag = "14")] - Merge(::prost::alloc::boxed::Box), - #[prost(message, tag = "15")] - Unresolved(super::UnresolvedShuffleExecNode), - #[prost(message, tag = "16")] - Repartition(::prost::alloc::boxed::Box), - #[prost(message, tag = "17")] - Window(::prost::alloc::boxed::Box), - #[prost(message, tag = "18")] - ShuffleWriter(::prost::alloc::boxed::Box), - #[prost(message, tag = "19")] - CrossJoin(::prost::alloc::boxed::Box), - #[prost(message, tag = "20")] - AvroScan(super::AvroScanExecNode), - #[prost(message, tag = "21")] - Extension(super::PhysicalExtensionNode), - #[prost(message, tag = "22")] - Union(super::UnionExecNode), - #[prost(message, tag = "23")] - Explain(super::ExplainExecNode), - #[prost(message, tag = "24")] - SortPreservingMerge( - ::prost::alloc::boxed::Box, - ), - } -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalExtensionNode { - #[prost(bytes = "vec", tag = "1")] - pub node: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "2")] - pub inputs: ::prost::alloc::vec::Vec, -} -/// physical expressions -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalExprNode { - #[prost( - oneof = "physical_expr_node::ExprType", - tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17" - )] - pub expr_type: ::core::option::Option, -} -/// Nested message and enum types in `PhysicalExprNode`. -pub mod physical_expr_node { - #[derive(Clone, PartialEq, ::prost::Oneof)] - pub enum ExprType { - /// column references - #[prost(message, tag = "1")] - Column(super::PhysicalColumn), - #[prost(message, tag = "2")] - Literal(::datafusion_proto::protobuf::ScalarValue), - /// binary expressions #[prost(message, tag = "3")] - BinaryExpr(::prost::alloc::boxed::Box), - /// aggregate expressions - #[prost(message, tag = "4")] - AggregateExpr(super::PhysicalAggregateExprNode), - /// null checks - #[prost(message, tag = "5")] - IsNullExpr(::prost::alloc::boxed::Box), - #[prost(message, tag = "6")] - IsNotNullExpr(::prost::alloc::boxed::Box), - #[prost(message, tag = "7")] - NotExpr(::prost::alloc::boxed::Box), - #[prost(message, tag = "8")] - Case(::prost::alloc::boxed::Box), - #[prost(message, tag = "9")] - Cast(::prost::alloc::boxed::Box), - #[prost(message, tag = "10")] - Sort(::prost::alloc::boxed::Box), - #[prost(message, tag = "11")] - Negative(::prost::alloc::boxed::Box), - #[prost(message, tag = "12")] - InList(::prost::alloc::boxed::Box), - #[prost(message, tag = "13")] - ScalarFunction(super::PhysicalScalarFunctionNode), - #[prost(message, tag = "14")] - TryCast(::prost::alloc::boxed::Box), - /// window expressions - #[prost(message, tag = "15")] - WindowExpr(::prost::alloc::boxed::Box), - #[prost(message, tag = "16")] - ScalarUdf(super::PhysicalScalarUdfNode), - #[prost(message, tag = "17")] - DateTimeIntervalExpr( - ::prost::alloc::boxed::Box, - ), + UnresolvedShuffle(super::UnresolvedShuffleExecNode), } } #[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalScalarUdfNode { +pub struct ShuffleWriterExecNode { + /// TODO it seems redundant to provide job and stage id here since we also have them + /// in the TaskDefinition that wraps this plan #[prost(string, tag = "1")] - pub name: ::prost::alloc::string::String, - #[prost(message, repeated, tag = "2")] - pub args: ::prost::alloc::vec::Vec, + pub job_id: ::prost::alloc::string::String, + #[prost(uint32, tag = "2")] + pub stage_id: u32, + #[prost(message, optional, tag = "3")] + pub input: ::core::option::Option<::datafusion_proto::protobuf::PhysicalPlanNode>, #[prost(message, optional, tag = "4")] - pub return_type: ::core::option::Option<::datafusion_proto::protobuf::ArrowType>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalAggregateExprNode { - #[prost(enumeration = "::datafusion_proto::protobuf::AggregateFunction", tag = "1")] - pub aggr_function: i32, - #[prost(message, repeated, tag = "2")] - pub expr: ::prost::alloc::vec::Vec, - #[prost(bool, tag = "3")] - pub distinct: bool, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalWindowExprNode { - #[prost(message, optional, boxed, tag = "4")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(oneof = "physical_window_expr_node::WindowFunction", tags = "1, 2")] - pub window_function: ::core::option::Option< - physical_window_expr_node::WindowFunction, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, >, } -/// Nested message and enum types in `PhysicalWindowExprNode`. -pub mod physical_window_expr_node { - #[derive(Clone, PartialEq, ::prost::Oneof)] - pub enum WindowFunction { - #[prost( - enumeration = "::datafusion_proto::protobuf::AggregateFunction", - tag = "1" - )] - AggrFunction(i32), - /// udaf = 3 - #[prost( - enumeration = "::datafusion_proto::protobuf::BuiltInWindowFunction", - tag = "2" - )] - BuiltInFunction(i32), - } -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalIsNull { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalIsNotNull { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalNot { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalAliasNode { - #[prost(message, optional, tag = "1")] - pub expr: ::core::option::Option, - #[prost(string, tag = "2")] - pub alias: ::prost::alloc::string::String, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalBinaryExprNode { - #[prost(message, optional, boxed, tag = "1")] - pub l: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "2")] - pub r: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(string, tag = "3")] - pub op: ::prost::alloc::string::String, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalDateTimeIntervalExprNode { - #[prost(message, optional, boxed, tag = "1")] - pub l: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "2")] - pub r: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(string, tag = "3")] - pub op: ::prost::alloc::string::String, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalSortExprNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(bool, tag = "2")] - pub asc: bool, - #[prost(bool, tag = "3")] - pub nulls_first: bool, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalWhenThen { - #[prost(message, optional, tag = "1")] - pub when_expr: ::core::option::Option, - #[prost(message, optional, tag = "2")] - pub then_expr: ::core::option::Option, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalInListNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, repeated, tag = "2")] - pub list: ::prost::alloc::vec::Vec, - #[prost(bool, tag = "3")] - pub negated: bool, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalCaseNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, repeated, tag = "2")] - pub when_then_expr: ::prost::alloc::vec::Vec, - #[prost(message, optional, boxed, tag = "3")] - pub else_expr: ::core::option::Option<::prost::alloc::boxed::Box>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalScalarFunctionNode { - #[prost(string, tag = "1")] - pub name: ::prost::alloc::string::String, - #[prost(enumeration = "::datafusion_proto::protobuf::ScalarFunction", tag = "2")] - pub fun: i32, - #[prost(message, repeated, tag = "3")] - pub args: ::prost::alloc::vec::Vec, - #[prost(message, optional, tag = "4")] - pub return_type: ::core::option::Option<::datafusion_proto::protobuf::ArrowType>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalTryCastNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, tag = "2")] - pub arrow_type: ::core::option::Option<::datafusion_proto::protobuf::ArrowType>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalCastNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, tag = "2")] - pub arrow_type: ::core::option::Option<::datafusion_proto::protobuf::ArrowType>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalNegativeNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, -} #[derive(Clone, PartialEq, ::prost::Message)] pub struct UnresolvedShuffleExecNode { #[prost(uint32, tag = "1")] @@ -317,180 +47,6 @@ pub struct UnresolvedShuffleExecNode { pub output_partition_count: u32, } #[derive(Clone, PartialEq, ::prost::Message)] -pub struct FilterExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, tag = "2")] - pub expr: ::core::option::Option, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct FileGroup { - #[prost(message, repeated, tag = "1")] - pub files: ::prost::alloc::vec::Vec, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ScanLimit { - /// wrap into a message to make it optional - #[prost(uint32, tag = "1")] - pub limit: u32, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct FileScanExecConf { - #[prost(message, repeated, tag = "1")] - pub file_groups: ::prost::alloc::vec::Vec, - #[prost(message, optional, tag = "2")] - pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, - #[prost(uint32, repeated, tag = "4")] - pub projection: ::prost::alloc::vec::Vec, - #[prost(message, optional, tag = "5")] - pub limit: ::core::option::Option, - #[prost(message, optional, tag = "6")] - pub statistics: ::core::option::Option, - #[prost(string, repeated, tag = "7")] - pub table_partition_cols: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, - #[prost(string, tag = "8")] - pub object_store_url: ::prost::alloc::string::String, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ParquetScanExecNode { - #[prost(message, optional, tag = "1")] - pub base_conf: ::core::option::Option, - #[prost(message, optional, tag = "2")] - pub pruning_predicate: ::core::option::Option< - ::datafusion_proto::protobuf::LogicalExprNode, - >, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CsvScanExecNode { - #[prost(message, optional, tag = "1")] - pub base_conf: ::core::option::Option, - #[prost(bool, tag = "2")] - pub has_header: bool, - #[prost(string, tag = "3")] - pub delimiter: ::prost::alloc::string::String, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct AvroScanExecNode { - #[prost(message, optional, tag = "1")] - pub base_conf: ::core::option::Option, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct HashJoinExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub left: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "2")] - pub right: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, repeated, tag = "3")] - pub on: ::prost::alloc::vec::Vec, - #[prost(enumeration = "::datafusion_proto::protobuf::JoinType", tag = "4")] - pub join_type: i32, - #[prost(enumeration = "PartitionMode", tag = "6")] - pub partition_mode: i32, - #[prost(bool, tag = "7")] - pub null_equals_null: bool, - #[prost(message, optional, tag = "8")] - pub filter: ::core::option::Option, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct UnionExecNode { - #[prost(message, repeated, tag = "1")] - pub inputs: ::prost::alloc::vec::Vec, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ExplainExecNode { - #[prost(message, optional, tag = "1")] - pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, - #[prost(message, repeated, tag = "2")] - pub stringified_plans: ::prost::alloc::vec::Vec< - ::datafusion_proto::protobuf::StringifiedPlan, - >, - #[prost(bool, tag = "3")] - pub verbose: bool, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CrossJoinExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub left: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "2")] - pub right: ::core::option::Option<::prost::alloc::boxed::Box>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalColumn { - #[prost(string, tag = "1")] - pub name: ::prost::alloc::string::String, - #[prost(uint32, tag = "2")] - pub index: u32, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct JoinOn { - #[prost(message, optional, tag = "1")] - pub left: ::core::option::Option, - #[prost(message, optional, tag = "2")] - pub right: ::core::option::Option, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct EmptyExecNode { - #[prost(bool, tag = "1")] - pub produce_one_row: bool, - #[prost(message, optional, tag = "2")] - pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ProjectionExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, repeated, tag = "2")] - pub expr: ::prost::alloc::vec::Vec, - #[prost(string, repeated, tag = "3")] - pub expr_name: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct WindowAggExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, repeated, tag = "2")] - pub window_expr: ::prost::alloc::vec::Vec, - #[prost(string, repeated, tag = "3")] - pub window_expr_name: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, - #[prost(message, optional, tag = "4")] - pub input_schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct AggregateExecNode { - #[prost(message, repeated, tag = "1")] - pub group_expr: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "2")] - pub aggr_expr: ::prost::alloc::vec::Vec, - #[prost(enumeration = "AggregateMode", tag = "3")] - pub mode: i32, - #[prost(message, optional, boxed, tag = "4")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(string, repeated, tag = "5")] - pub group_expr_name: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, - #[prost(string, repeated, tag = "6")] - pub aggr_expr_name: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, - /// we need the input schema to the partial aggregate to pass to the final aggregate - #[prost(message, optional, tag = "7")] - pub input_schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, - #[prost(message, repeated, tag = "8")] - pub null_expr: ::prost::alloc::vec::Vec, - #[prost(bool, repeated, tag = "9")] - pub groups: ::prost::alloc::vec::Vec, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ShuffleWriterExecNode { - /// TODO it seems redundant to provide job and stage id here since we also have them - /// in the TaskDefinition that wraps this plan - #[prost(string, tag = "1")] - pub job_id: ::prost::alloc::string::String, - #[prost(uint32, tag = "2")] - pub stage_id: u32, - #[prost(message, optional, boxed, tag = "3")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, tag = "4")] - pub output_partitioning: ::core::option::Option, -} -#[derive(Clone, PartialEq, ::prost::Message)] pub struct ShuffleReaderExecNode { #[prost(message, repeated, tag = "1")] pub partition: ::prost::alloc::vec::Vec, @@ -503,95 +59,6 @@ pub struct ShuffleReaderPartition { #[prost(message, repeated, tag = "1")] pub location: ::prost::alloc::vec::Vec, } -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct GlobalLimitExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - /// The number of rows to skip before fetch - #[prost(uint32, tag = "2")] - pub skip: u32, - /// Maximum number of rows to fetch; negative means no limit - #[prost(int64, tag = "3")] - pub fetch: i64, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct LocalLimitExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(uint32, tag = "2")] - pub fetch: u32, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct SortExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, repeated, tag = "2")] - pub expr: ::prost::alloc::vec::Vec, - /// Maximum number of highest/lowest rows to fetch; negative means no limit - #[prost(int64, tag = "3")] - pub fetch: i64, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct SortPreservingMergeExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, repeated, tag = "2")] - pub expr: ::prost::alloc::vec::Vec, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CoalesceBatchesExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(uint32, tag = "2")] - pub target_batch_size: u32, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CoalescePartitionsExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalHashRepartition { - #[prost(message, repeated, tag = "1")] - pub hash_expr: ::prost::alloc::vec::Vec, - #[prost(uint64, tag = "2")] - pub partition_count: u64, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct RepartitionExecNode { - #[prost(message, optional, boxed, tag = "1")] - pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(oneof = "repartition_exec_node::PartitionMethod", tags = "2, 3, 4")] - pub partition_method: ::core::option::Option, -} -/// Nested message and enum types in `RepartitionExecNode`. -pub mod repartition_exec_node { - #[derive(Clone, PartialEq, ::prost::Oneof)] - pub enum PartitionMethod { - #[prost(uint64, tag = "2")] - RoundRobin(u64), - #[prost(message, tag = "3")] - Hash(super::PhysicalHashRepartition), - #[prost(uint64, tag = "4")] - Unknown(u64), - } -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct JoinFilter { - #[prost(message, optional, tag = "1")] - pub expression: ::core::option::Option, - #[prost(message, repeated, tag = "2")] - pub column_indices: ::prost::alloc::vec::Vec, - #[prost(message, optional, tag = "3")] - pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ColumnIndex { - #[prost(uint32, tag = "1")] - pub index: u32, - #[prost(enumeration = "JoinSide", tag = "2")] - pub side: i32, -} /// ///////////////////////////////////////////////////////////////////////////////////////////////// /// Ballista Scheduling /// ///////////////////////////////////////////////////////////////////////////////////////////////// @@ -655,7 +122,9 @@ pub struct UnResolvedStage { #[prost(uint32, tag = "1")] pub stage_id: u32, #[prost(message, optional, tag = "2")] - pub output_partitioning: ::core::option::Option, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, #[prost(uint32, repeated, tag = "3")] pub output_links: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "4")] @@ -676,7 +145,9 @@ pub struct ResolvedStage { #[prost(uint32, tag = "2")] pub partitions: u32, #[prost(message, optional, tag = "3")] - pub output_partitioning: ::core::option::Option, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, #[prost(uint32, repeated, tag = "4")] pub output_links: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "5")] @@ -697,7 +168,9 @@ pub struct SuccessfulStage { #[prost(uint32, tag = "2")] pub partitions: u32, #[prost(message, optional, tag = "3")] - pub output_partitioning: ::core::option::Option, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, #[prost(uint32, repeated, tag = "4")] pub output_links: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "5")] @@ -718,7 +191,9 @@ pub struct FailedStage { #[prost(uint32, tag = "2")] pub partitions: u32, #[prost(message, optional, tag = "3")] - pub output_partitioning: ::core::option::Option, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, #[prost(uint32, repeated, tag = "4")] pub output_links: ::prost::alloc::vec::Vec, #[prost(bytes = "vec", tag = "5")] @@ -817,13 +292,15 @@ pub struct ExecutePartition { #[prost(uint32, repeated, tag = "3")] pub partition_id: ::prost::alloc::vec::Vec, #[prost(message, optional, tag = "4")] - pub plan: ::core::option::Option, + pub plan: ::core::option::Option<::datafusion_proto::protobuf::PhysicalPlanNode>, /// The task could need to read partitions from other executors #[prost(message, repeated, tag = "5")] pub partition_location: ::prost::alloc::vec::Vec, /// Output partition for shuffle writer #[prost(message, optional, tag = "6")] - pub output_partitioning: ::core::option::Option, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct FetchPartition { @@ -1211,7 +688,9 @@ pub struct TaskDefinition { pub plan: ::prost::alloc::vec::Vec, /// Output partition for shuffle writer #[prost(message, optional, tag = "8")] - pub output_partitioning: ::core::option::Option, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, #[prost(string, tag = "9")] pub session_id: ::prost::alloc::string::String, #[prost(uint64, tag = "10")] @@ -1234,7 +713,9 @@ pub struct MultiTaskDefinition { pub plan: ::prost::alloc::vec::Vec, /// Output partition for shuffle writer #[prost(message, optional, tag = "6")] - pub output_partitioning: ::core::option::Option, + pub output_partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, #[prost(string, tag = "7")] pub session_id: ::prost::alloc::string::String, #[prost(uint64, tag = "8")] @@ -1492,64 +973,6 @@ pub struct RunningTaskInfo { #[prost(uint32, tag = "4")] pub partition_id: u32, } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] -pub enum PartitionMode { - CollectLeft = 0, - Partitioned = 1, - Auto = 2, -} -impl PartitionMode { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - PartitionMode::CollectLeft => "COLLECT_LEFT", - PartitionMode::Partitioned => "PARTITIONED", - PartitionMode::Auto => "AUTO", - } - } -} -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] -pub enum AggregateMode { - Partial = 0, - Final = 1, - FinalPartitioned = 2, -} -impl AggregateMode { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - AggregateMode::Partial => "PARTIAL", - AggregateMode::Final => "FINAL", - AggregateMode::FinalPartitioned => "FINAL_PARTITIONED", - } - } -} -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] -pub enum JoinSide { - LeftSide = 0, - RightSide = 1, -} -impl JoinSide { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - JoinSide::LeftSide => "LEFT_SIDE", - JoinSide::RightSide => "RIGHT_SIDE", - } - } -} /// Generated client implementations. pub mod scheduler_grpc_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] @@ -1985,7 +1408,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 +1954,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..59e696746 100644 --- a/ballista/core/src/serde/mod.rs +++ b/ballista/core/src/serde/mod.rs @@ -19,25 +19,32 @@ //! as convenience code for interacting with the generated code. use crate::{error::BallistaError, serde::scheduler::Action as BallistaAction}; + use arrow_flight::sql::ProstMessageExt; -use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::common::DataFusionError; use datafusion::execution::FunctionRegistry; -use datafusion::physical_plan::joins::utils::JoinSide; -use datafusion::physical_plan::ExecutionPlan; -use datafusion_proto::logical_plan::{ - AsLogicalPlan, DefaultLogicalExtensionCodec, LogicalExtensionCodec, +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use datafusion_proto::common::proto_error; +use datafusion_proto::physical_plan::from_proto::parse_protobuf_hash_partitioning; +use datafusion_proto::{ + convert_required, + logical_plan::{AsLogicalPlan, DefaultLogicalExtensionCodec, LogicalExtensionCodec}, + physical_plan::{AsExecutionPlan, PhysicalExtensionCodec}, }; -use prost::bytes::BufMut; use prost::Message; use std::fmt::Debug; use std::marker::PhantomData; use std::sync::Arc; use std::{convert::TryInto, io::Cursor}; +use crate::execution_plans::{ + ShuffleReaderExec, ShuffleWriterExec, UnresolvedShuffleExec, +}; +use crate::serde::protobuf::ballista_physical_plan_node::PhysicalPlanType; +use crate::serde::scheduler::PartitionLocation; pub use generated::ballista as protobuf; pub mod generated; -pub mod physical_plan; pub mod scheduler; impl ProstMessageExt for protobuf::Action { @@ -61,76 +68,6 @@ pub fn decode_protobuf(bytes: &[u8]) -> Result { .and_then(|node| node.try_into()) } -pub(crate) fn proto_error>(message: S) -> BallistaError { - BallistaError::General(message.into()) -} - -pub trait AsExecutionPlan: Debug + Send + Sync + Clone { - fn try_decode(buf: &[u8]) -> Result - where - Self: Sized; - - fn try_encode(&self, buf: &mut B) -> Result<(), BallistaError> - where - B: BufMut, - Self: Sized; - - fn try_into_physical_plan( - &self, - registry: &dyn FunctionRegistry, - runtime: &RuntimeEnv, - extension_codec: &dyn PhysicalExtensionCodec, - ) -> Result, BallistaError>; - - fn try_from_physical_plan( - plan: Arc, - extension_codec: &dyn PhysicalExtensionCodec, - ) -> Result - where - Self: Sized; -} - -pub trait PhysicalExtensionCodec: Debug + Send + Sync { - fn try_decode( - &self, - buf: &[u8], - inputs: &[Arc], - registry: &dyn FunctionRegistry, - ) -> Result, BallistaError>; - - fn try_encode( - &self, - node: Arc, - buf: &mut Vec, - ) -> Result<(), BallistaError>; -} - -#[derive(Debug, Clone)] -pub struct DefaultPhysicalExtensionCodec {} - -impl PhysicalExtensionCodec for DefaultPhysicalExtensionCodec { - fn try_decode( - &self, - _buf: &[u8], - _inputs: &[Arc], - _registry: &dyn FunctionRegistry, - ) -> Result, BallistaError> { - Err(BallistaError::NotImplemented( - "PhysicalExtensionCodec is not provided".to_string(), - )) - } - - fn try_encode( - &self, - _node: Arc, - _buf: &mut Vec, - ) -> Result<(), BallistaError> { - Err(BallistaError::NotImplemented( - "PhysicalExtensionCodec is not provided".to_string(), - )) - } -} - #[derive(Clone, Debug)] pub struct BallistaCodec { logical_extension_codec: Arc, @@ -145,7 +82,7 @@ impl Default fn default() -> Self { Self { logical_extension_codec: Arc::new(DefaultLogicalExtensionCodec {}), - physical_extension_codec: Arc::new(DefaultPhysicalExtensionCodec {}), + physical_extension_codec: Arc::new(BallistaPhysicalExtensionCodec {}), logical_plan_repr: PhantomData, physical_plan_repr: PhantomData, } @@ -174,490 +111,188 @@ impl BallistaCodec {{ - if let Some(field) = $PB.as_ref() { - Ok(field - .try_into() - .map_err(|_| proto_error("Failed to convert!"))?) - } else { - Err(proto_error("Missing required field in protobuf")) - } - }}; -} - -#[macro_export] -macro_rules! into_required { - ($PB:expr) => {{ - if let Some(field) = $PB.as_ref() { - Ok(field.into()) - } else { - Err(proto_error("Missing required field in protobuf")) - } - }}; -} +#[derive(Debug)] +pub struct BallistaPhysicalExtensionCodec {} -#[macro_export] -macro_rules! convert_box_required { - ($PB:expr) => {{ - if let Some(field) = $PB.as_ref() { - field.as_ref().try_into() - } else { - Err(proto_error("Missing required field in protobuf")) - } - }}; -} - -impl From for JoinSide { - fn from(t: protobuf::JoinSide) -> Self { - match t { - protobuf::JoinSide::LeftSide => JoinSide::Left, - protobuf::JoinSide::RightSide => JoinSide::Right, - } - } -} - -impl From for protobuf::JoinSide { - fn from(t: JoinSide) -> Self { - match t { - JoinSide::Left => protobuf::JoinSide::LeftSide, - JoinSide::Right => protobuf::JoinSide::RightSide, - } - } -} - -fn byte_to_string(b: u8) -> Result { - let b = &[b]; - let b = std::str::from_utf8(b) - .map_err(|_| BallistaError::General("Invalid CSV delimiter".to_owned()))?; - Ok(b.to_owned()) -} - -fn str_to_byte(s: &str) -> Result { - if s.len() != 1 { - return Err(BallistaError::General("Invalid CSV delimiter".to_owned())); - } - Ok(s.as_bytes()[0]) -} - -#[cfg(test)] -mod tests { - use async_trait::async_trait; - use datafusion::arrow::datatypes::SchemaRef; - use datafusion::common::DFSchemaRef; - use datafusion::error::DataFusionError; - use datafusion::execution::{ - context::{QueryPlanner, SessionState, TaskContext}, - runtime_env::{RuntimeConfig, RuntimeEnv}, - FunctionRegistry, - }; - use datafusion::logical_expr::{ - col, Expr, Extension, LogicalPlan, UserDefinedLogicalNode, - }; - use datafusion::physical_plan::expressions::PhysicalSortExpr; - use datafusion::physical_plan::planner::{DefaultPhysicalPlanner, ExtensionPlanner}; - use datafusion::physical_plan::{ - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalPlanner, - SendableRecordBatchStream, Statistics, - }; - use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext}; - 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}; - use std::ops::Deref; - use std::sync::Arc; - - pub mod proto { - #[derive(Clone, PartialEq, ::prost::Message)] - pub struct TopKPlanProto { - #[prost(uint64, tag = "1")] - pub k: u64, - - #[prost(message, optional, tag = "2")] - pub expr: ::core::option::Option, - } - - #[derive(Clone, Eq, PartialEq, ::prost::Message)] - pub struct TopKExecProto { - #[prost(uint64, tag = "1")] - pub k: u64, - } - } - - use crate::error::BallistaError; - use crate::serde::protobuf::PhysicalPlanNode; - use crate::serde::{ - AsExecutionPlan, AsLogicalPlan, LogicalExtensionCodec, PhysicalExtensionCodec, - }; - use crate::utils::with_object_store_provider; - use datafusion_proto::protobuf::LogicalPlanNode; - use proto::{TopKExecProto, TopKPlanProto}; - - struct TopKPlanNode { - k: usize, - input: LogicalPlan, - /// The sort expression (this example only supports a single sort - /// expr) - expr: Expr, - } - - impl TopKPlanNode { - pub fn new(k: usize, input: LogicalPlan, expr: Expr) -> Self { - Self { k, input, expr } - } - } - - impl Debug for TopKPlanNode { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - self.fmt_for_explain(f) - } - } - - impl UserDefinedLogicalNode for TopKPlanNode { - fn as_any(&self) -> &dyn Any { - self - } - - fn inputs(&self) -> Vec<&LogicalPlan> { - vec![&self.input] - } - - /// Schema for TopK is the same as the input - fn schema(&self) -> &DFSchemaRef { - self.input.schema() - } - - fn expressions(&self) -> Vec { - vec![self.expr.clone()] - } - - /// For example: `TopK: k=10` - fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "TopK: k={}", self.k) - } - - fn from_template( - &self, - exprs: &[Expr], - inputs: &[LogicalPlan], - ) -> Arc { - assert_eq!(inputs.len(), 1, "input size inconsistent"); - assert_eq!(exprs.len(), 1, "expression size inconsistent"); - Arc::new(TopKPlanNode { - k: self.k, - input: inputs[0].clone(), - expr: exprs[0].clone(), - }) - } - } - - struct TopKExec { - input: Arc, - /// The maxium number of values - k: usize, - } - - impl TopKExec { - pub fn new(k: usize, input: Arc) -> Self { - Self { input, k } - } - } - - impl Debug for TopKExec { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - write!(f, "TopKExec") - } - } - - impl ExecutionPlan for TopKExec { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - fn required_input_distribution(&self) -> Vec { - vec![Distribution::SinglePartition] - } - - fn children(&self) -> Vec> { - vec![self.input.clone()] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> datafusion::error::Result> { - Ok(Arc::new(TopKExec { - input: children[0].clone(), - k: self.k, - })) - } - - /// Execute one partition and return an iterator over RecordBatch - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> datafusion::error::Result { - Err(DataFusionError::NotImplemented( - "not implemented".to_string(), - )) - } - - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default => { - write!(f, "TopKExec: k={}", self.k) - } +impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { + fn try_decode( + &self, + buf: &[u8], + inputs: &[Arc], + registry: &dyn FunctionRegistry, + ) -> Result, DataFusionError> { + let ballista_plan: protobuf::BallistaPhysicalPlanNode = + protobuf::BallistaPhysicalPlanNode::decode(buf).map_err(|e| { + DataFusionError::Internal(format!( + "Could not deserialize BallistaPhysicalPlanNode: {}", + e + )) + })?; + + let ballista_plan = + ballista_plan.physical_plan_type.as_ref().ok_or_else(|| { + DataFusionError::Internal( + "Could not deserialize BallistaPhysicalPlanNode because it's physical_plan_type is none".to_string() + ) + })?; + + match ballista_plan { + PhysicalPlanType::ShuffleWriter(shuffle_writer) => { + let input = inputs[0].clone(); + + let shuffle_output_partitioning = parse_protobuf_hash_partitioning( + shuffle_writer.output_partitioning.as_ref(), + registry, + input.schema().as_ref(), + )?; + + Ok(Arc::new(ShuffleWriterExec::try_new( + shuffle_writer.job_id.clone(), + shuffle_writer.stage_id as usize, + input, + "".to_string(), // this is intentional but hacky - the executor will fill this in + shuffle_output_partitioning, + )?)) } - } - - fn statistics(&self) -> Statistics { - // to improve the optimizability of this plan - // better statistics inference could be provided - Statistics::default() - } - } - - struct TopKPlanner {} - - #[async_trait] - impl ExtensionPlanner for TopKPlanner { - /// Create a physical plan for an extension node - async fn plan_extension( - &self, - _planner: &dyn PhysicalPlanner, - node: &dyn UserDefinedLogicalNode, - logical_inputs: &[&LogicalPlan], - physical_inputs: &[Arc], - _session_state: &SessionState, - ) -> datafusion::error::Result>> { - Ok( - if let Some(topk_node) = node.as_any().downcast_ref::() { - assert_eq!(logical_inputs.len(), 1, "Inconsistent number of inputs"); - assert_eq!(physical_inputs.len(), 1, "Inconsistent number of inputs"); - // figure out input name - Some(Arc::new(TopKExec { - input: physical_inputs[0].clone(), - k: topk_node.k, - })) - } else { - None - }, - ) - } - } - - struct TopKQueryPlanner {} - - #[async_trait] - impl QueryPlanner for TopKQueryPlanner { - /// Given a `LogicalPlan` created from above, create an - /// `ExecutionPlan` suitable for execution - async fn create_physical_plan( - &self, - logical_plan: &LogicalPlan, - session_state: &SessionState, - ) -> datafusion::error::Result> { - // Teach the default physical planner how to plan TopK nodes. - let physical_planner = - DefaultPhysicalPlanner::with_extension_planners(vec![Arc::new( - TopKPlanner {}, - )]); - // Delegate most work of physical planning to the default physical planner - physical_planner - .create_physical_plan(logical_plan, session_state) - .await - } - } - - #[derive(Debug)] - pub struct TopKExtensionCodec {} - - impl LogicalExtensionCodec for TopKExtensionCodec { - fn try_decode( - &self, - buf: &[u8], - inputs: &[LogicalPlan], - ctx: &SessionContext, - ) -> Result { - if let Some((input, _)) = inputs.split_first() { - let proto = TopKPlanProto::decode(buf).map_err(|e| { - DataFusionError::Internal(format!( - "failed to decode logical plan: {:?}", - e - )) - })?; - - if let Some(expr) = proto.expr.as_ref() { - let node = TopKPlanNode::new( - proto.k as usize, - input.clone(), - parse_expr(expr, ctx)?, - ); - - Ok(Extension { - node: Arc::new(node), + PhysicalPlanType::ShuffleReader(shuffle_reader) => { + let schema = Arc::new(convert_required!(shuffle_reader.schema)?); + let partition_location: Vec> = shuffle_reader + .partition + .iter() + .map(|p| { + p.location + .iter() + .map(|l| { + l.clone().try_into().map_err(|e| { + DataFusionError::Internal(format!( + "Fail to get partition location due to {:?}", + e + )) + }) + }) + .collect::, _>>() }) - } else { - Err(DataFusionError::Plan("invalid plan, no expr".to_string())) - } - } else { - Err(DataFusionError::Plan("invalid plan, no input".to_string())) + .collect::, DataFusionError>>()?; + let shuffle_reader = + ShuffleReaderExec::try_new(partition_location, schema)?; + Ok(Arc::new(shuffle_reader)) } - } - - fn try_encode( - &self, - node: &Extension, - buf: &mut Vec, - ) -> Result<(), DataFusionError> { - if let Some(exec) = node.node.as_any().downcast_ref::() { - let proto = TopKPlanProto { - k: exec.k as u64, - expr: Some((&exec.expr).try_into()?), - }; - - proto.encode(buf).map_err(|e| { - DataFusionError::Internal(format!( - "failed to encode logical plan: {:?}", - e - )) - })?; - - Ok(()) - } else { - Err(DataFusionError::Plan("unsupported plan type".to_string())) + PhysicalPlanType::UnresolvedShuffle(unresolved_shuffle) => { + let schema = Arc::new(convert_required!(unresolved_shuffle.schema)?); + Ok(Arc::new(UnresolvedShuffleExec { + stage_id: unresolved_shuffle.stage_id as usize, + schema, + input_partition_count: unresolved_shuffle.input_partition_count + as usize, + output_partition_count: unresolved_shuffle.output_partition_count + as usize, + })) } } - - fn try_decode_table_provider( - &self, - _buf: &[u8], - _schema: SchemaRef, - _ctx: &SessionContext, - ) -> Result, DataFusionError> - { - unimplemented!() - } - - fn try_encode_table_provider( - &self, - _node: Arc, - _buf: &mut Vec, - ) -> Result<(), DataFusionError> { - unimplemented!() - } } - impl PhysicalExtensionCodec for TopKExtensionCodec { - fn try_decode( - &self, - buf: &[u8], - inputs: &[Arc], - _registry: &dyn FunctionRegistry, - ) -> Result, BallistaError> { - if let Some((input, _)) = inputs.split_first() { - let proto = TopKExecProto::decode(buf).map_err(|e| { - BallistaError::Internal(format!( - "failed to decode execution plan: {:?}", - e - )) - })?; - Ok(Arc::new(TopKExec::new(proto.k as usize, input.clone()))) - } else { - Err(BallistaError::from("invalid plan, no input".to_string())) - } - } - - fn try_encode( - &self, - node: Arc, - buf: &mut Vec, - ) -> Result<(), BallistaError> { - if let Some(exec) = node.as_any().downcast_ref::() { - let proto = TopKExecProto { k: exec.k as u64 }; - - proto.encode(buf).map_err(|e| { - BallistaError::Internal(format!( - "failed to encode execution plan: {:?}", - e - )) - })?; - - Ok(()) - } else { - Err(BallistaError::from("unsupported plan type".to_string())) + fn try_encode( + &self, + node: Arc, + buf: &mut Vec, + ) -> Result<(), DataFusionError> { + if let Some(exec) = node.as_any().downcast_ref::() { + // note that we use shuffle_output_partitioning() rather than output_partitioning() + // to get the true output partitioning + let output_partitioning = match exec.shuffle_output_partitioning() { + Some(Partitioning::Hash(exprs, partition_count)) => { + Some(datafusion_proto::protobuf::PhysicalHashRepartition { + hash_expr: exprs + .iter() + .map(|expr| expr.clone().try_into()) + .collect::, DataFusionError>>()?, + partition_count: *partition_count as u64, + }) + } + None => None, + other => { + return Err(DataFusionError::Internal(format!( + "physical_plan::to_proto() invalid partitioning for ShuffleWriterExec: {:?}", + other + ))); + } + }; + + let proto = protobuf::BallistaPhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::ShuffleWriter( + protobuf::ShuffleWriterExecNode { + job_id: exec.job_id().to_string(), + stage_id: exec.stage_id() as u32, + input: None, + output_partitioning, + }, + )), + }; + + proto.encode(buf).map_err(|e| { + DataFusionError::Internal(format!( + "failed to encode shuffle writer execution plan: {:?}", + e + )) + })?; + + Ok(()) + } else if let Some(exec) = node.as_any().downcast_ref::() { + let mut partition = vec![]; + for location in &exec.partition { + partition.push(protobuf::ShuffleReaderPartition { + location: location + .iter() + .map(|l| { + l.clone().try_into().map_err(|e| { + DataFusionError::Internal(format!( + "Fail to get partition location due to {:?}", + e + )) + }) + }) + .collect::, _>>()?, + }); } + let proto = protobuf::BallistaPhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::ShuffleReader( + protobuf::ShuffleReaderExecNode { + partition, + schema: Some(exec.schema().as_ref().try_into()?), + }, + )), + }; + proto.encode(buf).map_err(|e| { + DataFusionError::Internal(format!( + "failed to encode shuffle reader execution plan: {:?}", + e + )) + })?; + + Ok(()) + } else if let Some(exec) = node.as_any().downcast_ref::() { + let proto = protobuf::BallistaPhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::UnresolvedShuffle( + protobuf::UnresolvedShuffleExecNode { + stage_id: exec.stage_id as u32, + schema: Some(exec.schema().as_ref().try_into()?), + input_partition_count: exec.input_partition_count as u32, + output_partition_count: exec.output_partition_count as u32, + }, + )), + }; + proto.encode(buf).map_err(|e| { + DataFusionError::Internal(format!( + "failed to encode unresolved shuffle execution plan: {:?}", + e + )) + })?; + + Ok(()) + } else { + Err(DataFusionError::Internal( + "unsupported plan type".to_string(), + )) } } - - #[tokio::test] - async fn test_extension_plan() -> crate::error::Result<()> { - let runtime = Arc::new( - RuntimeEnv::new(with_object_store_provider(RuntimeConfig::default())) - .unwrap(), - ); - let session_state = - SessionState::with_config_rt(SessionConfig::new(), runtime.clone()) - .with_query_planner(Arc::new(TopKQueryPlanner {})); - - let ctx = SessionContext::with_state(session_state); - - let scan = ctx - .read_csv("tests/customer.csv", CsvReadOptions::default()) - .await? - .to_logical_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 extension_codec = TopKExtensionCodec {}; - - 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) - ); - - let proto = PhysicalPlanNode::try_from_physical_plan( - topk_exec.clone(), - &extension_codec, - )?; - let physical_round_trip = - proto.try_into_physical_plan(&ctx, runtime.deref(), &extension_codec)?; - - 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 deleted file mode 100644 index eb951c763..000000000 --- a/ballista/core/src/serde/physical_plan/from_proto.rs +++ /dev/null @@ -1,418 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Serde code to convert from protocol buffers to Rust data structures. - -use std::convert::{TryFrom, TryInto}; -use std::ops::Deref; -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; -use datafusion::execution::FunctionRegistry; -use datafusion::logical_expr::window_function::WindowFunction; -use datafusion::physical_expr::expressions::DateTimeIntervalExpr; -use datafusion::physical_expr::ScalarFunctionExpr; -use datafusion::physical_plan::file_format::FileScanConfig; -use datafusion::physical_plan::{ - expressions::{ - BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, - Literal, NegativeExpr, NotExpr, TryCastExpr, DEFAULT_DATAFUSION_CAST_OPTIONS, - }, - functions, Partitioning, -}; -use datafusion::physical_plan::{ColumnStatistics, PhysicalExpr, Statistics}; -use datafusion_proto::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; - -use crate::convert_required; -use crate::error::BallistaError; -use crate::serde::{proto_error, protobuf}; - -impl From<&protobuf::PhysicalColumn> for Column { - fn from(c: &protobuf::PhysicalColumn) -> Column { - Column::new(&c.name, c.index as usize) - } -} - -pub(crate) fn parse_physical_expr( - proto: &protobuf::PhysicalExprNode, - registry: &dyn FunctionRegistry, - input_schema: &Schema, -) -> Result, BallistaError> { - let expr_type = proto - .expr_type - .as_ref() - .ok_or_else(|| proto_error("Unexpected empty physical expression"))?; - - let pexpr: Arc = match expr_type { - ExprType::Column(c) => { - let pcol: Column = c.into(); - Arc::new(pcol) - } - ExprType::Literal(scalar) => Arc::new(Literal::new(scalar.try_into()?)), - ExprType::BinaryExpr(binary_expr) => Arc::new(BinaryExpr::new( - parse_required_physical_box_expr( - &binary_expr.l, - registry, - "left", - input_schema, - )?, - from_proto_binary_op(&binary_expr.op)?, - parse_required_physical_box_expr( - &binary_expr.r, - registry, - "right", - input_schema, - )?, - )), - ExprType::DateTimeIntervalExpr(expr) => Arc::new(DateTimeIntervalExpr::try_new( - parse_required_physical_box_expr(&expr.l, registry, "left", input_schema)?, - from_proto_binary_op(&expr.op)?, - parse_required_physical_box_expr(&expr.r, registry, "right", input_schema)?, - input_schema, - )?), - ExprType::AggregateExpr(_) => { - return Err(BallistaError::General( - "Cannot convert aggregate expr node to physical expression".to_owned(), - )); - } - ExprType::WindowExpr(_) => { - return Err(BallistaError::General( - "Cannot convert window expr node to physical expression".to_owned(), - )); - } - ExprType::Sort(_) => { - return Err(BallistaError::General( - "Cannot convert sort expr node to physical expression".to_owned(), - )); - } - ExprType::IsNullExpr(e) => Arc::new(IsNullExpr::new( - parse_required_physical_box_expr(&e.expr, registry, "expr", input_schema)?, - )), - ExprType::IsNotNullExpr(e) => Arc::new(IsNotNullExpr::new( - parse_required_physical_box_expr(&e.expr, registry, "expr", input_schema)?, - )), - ExprType::NotExpr(e) => Arc::new(NotExpr::new(parse_required_physical_box_expr( - &e.expr, - registry, - "expr", - input_schema, - )?)), - ExprType::Negative(e) => Arc::new(NegativeExpr::new( - parse_required_physical_box_expr(&e.expr, registry, "expr", input_schema)?, - )), - ExprType::InList(e) => Arc::new(InListExpr::new( - parse_required_physical_box_expr(&e.expr, registry, "expr", input_schema)?, - e.list - .iter() - .map(|x| parse_physical_expr(x, registry, input_schema)) - .collect::, _>>()?, - e.negated, - input_schema, - )), - ExprType::Case(e) => Arc::new(CaseExpr::try_new( - e.expr - .as_ref() - .map(|e| parse_physical_expr(e.as_ref(), registry, input_schema)) - .transpose()?, - e.when_then_expr - .iter() - .map(|e| { - Ok(( - parse_required_physical_expr( - &e.when_expr, - registry, - "when_expr", - input_schema, - )?, - parse_required_physical_expr( - &e.then_expr, - registry, - "then_expr", - input_schema, - )?, - )) - }) - .collect::, BallistaError>>()?, - e.else_expr - .as_ref() - .map(|e| parse_physical_expr(e.as_ref(), registry, input_schema)) - .transpose()?, - )?), - ExprType::Cast(e) => Arc::new(CastExpr::new( - parse_required_physical_box_expr(&e.expr, registry, "expr", input_schema)?, - convert_required!(e.arrow_type)?, - DEFAULT_DATAFUSION_CAST_OPTIONS, - )), - ExprType::TryCast(e) => Arc::new(TryCastExpr::new( - parse_required_physical_box_expr(&e.expr, registry, "expr", input_schema)?, - convert_required!(e.arrow_type)?, - )), - ExprType::ScalarFunction(e) => { - let scalar_function = datafusion_proto::protobuf::ScalarFunction::from_i32( - e.fun, - ) - .ok_or_else(|| { - proto_error(format!("Received an unknown scalar function: {}", e.fun,)) - })?; - - let args = e - .args - .iter() - .map(|x| parse_physical_expr(x, registry, input_schema)) - .collect::, _>>()?; - - // TODO Do not create new the ExecutionProps - let execution_props = ExecutionProps::new(); - - let fun_expr = functions::create_physical_fun( - &(&scalar_function).into(), - &execution_props, - )?; - - Arc::new(ScalarFunctionExpr::new( - &e.name, - fun_expr, - args, - &convert_required!(e.return_type)?, - )) - } - ExprType::ScalarUdf(e) => { - let scalar_fun = registry.udf(e.name.as_str())?.deref().clone().fun; - - let args = e - .args - .iter() - .map(|x| parse_physical_expr(x, registry, input_schema)) - .collect::, _>>()?; - - Arc::new(ScalarFunctionExpr::new( - e.name.as_str(), - scalar_fun, - args, - &convert_required!(e.return_type)?, - )) - } - }; - - Ok(pexpr) -} - -fn parse_required_physical_box_expr( - expr: &Option>, - registry: &dyn FunctionRegistry, - field: &str, - input_schema: &Schema, -) -> Result, BallistaError> { - expr.as_ref() - .map(|e| parse_physical_expr(e.as_ref(), registry, input_schema)) - .transpose()? - .ok_or_else(|| { - BallistaError::General(format!("Missing required field {:?}", field)) - }) -} - -fn parse_required_physical_expr( - expr: &Option, - registry: &dyn FunctionRegistry, - field: &str, - input_schema: &Schema, -) -> Result, BallistaError> { - expr.as_ref() - .map(|e| parse_physical_expr(e, registry, input_schema)) - .transpose()? - .ok_or_else(|| { - BallistaError::General(format!("Missing required field {:?}", field)) - }) -} - -impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFunction { - type Error = BallistaError; - - fn try_from( - expr: &protobuf::physical_window_expr_node::WindowFunction, - ) -> Result { - match expr { - protobuf::physical_window_expr_node::WindowFunction::AggrFunction(n) => { - let f = datafusion_proto::protobuf::AggregateFunction::from_i32(*n) - .ok_or_else(|| { - proto_error(format!( - "Received an unknown window aggregate function: {}", - n - )) - })?; - - Ok(WindowFunction::AggregateFunction(f.into())) - } - protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(n) => { - let f = datafusion_proto::protobuf::BuiltInWindowFunction::from_i32(*n) - .ok_or_else(|| { - proto_error(format!( - "Received an unknown window builtin function: {}", - n - )) - })?; - - Ok(WindowFunction::BuiltInWindowFunction(f.into())) - } - } - } -} - -pub fn parse_protobuf_hash_partitioning( - partitioning: Option<&protobuf::PhysicalHashRepartition>, - registry: &dyn FunctionRegistry, - input_schema: &Schema, -) -> Result, BallistaError> { - match partitioning { - Some(hash_part) => { - let expr = hash_part - .hash_expr - .iter() - .map(|e| parse_physical_expr(e, registry, input_schema)) - .collect::>, _>>()?; - - Ok(Some(Partitioning::Hash( - expr, - hash_part.partition_count.try_into().unwrap(), - ))) - } - None => Ok(None), - } -} - -impl TryFrom<&protobuf::PartitionedFile> for PartitionedFile { - type Error = BallistaError; - - fn try_from(val: &protobuf::PartitionedFile) -> Result { - Ok(PartitionedFile { - object_meta: ObjectMeta { - location: Path::from(val.path.as_str()), - last_modified: Utc.timestamp_nanos(val.last_modified_ns as i64), - size: val.size as usize, - }, - partition_values: val - .partition_values - .iter() - .map(|v| v.try_into()) - .collect::, _>>()?, - range: val.range.as_ref().map(|v| v.try_into()).transpose()?, - extensions: None, - }) - } -} - -impl TryFrom<&protobuf::FileRange> for FileRange { - type Error = BallistaError; - - fn try_from(value: &protobuf::FileRange) -> Result { - Ok(FileRange { - start: value.start, - end: value.end, - }) - } -} - -impl TryFrom<&protobuf::FileGroup> for Vec { - type Error = BallistaError; - - fn try_from(val: &protobuf::FileGroup) -> Result { - val.files - .iter() - .map(|f| f.try_into()) - .collect::, _>>() - } -} - -impl From<&protobuf::ColumnStats> for ColumnStatistics { - fn from(cs: &protobuf::ColumnStats) -> ColumnStatistics { - ColumnStatistics { - null_count: Some(cs.null_count as usize), - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - distinct_count: Some(cs.distinct_count as usize), - } - } -} - -impl TryInto for &protobuf::Statistics { - type Error = BallistaError; - - fn try_into(self) -> Result { - let column_statistics = self - .column_stats - .iter() - .map(|s| s.into()) - .collect::>(); - Ok(Statistics { - num_rows: Some(self.num_rows as usize), - total_byte_size: Some(self.total_byte_size as usize), - // No column statistic (None) is encoded with empty array - column_statistics: if column_statistics.is_empty() { - None - } else { - Some(column_statistics) - }, - is_exact: self.is_exact, - }) - } -} - -impl TryInto for &protobuf::FileScanExecConf { - type Error = BallistaError; - - fn try_into(self) -> Result { - let schema = Arc::new(convert_required!(self.schema)?); - let projection = self - .projection - .iter() - .map(|i| *i as usize) - .collect::>(); - let projection = if projection.is_empty() { - None - } else { - Some(projection) - }; - 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 - .file_groups - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?, - statistics, - projection, - limit: self.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_cols: vec![], - // TODO add ordering info to the ballista proto file - output_ordering: None, - }) - } -} diff --git a/ballista/core/src/serde/physical_plan/mod.rs b/ballista/core/src/serde/physical_plan/mod.rs deleted file mode 100644 index cf7e54978..000000000 --- a/ballista/core/src/serde/physical_plan/mod.rs +++ /dev/null @@ -1,1707 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::convert::TryInto; -use std::sync::Arc; - -use datafusion::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; -use prost::bytes::BufMut; -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; -use datafusion::execution::runtime_env::RuntimeEnv; -use datafusion::execution::FunctionRegistry; -use datafusion::logical_expr::WindowFrame; -use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateMode}; -use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; -use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion::physical_plan::empty::EmptyExec; -use datafusion::physical_plan::explain::ExplainExec; -use datafusion::physical_plan::expressions::{Column, PhysicalSortExpr}; -use datafusion::physical_plan::file_format::{ - AvroExec, CsvExec, FileScanConfig, ParquetExec, -}; -use datafusion::physical_plan::filter::FilterExec; -use datafusion::physical_plan::joins::CrossJoinExec; -use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; -use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion::physical_plan::projection::ProjectionExec; -use datafusion::physical_plan::repartition::RepartitionExec; -use datafusion::physical_plan::sorts::sort::SortExec; -use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion::physical_plan::union::UnionExec; -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 crate::error::BallistaError; -use crate::execution_plans::{ - ShuffleReaderExec, ShuffleWriterExec, UnresolvedShuffleExec, -}; -use crate::serde::physical_plan::from_proto::{ - parse_physical_expr, parse_protobuf_hash_partitioning, -}; -use crate::serde::protobuf::physical_expr_node::ExprType; -use crate::serde::protobuf::physical_plan_node::PhysicalPlanType; -use crate::serde::protobuf::repartition_exec_node::PartitionMethod; -use crate::serde::protobuf::{PhysicalExtensionNode, PhysicalPlanNode}; -use crate::serde::scheduler::PartitionLocation; -use crate::serde::{ - byte_to_string, proto_error, protobuf, str_to_byte, AsExecutionPlan, - PhysicalExtensionCodec, -}; -use crate::{convert_required, into_physical_plan, into_required}; - -pub mod from_proto; -pub mod to_proto; - -impl AsExecutionPlan for PhysicalPlanNode { - fn try_decode(buf: &[u8]) -> Result - where - Self: Sized, - { - PhysicalPlanNode::decode(buf).map_err(|e| { - BallistaError::Internal(format!("failed to decode physical plan: {:?}", e)) - }) - } - - fn try_encode(&self, buf: &mut B) -> Result<(), BallistaError> - where - B: BufMut, - Self: Sized, - { - self.encode(buf).map_err(|e| { - BallistaError::Internal(format!("failed to encode physical plan: {:?}", e)) - }) - } - - #[allow(clippy::only_used_in_recursion)] - fn try_into_physical_plan( - &self, - registry: &dyn FunctionRegistry, - runtime: &RuntimeEnv, - extension_codec: &dyn PhysicalExtensionCodec, - ) -> Result, BallistaError> { - let plan = self.physical_plan_type.as_ref().ok_or_else(|| { - proto_error(format!( - "physical_plan::from_proto() Unsupported physical plan '{:?}'", - self - )) - })?; - match plan { - PhysicalPlanType::Explain(explain) => Ok(Arc::new(ExplainExec::new( - Arc::new(explain.schema.as_ref().unwrap().try_into()?), - explain - .stringified_plans - .iter() - .map(|plan| plan.into()) - .collect(), - explain.verbose, - ))), - PhysicalPlanType::Projection(projection) => { - let input: Arc = into_physical_plan!( - projection.input, - registry, - runtime, - extension_codec - )?; - let exprs = projection - .expr - .iter() - .zip(projection.expr_name.iter()) - .map(|(expr, name)| Ok((parse_physical_expr(expr,registry, input.schema().as_ref())?, name.to_string()))) - .collect::, String)>, BallistaError>>( - )?; - Ok(Arc::new(ProjectionExec::try_new(exprs, input)?)) - } - PhysicalPlanType::Filter(filter) => { - let input: Arc = into_physical_plan!( - filter.input, - registry, - runtime, - extension_codec - )?; - let predicate = filter - .expr - .as_ref() - .map(|expr| { - parse_physical_expr(expr, registry, input.schema().as_ref()) - }) - .transpose()? - .ok_or_else(|| { - BallistaError::General( - "filter (FilterExecNode) in PhysicalPlanNode is missing." - .to_owned(), - ) - })?; - Ok(Arc::new(FilterExec::try_new(predicate, input)?)) - } - PhysicalPlanType::CsvScan(scan) => Ok(Arc::new(CsvExec::new( - decode_scan_config(scan.base_conf.as_ref().unwrap())?, - scan.has_header, - str_to_byte(&scan.delimiter)?, - FileCompressionType::UNCOMPRESSED, - ))), - PhysicalPlanType::ParquetScan(scan) => { - let predicate = scan - .pruning_predicate - .as_ref() - .map(|expr| parse_expr(expr, registry)) - .transpose()?; - Ok(Arc::new(ParquetExec::new( - decode_scan_config(scan.base_conf.as_ref().unwrap())?, - predicate, - None, - ))) - } - PhysicalPlanType::AvroScan(scan) => Ok(Arc::new(AvroExec::new( - decode_scan_config(scan.base_conf.as_ref().unwrap())?, - ))), - PhysicalPlanType::CoalesceBatches(coalesce_batches) => { - let input: Arc = into_physical_plan!( - coalesce_batches.input, - registry, - runtime, - extension_codec - )?; - Ok(Arc::new(CoalesceBatchesExec::new( - input, - coalesce_batches.target_batch_size as usize, - ))) - } - PhysicalPlanType::Merge(merge) => { - let input: Arc = - into_physical_plan!(merge.input, registry, runtime, extension_codec)?; - Ok(Arc::new(CoalescePartitionsExec::new(input))) - } - PhysicalPlanType::Repartition(repart) => { - let input: Arc = into_physical_plan!( - repart.input, - registry, - runtime, - extension_codec - )?; - match repart.partition_method { - Some(PartitionMethod::Hash(ref hash_part)) => { - let expr = hash_part - .hash_expr - .iter() - .map(|e| { - parse_physical_expr(e, registry, input.schema().as_ref()) - }) - .collect::>, _>>()?; - - Ok(Arc::new(RepartitionExec::try_new( - input, - Partitioning::Hash( - expr, - hash_part.partition_count.try_into().unwrap(), - ), - )?)) - } - Some(PartitionMethod::RoundRobin(partition_count)) => { - Ok(Arc::new(RepartitionExec::try_new( - input, - Partitioning::RoundRobinBatch( - partition_count.try_into().unwrap(), - ), - )?)) - } - Some(PartitionMethod::Unknown(partition_count)) => { - Ok(Arc::new(RepartitionExec::try_new( - input, - Partitioning::UnknownPartitioning( - partition_count.try_into().unwrap(), - ), - )?)) - } - _ => Err(BallistaError::General( - "Invalid partitioning scheme".to_owned(), - )), - } - } - PhysicalPlanType::GlobalLimit(limit) => { - let input: Arc = - into_physical_plan!(limit.input, registry, runtime, extension_codec)?; - let fetch = if limit.fetch >= 0 { - Some(limit.fetch as usize) - } else { - None - }; - Ok(Arc::new(GlobalLimitExec::new( - input, - limit.skip as usize, - fetch, - ))) - } - PhysicalPlanType::LocalLimit(limit) => { - let input: Arc = - into_physical_plan!(limit.input, registry, runtime, extension_codec)?; - Ok(Arc::new(LocalLimitExec::new(input, limit.fetch as usize))) - } - PhysicalPlanType::Window(window_agg) => { - let input: Arc = into_physical_plan!( - window_agg.input, - registry, - runtime, - extension_codec - )?; - let input_schema = window_agg - .input_schema - .as_ref() - .ok_or_else(|| { - BallistaError::General( - "input_schema in WindowAggrNode is missing.".to_owned(), - ) - })? - .clone(); - let physical_schema: SchemaRef = - SchemaRef::new((&input_schema).try_into()?); - - let physical_window_expr: Vec> = window_agg - .window_expr - .iter() - .zip(window_agg.window_expr_name.iter()) - .map(|(expr, name)| { - let expr_type = expr.expr_type.as_ref().ok_or_else(|| { - proto_error("Unexpected empty window physical expression") - })?; - - match expr_type { - ExprType::WindowExpr(window_node) => { - let window_node_expr = window_node - .expr - .as_ref() - .map(|e| { - parse_physical_expr( - e.as_ref(), - registry, - &physical_schema, - ) - }) - .transpose()? - .ok_or_else(|| { - proto_error( - "missing window_node expr expression" - .to_string(), - ) - })?; - - Ok(create_window_expr( - &convert_required!(window_node.window_function)?, - name.to_owned(), - &[window_node_expr], - &[], - &[], - Some(Arc::new(WindowFrame::default())), - &physical_schema, - )?) - } - _ => Err(BallistaError::General( - "Invalid expression for WindowAggrExec".to_string(), - )), - } - }) - .collect::, _>>()?; - - Ok(Arc::new(WindowAggExec::try_new( - physical_window_expr, - input, - Arc::new((&input_schema).try_into()?), - vec![], - None, - )?)) - } - PhysicalPlanType::Aggregate(hash_agg) => { - let input: Arc = into_physical_plan!( - hash_agg.input, - registry, - runtime, - extension_codec - )?; - let mode = protobuf::AggregateMode::from_i32(hash_agg.mode).ok_or_else( - || { - proto_error(format!( - "Received a AggregateNode message with unknown AggregateMode {}", - hash_agg.mode - )) - }, - )?; - let agg_mode: AggregateMode = match mode { - protobuf::AggregateMode::Partial => AggregateMode::Partial, - protobuf::AggregateMode::Final => AggregateMode::Final, - protobuf::AggregateMode::FinalPartitioned => { - AggregateMode::FinalPartitioned - } - }; - - let num_expr = hash_agg.group_expr.len(); - - let group_expr = hash_agg - .group_expr - .iter() - .zip(hash_agg.group_expr_name.iter()) - .map(|(expr, name)| { - parse_physical_expr(expr, registry, input.schema().as_ref()) - .map(|expr| (expr, name.to_string())) - }) - .collect::, _>>()?; - - let null_expr = hash_agg - .null_expr - .iter() - .zip(hash_agg.group_expr_name.iter()) - .map(|(expr, name)| { - parse_physical_expr(expr, registry, input.schema().as_ref()) - .map(|expr| (expr, name.to_string())) - }) - .collect::, _>>()?; - - let groups: Vec> = if !hash_agg.groups.is_empty() { - hash_agg - .groups - .chunks(num_expr) - .map(|g| g.to_vec()) - .collect::>>() - } else { - vec![] - }; - - let input_schema = hash_agg - .input_schema - .as_ref() - .ok_or_else(|| { - BallistaError::General( - "input_schema in AggregateNode is missing.".to_owned(), - ) - })? - .clone(); - let physical_schema: SchemaRef = - SchemaRef::new((&input_schema).try_into()?); - - let physical_aggr_expr: Vec> = hash_agg - .aggr_expr - .iter() - .zip(hash_agg.aggr_expr_name.iter()) - .map(|(expr, name)| { - let expr_type = expr.expr_type.as_ref().ok_or_else(|| { - proto_error("Unexpected empty aggregate physical expression") - })?; - - match expr_type { - ExprType::AggregateExpr(agg_node) => { - let aggr_function = - datafusion_proto::protobuf::AggregateFunction::from_i32( - agg_node.aggr_function, - ) - .ok_or_else( - || { - proto_error(format!( - "Received an unknown aggregate function: {}", - agg_node.aggr_function - )) - }, - )?; - - let input_phy_expr: Vec> = agg_node.expr.iter() - .map(|e| parse_physical_expr(e, registry, &physical_schema).unwrap()).collect(); - - Ok(create_aggregate_expr( - &aggr_function.into(), - agg_node.distinct, - input_phy_expr.as_slice(), - &physical_schema, - name.to_string(), - )?) - } - _ => Err(BallistaError::General( - "Invalid aggregate expression for AggregateExec" - .to_string(), - )), - } - }) - .collect::, _>>()?; - - Ok(Arc::new(AggregateExec::try_new( - agg_mode, - PhysicalGroupBy::new(group_expr, null_expr, groups), - physical_aggr_expr, - input, - Arc::new((&input_schema).try_into()?), - )?)) - } - PhysicalPlanType::HashJoin(hashjoin) => { - let left: Arc = into_physical_plan!( - hashjoin.left, - registry, - runtime, - extension_codec - )?; - let right: Arc = into_physical_plan!( - hashjoin.right, - registry, - runtime, - extension_codec - )?; - let on: Vec<(Column, Column)> = hashjoin - .on - .iter() - .map(|col| { - let left = into_required!(col.left)?; - let right = into_required!(col.right)?; - Ok((left, right)) - }) - .collect::>()?; - let join_type = - datafusion_proto::protobuf::JoinType::from_i32(hashjoin.join_type) - .ok_or_else(|| { - proto_error(format!( - "Received a HashJoinNode message with unknown JoinType {}", - hashjoin.join_type - )) - })?; - let filter = hashjoin - .filter - .as_ref() - .map(|f| { - let schema = f - .schema - .as_ref() - .ok_or_else(|| proto_error("Missing JoinFilter schema"))? - .try_into()?; - - let expression = parse_physical_expr( - f.expression.as_ref().ok_or_else(|| { - proto_error("Unexpected empty filter expression") - })?, - registry, &schema - )?; - let column_indices = f.column_indices - .iter() - .map(|i| { - let side = protobuf::JoinSide::from_i32(i.side) - .ok_or_else(|| proto_error(format!( - "Received a HashJoinNode message with JoinSide in Filter {}", - i.side)) - )?; - - Ok(ColumnIndex{ - index: i.index as usize, - side: side.into(), - }) - }) - .collect::, BallistaError>>()?; - - Ok(JoinFilter::new(expression, column_indices, schema)) - }) - .map_or(Ok(None), |v: Result| v.map(Some))?; - - let partition_mode = - protobuf::PartitionMode::from_i32(hashjoin.partition_mode) - .ok_or_else(|| { - proto_error(format!( - "Received a HashJoinNode message with unknown PartitionMode {}", - hashjoin.partition_mode - )) - })?; - let partition_mode = match partition_mode { - protobuf::PartitionMode::CollectLeft => PartitionMode::CollectLeft, - protobuf::PartitionMode::Partitioned => PartitionMode::Partitioned, - protobuf::PartitionMode::Auto => PartitionMode::Auto, - }; - Ok(Arc::new(HashJoinExec::try_new( - left, - right, - on, - filter, - &join_type.into(), - partition_mode, - &hashjoin.null_equals_null, - )?)) - } - PhysicalPlanType::Union(union) => { - let mut inputs: Vec> = vec![]; - for input in &union.inputs { - inputs.push(input.try_into_physical_plan( - registry, - runtime, - extension_codec, - )?); - } - Ok(Arc::new(UnionExec::new(inputs))) - } - PhysicalPlanType::CrossJoin(crossjoin) => { - let left: Arc = into_physical_plan!( - crossjoin.left, - registry, - runtime, - extension_codec - )?; - let right: Arc = into_physical_plan!( - crossjoin.right, - registry, - runtime, - extension_codec - )?; - Ok(Arc::new(CrossJoinExec::try_new(left, right)?)) - } - PhysicalPlanType::ShuffleWriter(shuffle_writer) => { - let input: Arc = into_physical_plan!( - shuffle_writer.input, - registry, - runtime, - extension_codec - )?; - - let output_partitioning = parse_protobuf_hash_partitioning( - shuffle_writer.output_partitioning.as_ref(), - registry, - input.schema().as_ref(), - )?; - - Ok(Arc::new(ShuffleWriterExec::try_new( - shuffle_writer.job_id.clone(), - shuffle_writer.stage_id as usize, - input, - "".to_string(), // this is intentional but hacky - the executor will fill this in - output_partitioning, - )?)) - } - PhysicalPlanType::ShuffleReader(shuffle_reader) => { - let schema = Arc::new(convert_required!(shuffle_reader.schema)?); - let partition_location: Vec> = shuffle_reader - .partition - .iter() - .map(|p| { - p.location - .iter() - .map(|l| l.clone().try_into()) - .collect::, _>>() - }) - .collect::, BallistaError>>()?; - let shuffle_reader = - ShuffleReaderExec::try_new(partition_location, schema)?; - Ok(Arc::new(shuffle_reader)) - } - PhysicalPlanType::Empty(empty) => { - let schema = Arc::new(convert_required!(empty.schema)?); - Ok(Arc::new(EmptyExec::new(empty.produce_one_row, schema))) - } - PhysicalPlanType::Sort(sort) => { - let input: Arc = - into_physical_plan!(sort.input, registry, runtime, extension_codec)?; - let exprs = sort - .expr - .iter() - .map(|expr| { - let expr = expr.expr_type.as_ref().ok_or_else(|| { - proto_error(format!( - "physical_plan::from_proto() Unexpected expr {:?}", - self - )) - })?; - if let protobuf::physical_expr_node::ExprType::Sort(sort_expr) = expr { - let expr = sort_expr - .expr - .as_ref() - .ok_or_else(|| { - proto_error(format!( - "physical_plan::from_proto() Unexpected sort expr {:?}", - self - )) - })? - .as_ref(); - Ok(PhysicalSortExpr { - expr: parse_physical_expr(expr,registry, input.schema().as_ref())?, - options: SortOptions { - descending: !sort_expr.asc, - nulls_first: sort_expr.nulls_first, - }, - }) - } else { - Err(BallistaError::General(format!( - "physical_plan::from_proto() {:?}", - self - ))) - } - }) - .collect::, _>>()?; - let fetch = if sort.fetch < 0 { - None - } else { - Some(sort.fetch as usize) - }; - Ok(Arc::new(SortExec::try_new(exprs, input, fetch)?)) - } - PhysicalPlanType::SortPreservingMerge(sort) => { - let input: Arc = - into_physical_plan!(sort.input, registry, runtime, extension_codec)?; - let exprs = sort - .expr - .iter() - .map(|expr| { - let expr = expr.expr_type.as_ref().ok_or_else(|| { - proto_error(format!( - "physical_plan::from_proto() Unexpected expr {:?}", - self - )) - })?; - if let protobuf::physical_expr_node::ExprType::Sort(sort_expr) = expr { - let expr = sort_expr - .expr - .as_ref() - .ok_or_else(|| { - proto_error(format!( - "physical_plan::from_proto() Unexpected sort expr {:?}", - self - )) - })? - .as_ref(); - Ok(PhysicalSortExpr { - expr: parse_physical_expr(expr,registry, input.schema().as_ref())?, - options: SortOptions { - descending: !sort_expr.asc, - nulls_first: sort_expr.nulls_first, - }, - }) - } else { - Err(BallistaError::General(format!( - "physical_plan::from_proto() {:?}", - self - ))) - } - }) - .collect::, _>>()?; - Ok(Arc::new(SortPreservingMergeExec::new(exprs, input))) - } - PhysicalPlanType::Unresolved(unresolved_shuffle) => { - let schema = Arc::new(convert_required!(unresolved_shuffle.schema)?); - Ok(Arc::new(UnresolvedShuffleExec { - stage_id: unresolved_shuffle.stage_id as usize, - schema, - input_partition_count: unresolved_shuffle.input_partition_count - as usize, - output_partition_count: unresolved_shuffle.output_partition_count - as usize, - })) - } - PhysicalPlanType::Extension(extension) => { - let inputs: Vec> = extension - .inputs - .iter() - .map(|i| i.try_into_physical_plan(registry, runtime, extension_codec)) - .collect::>()?; - - let extension_node = extension_codec.try_decode( - extension.node.as_slice(), - &inputs, - registry, - )?; - - Ok(extension_node) - } - } - } - - fn try_from_physical_plan( - plan: Arc, - extension_codec: &dyn PhysicalExtensionCodec, - ) -> Result - where - Self: Sized, - { - let plan_clone = plan.clone(); - let plan = plan.as_any(); - - if let Some(exec) = plan.downcast_ref::() { - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Explain( - protobuf::ExplainExecNode { - schema: Some(exec.schema().as_ref().try_into()?), - stringified_plans: exec - .stringified_plans() - .iter() - .map(|plan| plan.into()) - .collect(), - verbose: exec.verbose(), - }, - )), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.input().to_owned(), - extension_codec, - )?; - let expr = exec - .expr() - .iter() - .map(|expr| expr.0.clone().try_into()) - .collect::, BallistaError>>()?; - let expr_name = exec.expr().iter().map(|expr| expr.1.clone()).collect(); - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Projection(Box::new( - protobuf::ProjectionExecNode { - input: Some(Box::new(input)), - expr, - expr_name, - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.input().to_owned(), - extension_codec, - )?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Filter(Box::new( - protobuf::FilterExecNode { - input: Some(Box::new(input)), - expr: Some(exec.predicate().clone().try_into()?), - }, - ))), - }) - } else if let Some(limit) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - limit.input().to_owned(), - extension_codec, - )?; - - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::GlobalLimit(Box::new( - protobuf::GlobalLimitExecNode { - input: Some(Box::new(input)), - skip: limit.skip() as u32, - fetch: match limit.fetch() { - Some(n) => n as i64, - _ => -1, // no limit - }, - }, - ))), - }) - } else if let Some(limit) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - limit.input().to_owned(), - extension_codec, - )?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::LocalLimit(Box::new( - protobuf::LocalLimitExecNode { - input: Some(Box::new(input)), - fetch: limit.fetch() as u32, - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let left = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.left().to_owned(), - extension_codec, - )?; - let right = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.right().to_owned(), - extension_codec, - )?; - let on: Vec = exec - .on() - .iter() - .map(|tuple| protobuf::JoinOn { - left: Some(protobuf::PhysicalColumn { - name: tuple.0.name().to_string(), - index: tuple.0.index() as u32, - }), - right: Some(protobuf::PhysicalColumn { - name: tuple.1.name().to_string(), - index: tuple.1.index() as u32, - }), - }) - .collect(); - let join_type: datafusion_proto::protobuf::JoinType = - exec.join_type().to_owned().into(); - let filter = exec - .filter() - .as_ref() - .map(|f| { - let expression = f.expression().to_owned().try_into()?; - let column_indices = f - .column_indices() - .iter() - .map(|i| { - let side: protobuf::JoinSide = i.side.to_owned().into(); - protobuf::ColumnIndex { - index: i.index as u32, - side: side.into(), - } - }) - .collect(); - let schema = f.schema().try_into()?; - Ok(protobuf::JoinFilter { - expression: Some(expression), - column_indices, - schema: Some(schema), - }) - }) - .map_or( - Ok(None), - |v: Result| v.map(Some), - )?; - - let partition_mode = match exec.partition_mode() { - PartitionMode::CollectLeft => protobuf::PartitionMode::CollectLeft, - PartitionMode::Partitioned => protobuf::PartitionMode::Partitioned, - PartitionMode::Auto => protobuf::PartitionMode::Auto, - }; - - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::HashJoin(Box::new( - protobuf::HashJoinExecNode { - left: Some(Box::new(left)), - right: Some(Box::new(right)), - on, - join_type: join_type.into(), - partition_mode: partition_mode.into(), - null_equals_null: *exec.null_equals_null(), - filter, - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let left = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.left().to_owned(), - extension_codec, - )?; - let right = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.right().to_owned(), - extension_codec, - )?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::CrossJoin(Box::new( - protobuf::CrossJoinExecNode { - left: Some(Box::new(left)), - right: Some(Box::new(right)), - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let groups: Vec = exec - .group_expr() - .groups() - .iter() - .flatten() - .copied() - .collect(); - - let group_names = exec - .group_expr() - .expr() - .iter() - .map(|expr| expr.1.to_owned()) - .collect(); - - let agg = exec - .aggr_expr() - .iter() - .map(|expr| expr.to_owned().try_into()) - .collect::, BallistaError>>()?; - let agg_names = exec - .aggr_expr() - .iter() - .map(|expr| match expr.field() { - Ok(field) => Ok(field.name().clone()), - Err(e) => Err(BallistaError::DataFusionError(e)), - }) - .collect::>()?; - - let agg_mode = match exec.mode() { - AggregateMode::Partial => protobuf::AggregateMode::Partial, - AggregateMode::Final => protobuf::AggregateMode::Final, - AggregateMode::FinalPartitioned => { - protobuf::AggregateMode::FinalPartitioned - } - }; - let input_schema = exec.input_schema(); - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.input().to_owned(), - extension_codec, - )?; - - let null_expr = exec - .group_expr() - .null_expr() - .iter() - .map(|expr| expr.0.to_owned().try_into()) - .collect::, BallistaError>>()?; - - let group_expr = exec - .group_expr() - .expr() - .iter() - .map(|expr| expr.0.to_owned().try_into()) - .collect::, BallistaError>>()?; - - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Aggregate(Box::new( - protobuf::AggregateExecNode { - group_expr, - group_expr_name: group_names, - aggr_expr: agg, - aggr_expr_name: agg_names, - mode: agg_mode as i32, - input: Some(Box::new(input)), - input_schema: Some(input_schema.as_ref().try_into()?), - null_expr, - groups, - }, - ))), - }) - } else if let Some(empty) = plan.downcast_ref::() { - let schema = empty.schema().as_ref().try_into()?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Empty( - protobuf::EmptyExecNode { - produce_one_row: empty.produce_one_row(), - schema: Some(schema), - }, - )), - }) - } else if let Some(coalesce_batches) = plan.downcast_ref::() - { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - coalesce_batches.input().to_owned(), - extension_codec, - )?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::CoalesceBatches(Box::new( - protobuf::CoalesceBatchesExecNode { - input: Some(Box::new(input)), - target_batch_size: coalesce_batches.target_batch_size() as u32, - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::CsvScan( - protobuf::CsvScanExecNode { - base_conf: Some(exec.base_config().try_into()?), - has_header: exec.has_header(), - delimiter: byte_to_string(exec.delimiter())?, - }, - )), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let pruning_expr = exec - .pruning_predicate() - .map(|pred| pred.logical_expr().try_into()) - .transpose()?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::ParquetScan( - protobuf::ParquetScanExecNode { - base_conf: Some(exec.base_config().try_into()?), - pruning_predicate: pruning_expr, - }, - )), - }) - } else if let Some(exec) = plan.downcast_ref::() { - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::AvroScan( - protobuf::AvroScanExecNode { - base_conf: Some(exec.base_config().try_into()?), - }, - )), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let mut partition = vec![]; - for location in &exec.partition { - partition.push(protobuf::ShuffleReaderPartition { - location: location - .iter() - .map(|l| l.clone().try_into()) - .collect::, _>>()?, - }); - } - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::ShuffleReader( - protobuf::ShuffleReaderExecNode { - partition, - schema: Some(exec.schema().as_ref().try_into()?), - }, - )), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.input().to_owned(), - extension_codec, - )?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Merge(Box::new( - protobuf::CoalescePartitionsExecNode { - input: Some(Box::new(input)), - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.input().to_owned(), - extension_codec, - )?; - - let pb_partition_method = match exec.partitioning() { - Partitioning::Hash(exprs, partition_count) => { - PartitionMethod::Hash(protobuf::PhysicalHashRepartition { - hash_expr: exprs - .iter() - .map(|expr| expr.clone().try_into()) - .collect::, BallistaError>>()?, - partition_count: *partition_count as u64, - }) - } - Partitioning::RoundRobinBatch(partition_count) => { - PartitionMethod::RoundRobin(*partition_count as u64) - } - Partitioning::UnknownPartitioning(partition_count) => { - PartitionMethod::Unknown(*partition_count as u64) - } - }; - - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Repartition(Box::new( - protobuf::RepartitionExecNode { - input: Some(Box::new(input)), - partition_method: Some(pb_partition_method), - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.input().to_owned(), - extension_codec, - )?; - let expr = exec - .expr() - .iter() - .map(|expr| { - let sort_expr = Box::new(protobuf::PhysicalSortExprNode { - expr: Some(Box::new(expr.expr.to_owned().try_into()?)), - asc: !expr.options.descending, - nulls_first: expr.options.nulls_first, - }); - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Sort( - sort_expr, - )), - }) - }) - .collect::, BallistaError>>()?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Sort(Box::new( - protobuf::SortExecNode { - input: Some(Box::new(input)), - expr, - fetch: match exec.fetch() { - Some(n) => n as i64, - _ => -1, - }, - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.children()[0].to_owned(), - extension_codec, - )?; - // note that we use shuffle_output_partitioning() rather than output_partitioning() - // to get the true output partitioning - let output_partitioning = match exec.shuffle_output_partitioning() { - Some(Partitioning::Hash(exprs, partition_count)) => { - Some(protobuf::PhysicalHashRepartition { - hash_expr: exprs - .iter() - .map(|expr| expr.clone().try_into()) - .collect::, BallistaError>>()?, - partition_count: *partition_count as u64, - }) - } - None => None, - other => { - return Err(BallistaError::General(format!( - "physical_plan::to_proto() invalid partitioning for ShuffleWriterExec: {:?}", - other - ))) - } - }; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::ShuffleWriter(Box::new( - protobuf::ShuffleWriterExecNode { - job_id: exec.job_id().to_string(), - stage_id: exec.stage_id() as u32, - input: Some(Box::new(input)), - output_partitioning, - }, - ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Unresolved( - protobuf::UnresolvedShuffleExecNode { - stage_id: exec.stage_id as u32, - schema: Some(exec.schema().as_ref().try_into()?), - input_partition_count: exec.input_partition_count as u32, - output_partition_count: exec.output_partition_count as u32, - }, - )), - }) - } else if let Some(union) = plan.downcast_ref::() { - let mut inputs: Vec = vec![]; - for input in union.inputs() { - inputs.push(protobuf::PhysicalPlanNode::try_from_physical_plan( - input.to_owned(), - extension_codec, - )?); - } - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Union( - protobuf::UnionExecNode { inputs }, - )), - }) - } else if let Some(exec) = plan.downcast_ref::() { - let input = protobuf::PhysicalPlanNode::try_from_physical_plan( - exec.input().to_owned(), - extension_codec, - )?; - let expr = exec - .expr() - .iter() - .map(|expr| { - let sort_expr = Box::new(protobuf::PhysicalSortExprNode { - expr: Some(Box::new(expr.expr.to_owned().try_into()?)), - asc: !expr.options.descending, - nulls_first: expr.options.nulls_first, - }); - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Sort( - sort_expr, - )), - }) - }) - .collect::, BallistaError>>()?; - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::SortPreservingMerge( - Box::new(protobuf::SortPreservingMergeExecNode { - input: Some(Box::new(input)), - expr, - }), - )), - }) - } else { - let mut buf: Vec = vec![]; - match extension_codec.try_encode(plan_clone.clone(), &mut buf) { - Ok(_) => { - let inputs: Vec = plan_clone - .children() - .into_iter() - .map(|i| { - PhysicalPlanNode::try_from_physical_plan(i, extension_codec) - }) - .collect::>()?; - - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Extension( - PhysicalExtensionNode { node: buf, inputs }, - )), - }) - } - Err(e) => Err(BallistaError::Internal(format!( - "Unsupported plan and extension codec failed with [{}]. Plan: {:?}", - e, plan_clone - ))), - } - } - } -} - -fn decode_scan_config( - proto: &protobuf::FileScanExecConf, -) -> Result { - let schema = Arc::new(convert_required!(proto.schema)?); - let projection = proto - .projection - .iter() - .map(|i| *i as usize) - .collect::>(); - let projection = if projection.is_empty() { - None - } else { - Some(projection) - }; - let statistics = convert_required!(proto.statistics)?; - - let file_groups: Vec> = proto - .file_groups - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?; - - let object_store_url = match proto.object_store_url.is_empty() { - false => ObjectStoreUrl::parse(&proto.object_store_url)?, - true => ObjectStoreUrl::local_filesystem(), - }; - - Ok(FileScanConfig { - config_options: Arc::new(RwLock::new(ConfigOptions::new())), // TODO add serde - object_store_url, - file_schema: schema, - file_groups, - statistics, - projection, - limit: proto.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_cols: vec![], - output_ordering: None, - }) -} - -#[macro_export] -macro_rules! into_physical_plan { - ($PB:expr, $REG:expr, $RUNTIME:expr, $CODEC:expr) => {{ - if let Some(field) = $PB.as_ref() { - field - .as_ref() - .try_into_physical_plan($REG, $RUNTIME, $CODEC) - } else { - Err(proto_error("Missing required field in protobuf")) - } - }}; -} - -#[cfg(test)] -mod roundtrip_tests { - use std::ops::Deref; - use std::sync::Arc; - - 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; - use datafusion::logical_expr::{BuiltinScalarFunction, Volatility}; - use datafusion::physical_expr::expressions::DateTimeIntervalExpr; - use datafusion::physical_expr::ScalarFunctionExpr; - use datafusion::physical_plan::aggregates::PhysicalGroupBy; - use datafusion::physical_plan::functions; - use datafusion::physical_plan::functions::make_scalar_function; - use datafusion::physical_plan::projection::ProjectionExec; - use datafusion::{ - arrow::{ - compute::kernels::sort::SortOptions, - datatypes::{DataType, Field, Schema}, - }, - datasource::listing::PartitionedFile, - logical_expr::{JoinType, Operator}, - physical_plan::{ - aggregates::{AggregateExec, AggregateMode}, - empty::EmptyExec, - expressions::{binary, col, lit, InListExpr, NotExpr}, - expressions::{Avg, Column, DistinctCount, PhysicalSortExpr}, - file_format::{FileScanConfig, ParquetExec}, - filter::FilterExec, - joins::{HashJoinExec, PartitionMode}, - limit::{GlobalLimitExec, LocalLimitExec}, - sorts::sort::SortExec, - AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, Statistics, - }, - prelude::SessionContext, - scalar::ScalarValue, - }; - - use crate::execution_plans::ShuffleWriterExec; - 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; - - fn roundtrip_test(exec_plan: Arc) -> Result<()> { - let ctx = SessionContext::new(); - let codec: BallistaCodec = - BallistaCodec::default(); - let proto: protobuf::PhysicalPlanNode = - protobuf::PhysicalPlanNode::try_from_physical_plan( - exec_plan.clone(), - codec.physical_extension_codec(), - ) - .expect("to proto"); - let runtime = ctx.runtime_env(); - let result_exec_plan: Arc = proto - .try_into_physical_plan( - &ctx, - runtime.deref(), - codec.physical_extension_codec(), - ) - .expect("from proto"); - assert_eq!( - format!("{:?}", exec_plan), - format!("{:?}", result_exec_plan) - ); - Ok(()) - } - - fn roundtrip_test_with_context( - exec_plan: Arc, - ctx: SessionContext, - ) -> Result<()> { - let codec: BallistaCodec = - BallistaCodec::default(); - let proto: protobuf::PhysicalPlanNode = - protobuf::PhysicalPlanNode::try_from_physical_plan( - exec_plan.clone(), - codec.physical_extension_codec(), - ) - .expect("to proto"); - let runtime = ctx.runtime_env(); - let result_exec_plan: Arc = proto - .try_into_physical_plan( - &ctx, - runtime.deref(), - codec.physical_extension_codec(), - ) - .expect("from proto"); - assert_eq!( - format!("{:?}", exec_plan), - format!("{:?}", result_exec_plan) - ); - Ok(()) - } - - #[test] - fn roundtrip_empty() -> Result<()> { - roundtrip_test(Arc::new(EmptyExec::new(false, Arc::new(Schema::empty())))) - } - - #[test] - fn roundtrip_date_time_interval() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("some_date", DataType::Date32, false), - Field::new( - "some_interval", - DataType::Interval(IntervalUnit::DayTime), - false, - ), - ]); - let input = Arc::new(EmptyExec::new(false, Arc::new(schema.clone()))); - let date_expr = col("some_date", &schema)?; - let literal_expr = col("some_interval", &schema)?; - let date_time_interval_expr = Arc::new(DateTimeIntervalExpr::try_new( - date_expr, - Operator::Plus, - literal_expr, - &schema, - )?); - let plan = Arc::new(ProjectionExec::try_new( - vec![(date_time_interval_expr, "result".to_string())], - input, - )?); - roundtrip_test(plan) - } - - #[test] - fn roundtrip_local_limit() -> Result<()> { - roundtrip_test(Arc::new(LocalLimitExec::new( - Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), - 25, - ))) - } - - #[test] - fn roundtrip_global_limit() -> Result<()> { - roundtrip_test(Arc::new(GlobalLimitExec::new( - Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), - 0, - Some(25), - ))) - } - - #[test] - fn roundtrip_global_skip_no_limit() -> Result<()> { - roundtrip_test(Arc::new(GlobalLimitExec::new( - Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), - 10, - None, // no limit - ))) - } - - #[test] - fn roundtrip_hash_join() -> Result<()> { - let field_a = Field::new("col", DataType::Int64, false); - let schema_left = Schema::new(vec![field_a.clone()]); - let schema_right = Schema::new(vec![field_a]); - let on = vec![( - Column::new("col", schema_left.index_of("col")?), - Column::new("col", schema_right.index_of("col")?), - )]; - - let schema_left = Arc::new(schema_left); - let schema_right = Arc::new(schema_right); - for join_type in &[ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftAnti, - JoinType::RightAnti, - JoinType::LeftSemi, - JoinType::RightSemi, - ] { - for partition_mode in - &[PartitionMode::Partitioned, PartitionMode::CollectLeft] - { - roundtrip_test(Arc::new(HashJoinExec::try_new( - Arc::new(EmptyExec::new(false, schema_left.clone())), - Arc::new(EmptyExec::new(false, schema_right.clone())), - on.clone(), - None, - join_type, - *partition_mode, - &false, - )?))?; - } - } - Ok(()) - } - - #[test] - fn rountrip_aggregate() -> Result<()> { - let field_a = Field::new("a", DataType::Int64, false); - let field_b = Field::new("b", DataType::Int64, false); - let schema = Arc::new(Schema::new(vec![field_a, field_b])); - - let groups: Vec<(Arc, String)> = - vec![(col("a", &schema)?, "unused".to_string())]; - - let aggregates: Vec> = vec![Arc::new(Avg::new( - col("b", &schema)?, - "AVG(b)".to_string(), - DataType::Float64, - ))]; - - roundtrip_test(Arc::new(AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), - Arc::new(EmptyExec::new(false, schema.clone())), - schema, - )?)) - } - - #[test] - fn roundtrip_filter_with_not_and_in_list() -> Result<()> { - let field_a = Field::new("a", DataType::Boolean, false); - let field_b = Field::new("b", DataType::Int64, false); - let field_c = Field::new("c", DataType::Int64, false); - let schema = Arc::new(Schema::new(vec![field_a, field_b, field_c])); - let not = Arc::new(NotExpr::new(col("a", &schema)?)); - let in_list = Arc::new(InListExpr::new( - col("b", &schema)?, - vec![ - lit(ScalarValue::Int64(Some(1))), - lit(ScalarValue::Int64(Some(2))), - ], - false, - schema.as_ref(), - )); - let and = binary(not, Operator::And, in_list, &schema)?; - roundtrip_test(Arc::new(FilterExec::try_new( - and, - Arc::new(EmptyExec::new(false, schema.clone())), - )?)) - } - - #[test] - fn roundtrip_sort() -> Result<()> { - let field_a = Field::new("a", DataType::Boolean, false); - let field_b = Field::new("b", DataType::Int64, false); - let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let sort_exprs = vec![ - PhysicalSortExpr { - expr: col("a", &schema)?, - options: SortOptions { - descending: true, - nulls_first: false, - }, - }, - PhysicalSortExpr { - expr: col("b", &schema)?, - options: SortOptions { - descending: false, - nulls_first: true, - }, - }, - ]; - roundtrip_test(Arc::new(SortExec::try_new( - sort_exprs, - Arc::new(EmptyExec::new(false, schema)), - None, - )?)) - } - - #[test] - fn roundtrip_shuffle_writer() -> Result<()> { - let field_a = Field::new("a", DataType::Int64, false); - let field_b = Field::new("b", DataType::Int64, false); - let schema = Arc::new(Schema::new(vec![field_a, field_b])); - - roundtrip_test(Arc::new(ShuffleWriterExec::try_new( - "job123".to_string(), - 123, - Arc::new(EmptyExec::new(false, schema)), - "".to_string(), - Some(Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 4)), - )?)) - } - - #[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", - DataType::Utf8, - false, - )])), - file_groups: vec![vec![PartitionedFile::new( - "/path/to/file.parquet".to_string(), - 1024, - )]], - statistics: Statistics { - num_rows: Some(100), - total_byte_size: Some(1024), - column_statistics: None, - is_exact: false, - }, - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: None, - }; - - let predicate = datafusion::prelude::col("col").eq(datafusion::prelude::lit("1")); - roundtrip_test(Arc::new(ParquetExec::new( - scan_config, - Some(predicate), - None, - ))) - } - - #[test] - fn roundtrip_builtin_scalar_function() -> Result<()> { - let field_a = Field::new("a", DataType::Int64, false); - let field_b = Field::new("b", DataType::Int64, false); - let schema = Arc::new(Schema::new(vec![field_a, field_b])); - - let input = Arc::new(EmptyExec::new(false, schema.clone())); - - let execution_props = ExecutionProps::new(); - - let fun_expr = functions::create_physical_fun( - &BuiltinScalarFunction::Abs, - &execution_props, - )?; - - let expr = ScalarFunctionExpr::new( - "abs", - fun_expr, - vec![col("a", &schema)?], - &DataType::Int64, - ); - - let project = - ProjectionExec::try_new(vec![(Arc::new(expr), "a".to_string())], input)?; - - roundtrip_test(Arc::new(project)) - } - - #[test] - fn roundtrip_scalar_udf() -> Result<()> { - let field_a = Field::new("a", DataType::Int64, false); - let field_b = Field::new("b", DataType::Int64, false); - let schema = Arc::new(Schema::new(vec![field_a, field_b])); - - let input = Arc::new(EmptyExec::new(false, schema.clone())); - - let fn_impl = |args: &[ArrayRef]| Ok(Arc::new(args[0].clone()) as ArrayRef); - - let scalar_fn = make_scalar_function(fn_impl); - - let udf = create_udf( - "dummy", - vec![DataType::Int64], - Arc::new(DataType::Int64), - Volatility::Immutable, - scalar_fn.clone(), - ); - - let expr = ScalarFunctionExpr::new( - "dummy", - scalar_fn, - vec![col("a", &schema)?], - &DataType::Int64, - ); - - let project = - ProjectionExec::try_new(vec![(Arc::new(expr), "a".to_string())], input)?; - - let mut ctx = SessionContext::new(); - - ctx.register_udf(udf); - - roundtrip_test_with_context(Arc::new(project), ctx) - } - - #[test] - fn roundtrip_distinct_count() -> Result<()> { - let field_a = Field::new("a", DataType::Int64, false); - let field_b = Field::new("b", DataType::Int64, false); - let schema = Arc::new(Schema::new(vec![field_a, field_b])); - - let aggregates: Vec> = vec![Arc::new(DistinctCount::new( - vec![DataType::Int64], - vec![col("b", &schema)?], - "COUNT(DISTINCT b)".to_string(), - DataType::Int64, - ))]; - - let groups: Vec<(Arc, String)> = - vec![(col("a", &schema)?, "unused".to_string())]; - - roundtrip_test(Arc::new(AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::new_single(groups), - aggregates.clone(), - Arc::new(EmptyExec::new(false, schema.clone())), - schema, - )?)) - } -} diff --git a/ballista/core/src/serde/physical_plan/to_proto.rs b/ballista/core/src/serde/physical_plan/to_proto.rs deleted file mode 100644 index c303ba453..000000000 --- a/ballista/core/src/serde/physical_plan/to_proto.rs +++ /dev/null @@ -1,468 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License.language governing permissions and -// limitations under the License. - -//! Serde code to convert Arrow schemas and DataFusion logical plans to Ballista protocol -//! buffer format, allowing DataFusion physical plans to be serialized and transmitted between -//! processes. - -use std::{ - convert::{TryFrom, TryInto}, - str::FromStr, - sync::Arc, -}; - -use datafusion::physical_plan::expressions::{CastExpr, TryCastExpr}; -use datafusion::physical_plan::ColumnStatistics; -use datafusion::physical_plan::{ - expressions::{ - CaseExpr, InListExpr, IsNotNullExpr, IsNullExpr, NegativeExpr, NotExpr, - }, - Statistics, -}; - -use datafusion::datasource::listing::{FileRange, PartitionedFile}; -use datafusion::physical_plan::file_format::FileScanConfig; - -use datafusion::physical_plan::expressions::{Count, DistinctCount, Literal}; - -use datafusion::physical_plan::expressions::{Avg, BinaryExpr, Column, Max, Min, Sum}; -use datafusion::physical_plan::{AggregateExpr, PhysicalExpr}; - -use crate::serde::{protobuf, BallistaError}; - -use datafusion::logical_expr::BuiltinScalarFunction; -use datafusion::physical_expr::expressions::DateTimeIntervalExpr; -use datafusion::physical_expr::ScalarFunctionExpr; - -impl TryInto for Arc { - type Error = BallistaError; - - fn try_into(self) -> Result { - use datafusion::physical_plan::expressions; - use datafusion_proto::protobuf::AggregateFunction; - - let mut distinct = false; - let aggr_function = if self.as_any().downcast_ref::().is_some() { - Ok(AggregateFunction::Avg.into()) - } else if self.as_any().downcast_ref::().is_some() { - Ok(AggregateFunction::Sum.into()) - } else if self.as_any().downcast_ref::().is_some() { - Ok(AggregateFunction::Count.into()) - } else if self.as_any().downcast_ref::().is_some() { - distinct = true; - Ok(AggregateFunction::Count.into()) - } else if self.as_any().downcast_ref::().is_some() { - Ok(AggregateFunction::Min.into()) - } else if self.as_any().downcast_ref::().is_some() { - Ok(AggregateFunction::Max.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::ApproxDistinct.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::ArrayAgg.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::Variance.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::VariancePop.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::Covariance.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::CovariancePop.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::Stddev.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::StddevPop.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::Correlation.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::ApproxPercentileCont.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::ApproxPercentileContWithWeight.into()) - } else if self - .as_any() - .downcast_ref::() - .is_some() - { - Ok(AggregateFunction::ApproxMedian.into()) - } else { - Err(BallistaError::NotImplemented(format!( - "Aggregate function not supported: {:?}", - self - ))) - }?; - let expressions: Vec = self - .expressions() - .iter() - .map(|e| e.clone().try_into()) - .collect::, BallistaError>>()?; - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( - protobuf::PhysicalAggregateExprNode { - aggr_function, - expr: expressions, - distinct, - }, - )), - }) - } -} - -impl TryFrom> for protobuf::PhysicalExprNode { - type Error = BallistaError; - - fn try_from(value: Arc) -> Result { - let expr = value.as_any(); - - if let Some(expr) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Column( - protobuf::PhysicalColumn { - name: expr.name().to_string(), - index: expr.index() as u32, - }, - )), - }) - } else if let Some(expr) = expr.downcast_ref::() { - let binary_expr = Box::new(protobuf::PhysicalBinaryExprNode { - l: Some(Box::new(expr.left().to_owned().try_into()?)), - r: Some(Box::new(expr.right().to_owned().try_into()?)), - op: format!("{:?}", expr.op()), - }); - - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::BinaryExpr( - binary_expr, - )), - }) - } else if let Some(expr) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some( - protobuf::physical_expr_node::ExprType::Case( - Box::new( - protobuf::PhysicalCaseNode { - expr: expr - .expr() - .as_ref() - .map(|exp| exp.clone().try_into().map(Box::new)) - .transpose()?, - when_then_expr: expr - .when_then_expr() - .iter() - .map(|(when_expr, then_expr)| { - try_parse_when_then_expr(when_expr, then_expr) - }) - .collect::, - Self::Error, - >>()?, - else_expr: expr - .else_expr() - .map(|a| a.clone().try_into().map(Box::new)) - .transpose()?, - }, - ), - ), - ), - }) - } else if let Some(expr) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::NotExpr( - Box::new(protobuf::PhysicalNot { - expr: Some(Box::new(expr.arg().to_owned().try_into()?)), - }), - )), - }) - } else if let Some(expr) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::IsNullExpr( - Box::new(protobuf::PhysicalIsNull { - expr: Some(Box::new(expr.arg().to_owned().try_into()?)), - }), - )), - }) - } else if let Some(expr) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::IsNotNullExpr( - Box::new(protobuf::PhysicalIsNotNull { - expr: Some(Box::new(expr.arg().to_owned().try_into()?)), - }), - )), - }) - } else if let Some(expr) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some( - protobuf::physical_expr_node::ExprType::InList( - Box::new( - protobuf::PhysicalInListNode { - expr: Some(Box::new(expr.expr().to_owned().try_into()?)), - list: expr - .list() - .iter() - .map(|a| a.clone().try_into()) - .collect::, - Self::Error, - >>()?, - negated: expr.negated(), - }, - ), - ), - ), - }) - } else if let Some(expr) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Negative( - Box::new(protobuf::PhysicalNegativeNode { - expr: Some(Box::new(expr.arg().to_owned().try_into()?)), - }), - )), - }) - } else if let Some(lit) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Literal( - lit.value().try_into()?, - )), - }) - } else if let Some(cast) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Cast(Box::new( - protobuf::PhysicalCastNode { - expr: Some(Box::new(cast.expr().clone().try_into()?)), - arrow_type: Some(cast.cast_type().try_into()?), - }, - ))), - }) - } else if let Some(cast) = expr.downcast_ref::() { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::TryCast( - Box::new(protobuf::PhysicalTryCastNode { - expr: Some(Box::new(cast.expr().clone().try_into()?)), - arrow_type: Some(cast.cast_type().try_into()?), - }), - )), - }) - } else if let Some(expr) = expr.downcast_ref::() { - let args: Vec = expr - .args() - .iter() - .map(|e| e.to_owned().try_into()) - .collect::, _>>()?; - if let Ok(fun) = BuiltinScalarFunction::from_str(expr.name()) { - let fun: datafusion_proto::protobuf::ScalarFunction = - (&fun).try_into()?; - - Ok(protobuf::PhysicalExprNode { - expr_type: Some( - protobuf::physical_expr_node::ExprType::ScalarFunction( - protobuf::PhysicalScalarFunctionNode { - name: expr.name().to_string(), - fun: fun.into(), - args, - return_type: Some(expr.return_type().try_into()?), - }, - ), - ), - }) - } else { - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::ScalarUdf( - protobuf::PhysicalScalarUdfNode { - name: expr.name().to_string(), - args, - return_type: Some(expr.return_type().try_into()?), - }, - )), - }) - } - } else if let Some(expr) = expr.downcast_ref::() { - let dti_expr = Box::new(protobuf::PhysicalDateTimeIntervalExprNode { - l: Some(Box::new(expr.lhs().to_owned().try_into()?)), - r: Some(Box::new(expr.rhs().to_owned().try_into()?)), - op: format!("{:?}", expr.op()), - }); - - Ok(protobuf::PhysicalExprNode { - expr_type: Some( - protobuf::physical_expr_node::ExprType::DateTimeIntervalExpr( - dti_expr, - ), - ), - }) - } else { - Err(BallistaError::General(format!( - "physical_plan::to_proto() unsupported expression {:?}", - value - ))) - } - } -} - -fn try_parse_when_then_expr( - when_expr: &Arc, - then_expr: &Arc, -) -> Result { - Ok(protobuf::PhysicalWhenThen { - when_expr: Some(when_expr.clone().try_into()?), - then_expr: Some(then_expr.clone().try_into()?), - }) -} - -impl TryFrom<&PartitionedFile> for protobuf::PartitionedFile { - type Error = BallistaError; - - fn try_from(pf: &PartitionedFile) -> Result { - Ok(protobuf::PartitionedFile { - path: pf.object_meta.location.as_ref().to_owned(), - size: pf.object_meta.size as u64, - last_modified_ns: pf.object_meta.last_modified.timestamp_nanos() as u64, - partition_values: pf - .partition_values - .iter() - .map(|v| v.try_into()) - .collect::, _>>()?, - range: pf.range.as_ref().map(|r| r.try_into()).transpose()?, - }) - } -} - -impl TryFrom<&FileRange> for protobuf::FileRange { - type Error = BallistaError; - - fn try_from(value: &FileRange) -> Result { - Ok(protobuf::FileRange { - start: value.start, - end: value.end, - }) - } -} - -impl TryFrom<&[PartitionedFile]> for protobuf::FileGroup { - type Error = BallistaError; - - fn try_from(gr: &[PartitionedFile]) -> Result { - Ok(protobuf::FileGroup { - files: gr - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?, - }) - } -} - -impl From<&ColumnStatistics> for protobuf::ColumnStats { - fn from(cs: &ColumnStatistics) -> protobuf::ColumnStats { - protobuf::ColumnStats { - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - null_count: cs.null_count.map(|n| n as u32).unwrap_or(0), - distinct_count: cs.distinct_count.map(|n| n as u32).unwrap_or(0), - } - } -} - -impl From<&Statistics> for protobuf::Statistics { - fn from(s: &Statistics) -> protobuf::Statistics { - let none_value = -1_i64; - let column_stats = match &s.column_statistics { - None => vec![], - Some(column_stats) => column_stats.iter().map(|s| s.into()).collect(), - }; - protobuf::Statistics { - num_rows: s.num_rows.map(|n| n as i64).unwrap_or(none_value), - total_byte_size: s.total_byte_size.map(|n| n as i64).unwrap_or(none_value), - column_stats, - is_exact: s.is_exact, - } - } -} - -impl TryFrom<&FileScanConfig> for protobuf::FileScanExecConf { - type Error = BallistaError; - fn try_from( - conf: &FileScanConfig, - ) -> Result { - let file_groups = conf - .file_groups - .iter() - .map(|p| p.as_slice().try_into()) - .collect::, _>>()?; - - Ok(protobuf::FileScanExecConf { - file_groups, - statistics: Some((&conf.statistics).into()), - limit: conf.limit.map(|l| protobuf::ScanLimit { limit: l as u32 }), - projection: conf - .projection - .as_ref() - .unwrap_or(&vec![]) - .iter() - .map(|n| *n as u32) - .collect(), - schema: Some(conf.file_schema.as_ref().try_into()?), - table_partition_cols: conf - .table_partition_cols - .iter() - .map(|col| col.0.to_owned()) - .collect(), - object_store_url: conf.object_store_url.to_string(), - }) - } -} diff --git a/ballista/core/src/serde/scheduler/from_proto.rs b/ballista/core/src/serde/scheduler/from_proto.rs index a9e0e66bf..6c008abb3 100644 --- a/ballista/core/src/serde/scheduler/from_proto.rs +++ b/ballista/core/src/serde/scheduler/from_proto.rs @@ -26,27 +26,29 @@ use std::sync::Arc; use std::time::Duration; use crate::error::BallistaError; -use crate::serde::protobuf; -use crate::serde::protobuf::action::ActionType; -use crate::serde::protobuf::{operator_metric, NamedCount, NamedGauge, NamedTime}; use crate::serde::scheduler::{ Action, ExecutorData, ExecutorMetadata, ExecutorSpecification, PartitionId, PartitionLocation, PartitionStats, TaskDefinition, }; +use crate::serde::protobuf; +use protobuf::{operator_metric, NamedCount, NamedGauge, NamedTime}; + impl TryInto for protobuf::Action { type Error = BallistaError; fn try_into(self) -> Result { match self.action_type { - Some(ActionType::FetchPartition(fetch)) => Ok(Action::FetchPartition { - job_id: fetch.job_id, - stage_id: fetch.stage_id as usize, - partition_id: fetch.partition_id as usize, - path: fetch.path, - host: fetch.host, - port: fetch.port as u16, - }), + Some(protobuf::action::ActionType::FetchPartition(fetch)) => { + Ok(Action::FetchPartition { + job_id: fetch.job_id, + stage_id: fetch.stage_id as usize, + partition_id: fetch.partition_id as usize, + path: fetch.path, + host: fetch.host, + port: fetch.port as u16, + }) + } _ => Err(BallistaError::General( "scheduler::from_proto(Action) invalid or missing action".to_owned(), )), diff --git a/ballista/core/src/serde/scheduler/mod.rs b/ballista/core/src/serde/scheduler/mod.rs index 370dd334f..d8ff0906a 100644 --- a/ballista/core/src/serde/scheduler/mod.rs +++ b/ballista/core/src/serde/scheduler/mod.rs @@ -23,10 +23,10 @@ use datafusion::arrow::array::{ use datafusion::arrow::datatypes::{DataType, Field}; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::Partitioning; +use datafusion_proto::protobuf as datafusion_protobuf; use serde::Serialize; use crate::error::BallistaError; -use crate::serde::protobuf::PhysicalHashRepartition; pub mod from_proto; pub mod to_proto; @@ -281,7 +281,7 @@ pub struct TaskDefinition { pub stage_attempt_num: usize, pub partition_id: usize, pub plan: Vec, - pub output_partitioning: Option, + pub output_partitioning: Option, pub session_id: String, pub launch_time: u64, pub props: HashMap, diff --git a/ballista/core/src/serde/scheduler/to_proto.rs b/ballista/core/src/serde/scheduler/to_proto.rs index d453548ea..3d7ee9464 100644 --- a/ballista/core/src/serde/scheduler/to_proto.rs +++ b/ballista/core/src/serde/scheduler/to_proto.rs @@ -15,21 +15,23 @@ // specific language governing permissions and limitations // under the License. +use datafusion::error::DataFusionError; use datafusion::physical_plan::metrics::{MetricValue, MetricsSet}; use std::convert::TryInto; use crate::error::BallistaError; + use crate::serde::protobuf; -use crate::serde::protobuf::action::ActionType; +use datafusion_proto::protobuf as datafusion_protobuf; -use crate::serde::protobuf::{ - operator_metric, KeyValuePair, NamedCount, NamedGauge, NamedTime, -}; use crate::serde::scheduler::{ Action, ExecutorData, ExecutorMetadata, ExecutorSpecification, PartitionId, PartitionLocation, PartitionStats, TaskDefinition, }; use datafusion::physical_plan::Partitioning; +use protobuf::{ + action::ActionType, operator_metric, KeyValuePair, NamedCount, NamedGauge, NamedTime, +}; impl TryInto for Action { type Error = BallistaError; @@ -98,14 +100,14 @@ impl Into for PartitionStats { pub fn hash_partitioning_to_proto( output_partitioning: Option<&Partitioning>, -) -> Result, BallistaError> { +) -> Result, BallistaError> { match output_partitioning { Some(Partitioning::Hash(exprs, partition_count)) => { - Ok(Some(protobuf::PhysicalHashRepartition { + Ok(Some(datafusion_protobuf::PhysicalHashRepartition { hash_expr: exprs .iter() .map(|expr| expr.clone().try_into()) - .collect::, BallistaError>>()?, + .collect::, DataFusionError>>()?, partition_count: *partition_count as u64, })) } diff --git a/ballista/core/src/utils.rs b/ballista/core/src/utils.rs index c83e94aa8..e11b4e552 100644 --- a/ballista/core/src/utils.rs +++ b/ballista/core/src/utils.rs @@ -327,7 +327,7 @@ pub fn create_df_ctx_with_ballista_query_planner( ), ) .with_query_planner(planner); - session_state.session_id = session_id; + session_state = session_state.with_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..24fe4a3a7 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.1.0" +datafusion-proto = "16.1.0" futures = "0.3" hyper = "0.14.4" log = "0.4" diff --git a/ballista/executor/src/execution_loop.rs b/ballista/executor/src/execution_loop.rs index 4efbbe18c..17e4271a8 100644 --- a/ballista/executor/src/execution_loop.rs +++ b/ballista/executor/src/execution_loop.rs @@ -27,12 +27,13 @@ use crate::cpu_bound_executor::DedicatedExecutor; use crate::executor::Executor; use crate::{as_task_status, TaskExecutionTimes}; use ballista_core::error::BallistaError; -use ballista_core::serde::physical_plan::from_proto::parse_protobuf_hash_partitioning; use ballista_core::serde::scheduler::{ExecutorSpecification, PartitionId}; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; use ballista_core::utils::collect_plan_metrics; use datafusion::execution::context::TaskContext; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::from_proto::parse_protobuf_hash_partitioning; +use datafusion_proto::physical_plan::AsExecutionPlan; use futures::FutureExt; use log::{debug, error, info, warn}; use std::any::Any; diff --git a/ballista/executor/src/executor_process.rs b/ballista/executor/src/executor_process.rs index 969b0573f..a5b152a75 100644 --- a/ballista/executor/src/executor_process.rs +++ b/ballista/executor/src/executor_process.rs @@ -37,13 +37,13 @@ use tracing_subscriber::EnvFilter; use uuid::Uuid; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; -use datafusion_proto::protobuf::LogicalPlanNode; +use datafusion_proto::protobuf::{LogicalPlanNode, PhysicalPlanNode}; use ballista_core::config::{LogRotationPolicy, TaskSchedulingPolicy}; use ballista_core::error::BallistaError; use ballista_core::serde::protobuf::{ executor_registration, scheduler_grpc_client::SchedulerGrpcClient, - ExecutorRegistration, ExecutorStoppedParams, PhysicalPlanNode, + ExecutorRegistration, ExecutorStoppedParams, }; use ballista_core::serde::scheduler::ExecutorSpecification; use ballista_core::serde::BallistaCodec; diff --git a/ballista/executor/src/executor_server.rs b/ballista/executor/src/executor_server.rs index de4696d29..aee3d0cfa 100644 --- a/ballista/executor/src/executor_server.rs +++ b/ballista/executor/src/executor_server.rs @@ -29,28 +29,28 @@ use tonic::transport::Channel; use tonic::{Request, Response, Status}; use ballista_core::error::BallistaError; -use ballista_core::serde::physical_plan::from_proto::parse_protobuf_hash_partitioning; -use ballista_core::serde::protobuf::executor_grpc_server::{ - ExecutorGrpc, ExecutorGrpcServer, -}; -use ballista_core::serde::protobuf::scheduler_grpc_client::SchedulerGrpcClient; use ballista_core::serde::protobuf::{ - executor_metric, executor_status, CancelTasksParams, CancelTasksResult, - ExecutorMetric, ExecutorStatus, HeartBeatParams, LaunchMultiTaskParams, - LaunchMultiTaskResult, LaunchTaskParams, LaunchTaskResult, RegisterExecutorParams, - RemoveJobDataParams, RemoveJobDataResult, StopExecutorParams, StopExecutorResult, - TaskStatus, UpdateTaskStatusParams, + executor_grpc_server::{ExecutorGrpc, ExecutorGrpcServer}, + executor_metric, executor_status, + scheduler_grpc_client::SchedulerGrpcClient, + CancelTasksParams, CancelTasksResult, ExecutorMetric, ExecutorStatus, + HeartBeatParams, LaunchMultiTaskParams, LaunchMultiTaskResult, LaunchTaskParams, + LaunchTaskResult, RegisterExecutorParams, RemoveJobDataParams, RemoveJobDataResult, + StopExecutorParams, StopExecutorResult, TaskStatus, UpdateTaskStatusParams, }; use ballista_core::serde::scheduler::PartitionId; use ballista_core::serde::scheduler::TaskDefinition; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; use ballista_core::utils::{ collect_plan_metrics, create_grpc_client_connection, create_grpc_server, }; use dashmap::DashMap; use datafusion::execution::context::TaskContext; use datafusion::physical_plan::ExecutionPlan; -use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::{ + logical_plan::AsLogicalPlan, + physical_plan::{from_proto::parse_protobuf_hash_partitioning, AsExecutionPlan}, +}; use tokio::sync::mpsc::error::TryRecvError; use tokio::task::JoinHandle; diff --git a/ballista/executor/src/standalone.rs b/ballista/executor/src/standalone.rs index 4cd290b57..d38a725f4 100644 --- a/ballista/executor/src/standalone.rs +++ b/ballista/executor/src/standalone.rs @@ -19,7 +19,7 @@ use crate::metrics::LoggingMetricsCollector; use crate::{execution_loop, executor::Executor, flight_service::BallistaFlightService}; use arrow_flight::flight_service_server::FlightServiceServer; use ballista_core::serde::scheduler::ExecutorSpecification; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; use ballista_core::utils::{create_grpc_server, with_object_store_provider}; use ballista_core::{ error::Result, @@ -29,6 +29,7 @@ use ballista_core::{ }; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use log::info; use std::sync::Arc; use tempfile::TempDir; diff --git a/ballista/scheduler/Cargo.toml b/ballista/scheduler/Cargo.toml index db8d22a0b..d3ab965fa 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,8 @@ 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.1.0" +datafusion-proto = "16.1.0" etcd-client = { version = "0.10", optional = true } flatbuffers = { version = "22.9.29" } futures = "0.3" diff --git a/ballista/scheduler/src/api/handlers.rs b/ballista/scheduler/src/api/handlers.rs index 7e77aa34e..63ae2436b 100644 --- a/ballista/scheduler/src/api/handlers.rs +++ b/ballista/scheduler/src/api/handlers.rs @@ -15,10 +15,10 @@ use crate::scheduler_server::SchedulerServer; use crate::state::execution_graph::ExecutionStage; use crate::state::execution_graph_dot::ExecutionGraphDot; use ballista_core::serde::protobuf::job_status::Status; -use ballista_core::serde::AsExecutionPlan; use ballista_core::BALLISTA_VERSION; use datafusion::physical_plan::metrics::{MetricValue, MetricsSet, Time}; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use graphviz_rust::cmd::{CommandArg, Format}; use graphviz_rust::exec; use graphviz_rust::printer::PrinterContext; diff --git a/ballista/scheduler/src/api/mod.rs b/ballista/scheduler/src/api/mod.rs index c8defd67c..8f5555d06 100644 --- a/ballista/scheduler/src/api/mod.rs +++ b/ballista/scheduler/src/api/mod.rs @@ -14,8 +14,8 @@ mod handlers; use crate::scheduler_server::SchedulerServer; use anyhow::Result; -use ballista_core::serde::AsExecutionPlan; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use std::{ pin::Pin, task::{Context as TaskContext, Poll}, diff --git a/ballista/scheduler/src/flight_sql.rs b/ballista/scheduler/src/flight_sql.rs index a5a613c44..b8d01d423 100644 --- a/ballista/scheduler/src/flight_sql.rs +++ b/ballista/scheduler/src/flight_sql.rs @@ -49,7 +49,6 @@ use ballista_core::serde::protobuf; use ballista_core::serde::protobuf::action::ActionType::FetchPartition; use ballista_core::serde::protobuf::job_status; use ballista_core::serde::protobuf::JobStatus; -use ballista_core::serde::protobuf::PhysicalPlanNode; use ballista_core::serde::protobuf::SuccessfulJob; use ballista_core::utils::create_grpc_client_connection; use dashmap::DashMap; @@ -63,7 +62,7 @@ use datafusion::common::DFSchemaRef; use datafusion::logical_expr::LogicalPlan; use datafusion::physical_plan::common::batch_byte_size; use datafusion::prelude::SessionContext; -use datafusion_proto::protobuf::LogicalPlanNode; +use datafusion_proto::protobuf::{LogicalPlanNode, PhysicalPlanNode}; use prost::Message; use tokio::sync::mpsc::{channel, Receiver, Sender}; use tokio::time::sleep; @@ -180,7 +179,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..6c07c6477 100644 --- a/ballista/scheduler/src/planner.rs +++ b/ballista/scheduler/src/planner.rs @@ -323,7 +323,7 @@ mod test { use crate::test_utils::datafusion_test_context; use ballista_core::error::BallistaError; use ballista_core::execution_plans::UnresolvedShuffleExec; - use ballista_core::serde::{protobuf, AsExecutionPlan, BallistaCodec}; + use ballista_core::serde::BallistaCodec; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode}; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::joins::HashJoinExec; @@ -335,8 +335,9 @@ mod test { use datafusion::prelude::SessionContext; use std::ops::Deref; - use ballista_core::serde::protobuf::PhysicalPlanNode; + use datafusion_proto::physical_plan::AsExecutionPlan; use datafusion_proto::protobuf::LogicalPlanNode; + use datafusion_proto::protobuf::PhysicalPlanNode; use std::sync::Arc; use uuid::Uuid; @@ -349,6 +350,7 @@ mod test { #[tokio::test] async fn distributed_aggregate_plan() -> Result<(), BallistaError> { let ctx = datafusion_test_context("testdata").await?; + let session_state = ctx.state(); // simplified form of TPC-H query 1 let df = ctx @@ -360,9 +362,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 = session_state.optimize(&plan)?; + let plan = session_state.create_physical_plan(&plan).await?; let mut planner = DistributedPlanner::new(); let job_uuid = Uuid::new_v4(); @@ -434,6 +436,7 @@ mod test { #[tokio::test] async fn distributed_join_plan() -> Result<(), BallistaError> { let ctx = datafusion_test_context("testdata").await?; + let session_state = ctx.state(); // simplified form of TPC-H query 12 let df = ctx @@ -472,9 +475,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 = session_state.optimize(&plan)?; + let plan = session_state.create_physical_plan(&plan).await?; let mut planner = DistributedPlanner::new(); let job_uuid = Uuid::new_v4(); @@ -607,6 +610,7 @@ order by #[tokio::test] async fn roundtrip_serde_aggregate() -> Result<(), BallistaError> { let ctx = datafusion_test_context("testdata").await?; + let session_state = ctx.state(); // simplified form of TPC-H query 1 let df = ctx @@ -618,16 +622,16 @@ 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 = session_state.optimize(&plan)?; + let plan = session_state.create_physical_plan(&plan).await?; let mut planner = DistributedPlanner::new(); let job_uuid = Uuid::new_v4(); let stages = planner.plan_query_stages(&job_uuid.to_string(), plan)?; let partial_hash = stages[0].children()[0].clone(); - let partial_hash_serde = roundtrip_operator(partial_hash.clone())?; + let partial_hash_serde = roundtrip_operator(&ctx, partial_hash.clone())?; let partial_hash = downcast_exec!(partial_hash, AggregateExec); let partial_hash_serde = downcast_exec!(partial_hash_serde, AggregateExec); @@ -641,19 +645,19 @@ order by } fn roundtrip_operator( + ctx: &SessionContext, plan: Arc, ) -> Result, BallistaError> { - let ctx = SessionContext::new(); let codec: BallistaCodec = BallistaCodec::default(); - let proto: protobuf::PhysicalPlanNode = - protobuf::PhysicalPlanNode::try_from_physical_plan( + let proto: datafusion_proto::protobuf::PhysicalPlanNode = + datafusion_proto::protobuf::PhysicalPlanNode::try_from_physical_plan( plan.clone(), codec.physical_extension_codec(), )?; let runtime = ctx.runtime_env(); let result_exec_plan: Arc = (proto).try_into_physical_plan( - &ctx, + ctx, runtime.deref(), codec.physical_extension_codec(), )?; diff --git a/ballista/scheduler/src/scheduler_process.rs b/ballista/scheduler/src/scheduler_process.rs index 44cf7bf45..a44568671 100644 --- a/ballista/scheduler/src/scheduler_process.rs +++ b/ballista/scheduler/src/scheduler_process.rs @@ -26,11 +26,9 @@ use std::{net::SocketAddr, sync::Arc}; use tonic::transport::server::Connected; use tower::Service; -use datafusion_proto::protobuf::LogicalPlanNode; +use datafusion_proto::protobuf::{LogicalPlanNode, PhysicalPlanNode}; -use ballista_core::serde::protobuf::{ - scheduler_grpc_server::SchedulerGrpcServer, PhysicalPlanNode, -}; +use ballista_core::serde::protobuf::scheduler_grpc_server::SchedulerGrpcServer; use ballista_core::serde::BallistaCodec; use ballista_core::utils::create_grpc_server; use ballista_core::BALLISTA_VERSION; diff --git a/ballista/scheduler/src/scheduler_server/external_scaler.rs b/ballista/scheduler/src/scheduler_server/external_scaler.rs index 60237dde2..104037e20 100644 --- a/ballista/scheduler/src/scheduler_server/external_scaler.rs +++ b/ballista/scheduler/src/scheduler_server/external_scaler.rs @@ -20,8 +20,8 @@ use crate::scheduler_server::externalscaler::{ GetMetricsResponse, IsActiveResponse, MetricSpec, MetricValue, ScaledObjectRef, }; use crate::scheduler_server::SchedulerServer; -use ballista_core::serde::AsExecutionPlan; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use tonic::{Request, Response}; diff --git a/ballista/scheduler/src/scheduler_server/grpc.rs b/ballista/scheduler/src/scheduler_server/grpc.rs index ad04efb21..91a161cc1 100644 --- a/ballista/scheduler/src/scheduler_server/grpc.rs +++ b/ballista/scheduler/src/scheduler_server/grpc.rs @@ -30,21 +30,20 @@ use ballista_core::serde::protobuf::{ RegisterExecutorResult, UpdateTaskStatusParams, UpdateTaskStatusResult, }; use ballista_core::serde::scheduler::{ExecutorData, ExecutorMetadata}; -use ballista_core::serde::AsExecutionPlan; - -use object_store::{local::LocalFileSystem, path::Path, ObjectStore}; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::file_format::FileFormat; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use futures::TryStreamExt; use log::{debug, error, info, trace, warn}; +use object_store::{local::LocalFileSystem, path::Path, ObjectStore}; 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}; @@ -288,15 +287,17 @@ impl SchedulerGrpc &self, request: Request, ) -> Result, Status> { + // Here, we use the default config, since we don't know the session id + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + // TODO support multiple object stores let obj_store: Arc = Arc::new(LocalFileSystem::new()); // TODO shouldn't this take a ListingOption object as input? 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::default())), // TODO implement for CSV _ => Err(tonic::Status::unimplemented( "get_file_metadata unsupported file type", @@ -321,7 +322,7 @@ impl SchedulerGrpc })?; let schema = file_format - .infer_schema(&obj_store, &file_metas) + .infer_schema(&state, &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); @@ -574,6 +575,7 @@ mod test { use std::time::Duration; use datafusion_proto::protobuf::LogicalPlanNode; + use datafusion_proto::protobuf::PhysicalPlanNode; use tonic::Request; use crate::config::SchedulerConfig; @@ -581,8 +583,8 @@ mod test { use ballista_core::error::BallistaError; use ballista_core::serde::protobuf::{ executor_registration::OptionalHost, executor_status, ExecutorRegistration, - ExecutorStatus, ExecutorStoppedParams, HeartBeatParams, PhysicalPlanNode, - PollWorkParams, RegisterExecutorParams, + ExecutorStatus, ExecutorStoppedParams, HeartBeatParams, PollWorkParams, + RegisterExecutorParams, }; use ballista_core::serde::scheduler::ExecutorSpecification; use ballista_core::serde::BallistaCodec; diff --git a/ballista/scheduler/src/scheduler_server/mod.rs b/ballista/scheduler/src/scheduler_server/mod.rs index f39e0d1b4..b7468594f 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -21,13 +21,14 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use ballista_core::error::Result; use ballista_core::event_loop::{EventLoop, EventSender}; use ballista_core::serde::protobuf::{StopExecutorParams, TaskStatus}; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; use ballista_core::utils::default_session_builder; use datafusion::execution::context::SessionState; use datafusion::logical_expr::LogicalPlan; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use crate::config::SchedulerConfig; use crate::metrics::SchedulerMetricsCollector; @@ -350,6 +351,7 @@ mod test { use datafusion::test_util::scan_empty; use datafusion_proto::protobuf::LogicalPlanNode; + use datafusion_proto::protobuf::PhysicalPlanNode; use ballista_core::config::{ BallistaConfig, TaskSchedulingPolicy, BALLISTA_DEFAULT_SHUFFLE_PARTITIONS, @@ -360,8 +362,8 @@ mod test { use ballista_core::serde::protobuf::{ failed_task, job_status, task_status, ExecutionError, FailedTask, JobStatus, - MultiTaskDefinition, PhysicalPlanNode, ShuffleWritePartition, SuccessfulJob, - SuccessfulTask, TaskId, TaskStatus, + MultiTaskDefinition, ShuffleWritePartition, SuccessfulJob, SuccessfulTask, + TaskId, TaskStatus, }; use ballista_core::serde::scheduler::{ ExecutorData, ExecutorMetadata, ExecutorSpecification, @@ -613,7 +615,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/scheduler_server/query_stage_scheduler.rs b/ballista/scheduler/src/scheduler_server/query_stage_scheduler.rs index 29f3d2d7a..b2919d22b 100644 --- a/ballista/scheduler/src/scheduler_server/query_stage_scheduler.rs +++ b/ballista/scheduler/src/scheduler_server/query_stage_scheduler.rs @@ -26,8 +26,8 @@ use ballista_core::event_loop::{EventAction, EventSender}; use crate::metrics::SchedulerMetricsCollector; use crate::scheduler_server::timestamp_millis; -use ballista_core::serde::AsExecutionPlan; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use tokio::sync::mpsc; use crate::scheduler_server::event::QueryStageSchedulerEvent; diff --git a/ballista/scheduler/src/standalone.rs b/ballista/scheduler/src/standalone.rs index 0fe608f87..4438c44a9 100644 --- a/ballista/scheduler/src/standalone.rs +++ b/ballista/scheduler/src/standalone.rs @@ -19,7 +19,6 @@ use crate::config::SchedulerConfig; use crate::metrics::default_metrics_collector; use crate::state::backend::cluster::DefaultClusterState; use crate::{scheduler_server::SchedulerServer, state::backend::sled::SledClient}; -use ballista_core::serde::protobuf::PhysicalPlanNode; use ballista_core::serde::BallistaCodec; use ballista_core::utils::create_grpc_server; use ballista_core::{ @@ -27,6 +26,7 @@ use ballista_core::{ BALLISTA_VERSION, }; use datafusion_proto::protobuf::LogicalPlanNode; +use datafusion_proto::protobuf::PhysicalPlanNode; use log::info; use std::{net::SocketAddr, sync::Arc}; use tokio::net::TcpListener; diff --git a/ballista/scheduler/src/state/execution_graph.rs b/ballista/scheduler/src/state/execution_graph.rs index 976ec5f51..8105b0ac8 100644 --- a/ballista/scheduler/src/state/execution_graph.rs +++ b/ballista/scheduler/src/state/execution_graph.rs @@ -42,7 +42,8 @@ use ballista_core::serde::protobuf::{task_status, RunningTask}; use ballista_core::serde::scheduler::{ ExecutorMetadata, PartitionId, PartitionLocation, PartitionStats, }; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; +use datafusion_proto::physical_plan::AsExecutionPlan; use crate::display::print_stage_metrics; use crate::planner::DistributedPlanner; @@ -1589,8 +1590,8 @@ mod test { use std::sync::Arc; use datafusion::arrow::datatypes::{DataType, Field, Schema}; - use datafusion::logical_expr::JoinType; - use datafusion::logical_expr::{col, count, sum, Expr}; + use datafusion::logical_expr::expr::Sort; + use datafusion::logical_expr::{col, count, sum, Expr, JoinType}; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion::test_util::scan_empty; @@ -2760,6 +2761,7 @@ mod test { async fn test_aggregation_plan(partition: usize) -> ExecutionGraph { let config = SessionConfig::new().with_target_partitions(partition); let ctx = Arc::new(SessionContext::with_config(config)); + let session_state = ctx.state(); let schema = Schema::new(vec![ Field::new("id", DataType::Utf8, false), @@ -2773,9 +2775,12 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = session_state.optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = session_state + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2785,6 +2790,7 @@ mod test { async fn test_two_aggregations_plan(partition: usize) -> ExecutionGraph { let config = SessionConfig::new().with_target_partitions(partition); let ctx = Arc::new(SessionContext::with_config(config)); + let session_state = ctx.state(); let schema = Schema::new(vec![ Field::new("id", DataType::Utf8, false), @@ -2801,9 +2807,12 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = session_state.optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = session_state + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2813,6 +2822,7 @@ mod test { async fn test_coalesce_plan(partition: usize) -> ExecutionGraph { let config = SessionConfig::new().with_target_partitions(partition); let ctx = Arc::new(SessionContext::with_config(config)); + let session_state = ctx.state(); let schema = Schema::new(vec![ Field::new("id", DataType::Utf8, false), @@ -2826,16 +2836,24 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = session_state.optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = session_state + .create_physical_plan(&optimized_plan) + .await + .unwrap(); ExecutionGraph::new("localhost:50050", "job", "", "session", plan, 0).unwrap() } async fn test_join_plan(partition: usize) -> ExecutionGraph { - let config = SessionConfig::new().with_target_partitions(partition); + let mut config = SessionConfig::new().with_target_partitions(partition); + config + .config_options_mut() + .optimizer + .enable_round_robin_repartition = false; let ctx = Arc::new(SessionContext::with_config(config)); + let session_state = ctx.state(); let schema = Schema::new(vec![ Field::new("id", DataType::Utf8, false), @@ -2849,14 +2867,10 @@ mod test { .build() .unwrap(); - let sort_expr = Expr::Sort { - expr: Box::new(col("id")), - asc: false, - nulls_first: false, - }; + let sort_expr = Expr::Sort(Sort::new(Box::new(col("id")), false, 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 +2879,12 @@ mod test { .build() .unwrap(); - let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + let optimized_plan = session_state.optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = session_state + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2882,17 +2899,21 @@ mod test { async fn test_union_all_plan(partition: usize) -> ExecutionGraph { let config = SessionConfig::new().with_target_partitions(partition); let ctx = Arc::new(SessionContext::with_config(config)); + let session_state = ctx.state(); let logical_plan = ctx .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 = session_state.optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = session_state + .create_physical_plan(&optimized_plan) + .await + .unwrap(); println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); @@ -2907,17 +2928,21 @@ mod test { async fn test_union_plan(partition: usize) -> ExecutionGraph { let config = SessionConfig::new().with_target_partitions(partition); let ctx = Arc::new(SessionContext::with_config(config)); + let session_state = ctx.state(); let logical_plan = ctx .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 = session_state.optimize(&logical_plan).unwrap(); - let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + let plan = session_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..b49d7214e 100644 --- a/ballista/scheduler/src/state/execution_graph/execution_stage.rs +++ b/ballista/scheduler/src/state/execution_graph/execution_stage.rs @@ -32,7 +32,6 @@ use datafusion_proto::logical_plan::AsLogicalPlan; use log::{debug, warn}; use ballista_core::error::{BallistaError, Result}; -use ballista_core::serde::physical_plan::from_proto::parse_protobuf_hash_partitioning; use ballista_core::serde::protobuf::failed_task::FailedReason; use ballista_core::serde::protobuf::{ self, task_info, FailedTask, GraphStageInput, OperatorMetricsSet, ResultLost, @@ -41,7 +40,9 @@ use ballista_core::serde::protobuf::{ use ballista_core::serde::protobuf::{task_status, RunningTask}; use ballista_core::serde::scheduler::to_proto::hash_partitioning_to_proto; use ballista_core::serde::scheduler::PartitionLocation; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; +use datafusion_proto::physical_plan::from_proto::parse_protobuf_hash_partitioning; +use datafusion_proto::physical_plan::AsExecutionPlan; use crate::display::DisplayableBallistaExecutionPlan; @@ -364,7 +365,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::default().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..cf7a4b963 100644 --- a/ballista/scheduler/src/state/execution_graph_dot.rs +++ b/ballista/scheduler/src/state/execution_graph_dot.rs @@ -633,8 +633,14 @@ filter_expr="] } async fn test_graph() -> Result { - let ctx = - SessionContext::with_config(SessionConfig::new().with_target_partitions(48)); + let mut config = SessionConfig::new() + .with_target_partitions(48) + .with_batch_size(4096); + config + .config_options_mut() + .optimizer + .enable_round_robin_repartition = false; + let ctx = SessionContext::with_config(config); let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::UInt32, false), Field::new("b", DataType::UInt32, false), @@ -646,16 +652,22 @@ 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) } // With the improvement of https://github.com/apache/arrow-datafusion/pull/4122, // Redundant RepartitionExec can be removed so that the stage number will be reduced async fn test_graph_optimized() -> Result { - let ctx = - SessionContext::with_config(SessionConfig::new().with_target_partitions(48)); + let mut config = SessionConfig::new() + .with_target_partitions(48) + .with_batch_size(4096); + config + .config_options_mut() + .optimizer + .enable_round_robin_repartition = false; + let ctx = SessionContext::with_config(config); let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::UInt32, false)])); let table = Arc::new(MemTable::try_new(schema.clone(), vec![])?); @@ -665,8 +677,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..1a5d50e4e 100644 --- a/ballista/scheduler/src/state/mod.rs +++ b/ballista/scheduler/src/state/mod.rs @@ -36,11 +36,12 @@ use crate::state::execution_graph::TaskDescription; use backend::cluster::ClusterState; use ballista_core::error::{BallistaError, Result}; use ballista_core::serde::protobuf::TaskStatus; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; use datafusion::logical_expr::LogicalPlan; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::prelude::SessionContext; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use log::{debug, error, info}; use prost::Message; @@ -338,7 +339,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 +392,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/ballista/scheduler/src/state/task_manager.rs b/ballista/scheduler/src/state/task_manager.rs index 53b592196..944cdc0e5 100644 --- a/ballista/scheduler/src/state/task_manager.rs +++ b/ballista/scheduler/src/state/task_manager.rs @@ -36,11 +36,12 @@ use ballista_core::serde::protobuf::{ }; use ballista_core::serde::scheduler::to_proto::hash_partitioning_to_proto; use ballista_core::serde::scheduler::ExecutorMetadata; -use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; +use ballista_core::serde::BallistaCodec; use dashmap::DashMap; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; use datafusion_proto::logical_plan::AsLogicalPlan; +use datafusion_proto::physical_plan::AsExecutionPlan; use log::{debug, error, info, warn}; use rand::distributions::Alphanumeric; use rand::{thread_rng, Rng}; diff --git a/ballista/scheduler/src/test_utils.rs b/ballista/scheduler/src/test_utils.rs index 3f98e3708..c802c6f61 100644 --- a/ballista/scheduler/src/test_utils.rs +++ b/ballista/scheduler/src/test_utils.rs @@ -35,8 +35,8 @@ use crate::state::task_manager::TaskLauncher; use ballista_core::config::{BallistaConfig, BALLISTA_DEFAULT_SHUFFLE_PARTITIONS}; use ballista_core::serde::protobuf::job_status::Status; use ballista_core::serde::protobuf::{ - task_status, JobStatus, MultiTaskDefinition, PhysicalPlanNode, ShuffleWritePartition, - SuccessfulTask, TaskId, TaskStatus, + task_status, JobStatus, MultiTaskDefinition, ShuffleWritePartition, SuccessfulTask, + TaskId, TaskStatus, }; use ballista_core::serde::scheduler::{ ExecutorData, ExecutorMetadata, ExecutorSpecification, @@ -52,7 +52,7 @@ use datafusion::prelude::CsvReadOptions; use crate::scheduler_server::event::QueryStageSchedulerEvent; use crate::state::backend::cluster::DefaultClusterState; -use datafusion_proto::protobuf::LogicalPlanNode; +use datafusion_proto::protobuf::{LogicalPlanNode, PhysicalPlanNode}; use parking_lot::Mutex; use tokio::sync::mpsc::{channel, Receiver, Sender}; diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index 27121d219..3fa9e0a51 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.1.0" +datafusion-proto = "16.1.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..d02b02cff 100644 --- a/benchmarks/src/bin/nyctaxi.rs +++ b/benchmarks/src/bin/nyctaxi.rs @@ -119,12 +119,13 @@ 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 session_state = ctx.state(); + let plan = session_state.create_logical_plan(sql).await?; + let plan = session_state.optimize(&plan)?; if debug { println!("Optimized logical plan:\n{:?}", plan); } - let physical_plan = ctx.create_physical_plan(&plan).await?; + let physical_plan = session_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..3798b14dc 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -62,6 +62,7 @@ use std::{ time::{Instant, SystemTime}, }; use structopt::StructOpt; +use tokio::task::JoinHandle; #[cfg(feature = "snmalloc")] #[global_allocator] @@ -293,7 +294,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,7 +395,7 @@ async fn benchmark_ballista(opt: BallistaBenchmarkOpt) -> Result<()> { .await .map_err(|e| DataFusionError::Plan(format!("{:?}", e))) .unwrap(); - let plan = df.to_logical_plan()?; + let plan = df.clone().into_optimized_plan()?; if opt.debug { println!("=== Optimized logical plan ===\n{:?}\n", plan); } @@ -676,11 +677,29 @@ 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> { - let sql = get_query_sql(query)?; - sql.iter() - .map(|sql| ctx.create_logical_plan(sql.as_str())) - .collect::>>() +async fn create_logical_plans( + ctx: &SessionContext, + query: usize, +) -> Result> { + let session_state = ctx.state(); + let sqls = get_query_sql(query)?; + let join_handles = sqls + .into_iter() + .map(|sql| { + let session_state = session_state.clone(); + tokio::spawn( + async move { session_state.create_logical_plan(sql.as_str()).await }, + ) + }) + .collect::>>>(); + futures::future::join_all(join_handles) + .await + .into_iter() + .collect::>, tokio::task::JoinError>>( + ) + .map_err(|e| DataFusionError::Internal(format!("{:?}", e)))? + .into_iter() + .collect() } async fn execute_query( @@ -691,11 +710,12 @@ async fn execute_query( if debug { println!("=== Logical plan ===\n{:?}\n", plan); } - let plan = ctx.optimize(plan)?; + let session_state = ctx.state(); + let plan = session_state.optimize(plan)?; if debug { println!("=== Optimized logical plan ===\n{:?}\n", plan); } - let physical_plan = ctx.create_physical_plan(&plan).await?; + let physical_plan = session_state.create_physical_plan(&plan).await?; if debug { println!( "=== Physical plan ===\n{}\n", @@ -730,6 +750,7 @@ async fn convert_tbl(opt: ConvertOpt) -> Result<()> { let config = SessionConfig::new().with_batch_size(opt.batch_size); let ctx = SessionContext::with_config(config); + let session_state = ctx.state(); // build plan to read the TBL file let mut csv = ctx.read_csv(&input_path, options).await?; @@ -740,9 +761,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 = session_state.optimize(&csv)?; + let csv = session_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 +837,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::default().with_enable_pruning(Some(true)); (Arc::new(format), path, DEFAULT_PARQUET_EXTENSION) } @@ -834,6 +854,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 +1523,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?; } @@ -1541,10 +1562,11 @@ mod tests { mod ballista_round_trip { use super::*; - use ballista_core::serde::{protobuf, AsExecutionPlan, BallistaCodec}; + use ballista_core::serde::BallistaCodec; use datafusion::datasource::listing::ListingTableUrl; use datafusion::physical_plan::ExecutionPlan; use datafusion_proto::logical_plan::AsLogicalPlan; + use datafusion_proto::physical_plan::AsExecutionPlan; use std::ops::Deref; async fn round_trip_query(n: usize) -> Result<()> { @@ -1552,9 +1574,10 @@ mod tests { .with_target_partitions(1) .with_batch_size(10); let ctx = SessionContext::with_config(config); + let session_state = ctx.state(); let codec: BallistaCodec< datafusion_proto::protobuf::LogicalPlanNode, - protobuf::PhysicalPlanNode, + datafusion_proto::protobuf::PhysicalPlanNode, > = BallistaCodec::default(); // set tpch_data_path to dummy value and skip physical plan serde test when TPCH_DATA @@ -1579,10 +1602,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 = session_state.optimize(&plan)?; let proto: datafusion_proto::protobuf::LogicalPlanNode = datafusion_proto::protobuf::LogicalPlanNode::try_from_logical_plan( &plan, @@ -1600,9 +1623,9 @@ mod tests { // test physical plan roundtrip if env::var("TPCH_DATA").is_ok() { - let physical_plan = ctx.create_physical_plan(&plan).await?; - let proto: protobuf::PhysicalPlanNode = - protobuf::PhysicalPlanNode::try_from_physical_plan( + let physical_plan = session_state.create_physical_plan(&plan).await?; + let proto: datafusion_proto::protobuf::PhysicalPlanNode = + datafusion_proto::protobuf::PhysicalPlanNode::try_from_physical_plan( physical_plan.clone(), codec.physical_extension_codec(), ) diff --git a/examples/Cargo.toml b/examples/Cargo.toml index 06aa536d6..e38a58ec5 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.1.0" futures = "0.3" num_cpus = "1.13.0" prost = "0.11" diff --git a/python/Cargo.toml b/python/Cargo.toml index 5879d7443..2d4b32410 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.1.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 e1be3af8c68bd5f38bdb15116a506f4a1c034e6a Mon Sep 17 00:00:00 2001 From: yangzhong Date: Sun, 29 Jan 2023 14:35:41 +0800 Subject: [PATCH 2/4] Fix python --- python/Cargo.toml | 2 + python/src/catalog.rs | 15 ++-- python/src/context.rs | 22 ++++-- python/src/dataframe.rs | 36 +++++---- python/src/dataset_exec.rs | 4 - python/src/functions.rs | 150 ++++++++++++++++++++++--------------- python/src/udaf.rs | 12 +-- 7 files changed, 138 insertions(+), 103 deletions(-) diff --git a/python/Cargo.toml b/python/Cargo.toml index 2d4b32410..59a9f8475 100644 --- a/python/Cargo.toml +++ b/python/Cargo.toml @@ -37,6 +37,8 @@ default = ["mimalloc"] async-trait = "0.1" ballista = { path = "../ballista/client", version = "0.10.0" } datafusion = { version = "16.1.0", features = ["pyarrow"] } +datafusion-common = "16.1.0" +datafusion-expr = "16.1.0" futures = "0.3" mimalloc = { version = "*", optional = true, default-features = false } pyo3 = { version = "~0.17.1", features = ["extension-module", "abi3", "abi3-py37"] } diff --git a/python/src/catalog.rs b/python/src/catalog.rs index f3299d2a4..07f8d6917 100644 --- a/python/src/catalog.rs +++ b/python/src/catalog.rs @@ -27,6 +27,9 @@ use datafusion::{ datasource::{TableProvider, TableType}, }; +use crate::errors::DataFusionError; +use crate::utils::wait_for_future; + #[pyclass(name = "Catalog", module = "ballista", subclass)] pub(crate) struct PyCatalog { catalog: Arc, @@ -88,13 +91,11 @@ impl PyDatabase { self.database.table_names().into_iter().collect() } - fn table(&self, name: &str) -> PyResult { - match self.database.table(name) { - Some(table) => Ok(PyTable::new(table)), - None => Err(PyKeyError::new_err(format!( - "Table with name {} doesn't exist.", - name - ))), + fn table(&self, name: &str, py: Python) -> PyResult { + if let Some(table) = wait_for_future(py, self.database.table(name)) { + Ok(PyTable::new(table)) + } else { + Err(DataFusionError::Common(format!("Table with name {name} doesn't exist.")).into()) } } diff --git a/python/src/context.rs b/python/src/context.rs index d479a5f42..d94a5e077 100644 --- a/python/src/context.rs +++ b/python/src/context.rs @@ -29,7 +29,7 @@ use datafusion::arrow::record_batch::RecordBatch; use datafusion::datasource::datasource::TableProvider; use datafusion::datasource::MemTable; use datafusion::execution::context::{SessionConfig, SessionContext}; -use datafusion::prelude::{CsvReadOptions, ParquetReadOptions}; +use datafusion::prelude::{CsvReadOptions, DataFrame, ParquetReadOptions}; use crate::catalog::{PyCatalog, PyTable}; use crate::dataframe::PyDataFrame; @@ -75,7 +75,7 @@ impl PySessionContext { // TODO: config_options ) -> Self { let cfg = SessionConfig::new() - .create_default_catalog_and_schema(create_default_catalog_and_schema) + .with_create_default_catalog_and_schema(create_default_catalog_and_schema) .with_default_catalog_and_schema(default_catalog, default_schema) .with_information_schema(information_schema) .with_repartition_joins(repartition_joins) @@ -103,6 +103,7 @@ impl PySessionContext { fn create_dataframe( &mut self, partitions: PyArrowType>>, + py: Python, ) -> PyResult { let table = MemTable::try_new(partitions.0[0][0].schema(), partitions.0) .map_err(DataFusionError::from)?; @@ -117,10 +118,9 @@ impl PySessionContext { self.ctx .register_table(&*name, Arc::new(table)) .map_err(DataFusionError::from)?; - let table = self.ctx.table(&*name).map_err(DataFusionError::from)?; + let table = wait_for_future(py, self._table(&name)).map_err(DataFusionError::from)?; - let df = PyDataFrame::new(table); - Ok(df) + Ok(PyDataFrame::new(table)) } fn register_table(&mut self, name: &str, table: &PyTable) -> PyResult<()> { @@ -248,8 +248,10 @@ impl PySessionContext { self.ctx.tables().unwrap() } - fn table(&self, name: &str) -> PyResult { - Ok(PyDataFrame::new(self.ctx.table(name)?)) + fn table(&self, name: &str, py: Python) -> PyResult { + let table = wait_for_future(py, self._table(name)).map_err(DataFusionError::from)?; + + Ok(PyDataFrame::new(table)) } fn empty_table(&self) -> PyResult { @@ -257,6 +259,12 @@ impl PySessionContext { } } +impl PySessionContext { + async fn _table(&self, name: &str) -> datafusion_common::Result { + self.ctx.table(name).await + } +} + fn convert_table_partition_cols( table_partition_cols: Vec<(String, PyDataType)>, ) -> Vec<(String, DataType)> { diff --git a/python/src/dataframe.rs b/python/src/dataframe.rs index 295f89dc9..8f7f56ea3 100644 --- a/python/src/dataframe.rs +++ b/python/src/dataframe.rs @@ -38,8 +38,8 @@ pub(crate) struct PyDataFrame { impl PyDataFrame { /// creates a new PyDataFrame - pub fn new(df: Arc) -> Self { - Self { df } + pub fn new(df: DataFrame) -> Self { + Self { df: Arc::new(df) } } } @@ -71,43 +71,43 @@ impl PyDataFrame { #[args(args = "*")] fn select_columns(&self, args: Vec<&str>) -> PyResult { - let df = self.df.select_columns(&args)?; + let df = self.df.as_ref().clone().select_columns(&args)?; Ok(Self::new(df)) } #[args(args = "*")] fn select(&self, args: Vec) -> PyResult { let expr = args.into_iter().map(|e| e.into()).collect(); - let df = self.df.select(expr)?; + let df = self.df.as_ref().clone().select(expr)?; Ok(Self::new(df)) } fn filter(&self, predicate: PyExpr) -> PyResult { - let df = self.df.filter(predicate.into())?; + let df = self.df.as_ref().clone().filter(predicate.into())?; Ok(Self::new(df)) } fn with_column(&self, name: &str, expr: PyExpr) -> PyResult { - let df = self.df.with_column(name, expr.into())?; + let df = self.df.as_ref().clone().with_column(name, expr.into())?; Ok(Self::new(df)) } fn aggregate(&self, group_by: Vec, aggs: Vec) -> PyResult { let group_by = group_by.into_iter().map(|e| e.into()).collect(); let aggs = aggs.into_iter().map(|e| e.into()).collect(); - let df = self.df.aggregate(group_by, aggs)?; + let df = self.df.as_ref().clone().aggregate(group_by, aggs)?; Ok(Self::new(df)) } #[args(exprs = "*")] fn sort(&self, exprs: Vec) -> PyResult { let exprs = exprs.into_iter().map(|e| e.into()).collect(); - let df = self.df.sort(exprs)?; + let df = self.df.as_ref().clone().sort(exprs)?; Ok(Self::new(df)) } fn limit(&self, count: usize) -> PyResult { - let df = self.df.limit(0, Some(count))?; + let df = self.df.as_ref().clone().limit(0, Some(count))?; Ok(Self::new(df)) } @@ -115,7 +115,7 @@ impl PyDataFrame { /// Unless some order is specified in the plan, there is no /// guarantee of the order of the result. fn collect(&self, py: Python) -> PyResult> { - let batches = wait_for_future(py, self.df.collect())?; + let batches = wait_for_future(py, self.df.as_ref().clone().collect())?; // cannot use PyResult> return type due to // https://github.com/PyO3/pyo3/issues/1813 batches.into_iter().map(|rb| rb.to_pyarrow(py)).collect() @@ -124,7 +124,7 @@ impl PyDataFrame { /// Print the result, 20 lines by default #[args(num = "20")] fn show(&self, py: Python, num: usize) -> PyResult<()> { - let df = self.df.limit(0, Some(num))?; + let df = self.df.as_ref().clone().limit(0, Some(num))?; let batches = wait_for_future(py, df.collect())?; pretty::print_batches(&batches) .map_err(|err| PyArrowException::new_err(err.to_string())) @@ -153,16 +153,20 @@ impl PyDataFrame { } }; - let df = self - .df - .join(right.df, join_type, &join_keys.0, &join_keys.1, None)?; + let df = self.df.as_ref().clone().join( + right.df.as_ref().clone(), + join_type, + &join_keys.0, + &join_keys.1, + None, + )?; Ok(Self::new(df)) } /// Print the explain output to stdout #[args(verbose = false, analyze = false)] fn explain(&self, py: Python, verbose: bool, analyze: bool) -> PyResult<()> { - let df = self.df.explain(verbose, analyze)?; + let df = self.df.as_ref().clone().explain(verbose, analyze)?; let batches = wait_for_future(py, df.collect())?; pretty::print_batches(&batches) .map_err(|err| PyArrowException::new_err(err.to_string())) @@ -176,7 +180,7 @@ impl PyDataFrame { verbose: bool, analyze: bool, ) -> PyResult { - let df = self.df.explain(verbose, analyze)?; + let df = self.df.as_ref().clone().explain(verbose, analyze)?; let batches = wait_for_future(py, df.collect())?; let display = pretty::pretty_format_batches(&batches) .map_err(|err| PyArrowException::new_err(err.to_string()))?; diff --git a/python/src/dataset_exec.rs b/python/src/dataset_exec.rs index f238d54a8..a3c77a720 100644 --- a/python/src/dataset_exec.rs +++ b/python/src/dataset_exec.rs @@ -162,10 +162,6 @@ impl ExecutionPlan for DatasetExec { }) } - fn relies_on_input_order(&self) -> bool { - false - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { None } diff --git a/python/src/functions.rs b/python/src/functions.rs index be6fff25c..9e10f8aff 100644 --- a/python/src/functions.rs +++ b/python/src/functions.rs @@ -18,22 +18,14 @@ use datafusion::prelude::lit; use pyo3::{prelude::*, wrap_pyfunction}; -use datafusion::logical_expr::{self, BuiltinScalarFunction, WindowFunction}; use datafusion::physical_plan::aggregates::AggregateFunction; +use datafusion_expr::{self, BuiltinScalarFunction, window_function::find_df_window_func}; -use crate::errors; use crate::expression::PyExpr; -#[pyfunction] -fn array(value: Vec) -> PyExpr { - PyExpr { - expr: logical_expr::array(value.into_iter().map(|x| x.expr).collect::>()), - } -} - #[pyfunction] fn in_list(expr: PyExpr, value: Vec, negated: bool) -> PyExpr { - logical_expr::in_list( + datafusion_expr::expr_fn::in_list( expr.expr, value.into_iter().map(|x| x.expr).collect::>(), negated, @@ -41,29 +33,12 @@ fn in_list(expr: PyExpr, value: Vec, negated: bool) -> PyExpr { .into() } -/// Current date and time -#[pyfunction] -fn now() -> PyExpr { - PyExpr { - // here lit(0) is a stub for conform to arity - expr: logical_expr::now(), - } -} - -/// Returns a random value in the range 0.0 <= x < 1.0 -#[pyfunction] -fn random() -> PyExpr { - PyExpr { - expr: logical_expr::random(), - } -} - /// Computes a binary hash of the given data. type is the algorithm to use. /// Standard algorithms are md5, sha224, sha256, sha384, sha512, blake2s, blake2b, and blake3. #[pyfunction(value, method)] fn digest(value: PyExpr, method: PyExpr) -> PyExpr { PyExpr { - expr: logical_expr::digest(value.expr, method.expr), + expr: datafusion_expr::expr_fn::digest(value.expr, method.expr), } } @@ -72,7 +47,7 @@ fn digest(value: PyExpr, method: PyExpr) -> PyExpr { #[pyfunction(args = "*")] fn concat(args: Vec) -> PyResult { let args = args.into_iter().map(|e| e.expr).collect::>(); - Ok(logical_expr::concat(&args).into()) + Ok(datafusion_expr::expr_fn::concat(&args).into()) } /// Concatenates all but the first argument, with separators. @@ -81,7 +56,7 @@ fn concat(args: Vec) -> PyResult { #[pyfunction(sep, args = "*")] fn concat_ws(sep: String, args: Vec) -> PyResult { let args = args.into_iter().map(|e| e.expr).collect::>(); - Ok(logical_expr::concat_ws(lit(sep), args).into()) + Ok(datafusion_expr::expr_fn::concat_ws(lit(sep), args).into()) } /// Creates a new Sort expression @@ -92,11 +67,13 @@ fn order_by( nulls_first: Option, ) -> PyResult { Ok(PyExpr { - expr: datafusion::logical_expr::Expr::Sort { - expr: Box::new(expr.expr), - asc: asc.unwrap_or(true), - nulls_first: nulls_first.unwrap_or(true), - }, + expr: datafusion_expr::expr::Expr::Sort ( + datafusion_expr::expr::Sort { + expr: Box::new(expr.expr), + asc: asc.unwrap_or(true), + nulls_first: nulls_first.unwrap_or(true), + } + ), }) } @@ -104,7 +81,7 @@ fn order_by( #[pyfunction] fn alias(expr: PyExpr, name: &str) -> PyResult { Ok(PyExpr { - expr: datafusion::logical_expr::Expr::Alias( + expr: datafusion_expr::Expr::Alias( Box::new(expr.expr), String::from(name), ), @@ -119,25 +96,25 @@ fn window( partition_by: Option>, order_by: Option>, ) -> PyResult { - use std::str::FromStr; - let fun = WindowFunction::from_str(name) - .map_err(|e| -> errors::DataFusionError { e.into() })?; + let fun = find_df_window_func(name).unwrap(); + let has_order_by = order_by.is_some(); Ok(PyExpr { - expr: datafusion::logical_expr::Expr::WindowFunction { - fun, - args: args.into_iter().map(|x| x.expr).collect::>(), - partition_by: partition_by - .unwrap_or_default() - .into_iter() - .map(|x| x.expr) - .collect::>(), - order_by: order_by - .unwrap_or_default() - .into_iter() - .map(|x| x.expr) - .collect::>(), - window_frame: None, - }, + expr: datafusion_expr::expr::Expr::WindowFunction ( + datafusion_expr::expr::WindowFunction { + fun, + args: args.into_iter().map(|x| x.expr).collect::>(), + partition_by: partition_by + .unwrap_or_default() + .into_iter() + .map(|x| x.expr) + .collect::>(), + order_by: order_by + .unwrap_or_default() + .into_iter() + .map(|x| x.expr) + .collect::>(), + window_frame: datafusion_expr::window_frame::WindowFrame::new(has_order_by), + }), }) } @@ -149,7 +126,7 @@ macro_rules! scalar_function { #[doc = $DOC] #[pyfunction(args = "*")] fn $NAME(args: Vec) -> PyExpr { - let expr = logical_expr::Expr::ScalarFunction { + let expr = datafusion_expr::Expr::ScalarFunction { fun: BuiltinScalarFunction::$FUNC, args: args.into_iter().map(|e| e.into()).collect(), }; @@ -166,12 +143,14 @@ macro_rules! aggregate_function { #[doc = $DOC] #[pyfunction(args = "*", distinct = "false")] fn $NAME(args: Vec, distinct: bool) -> PyExpr { - let expr = logical_expr::Expr::AggregateFunction { - fun: AggregateFunction::$FUNC, - args: args.into_iter().map(|e| e.into()).collect(), - distinct, - filter: None, - }; + let expr = datafusion_expr::Expr::AggregateFunction({ + datafusion_expr::expr::AggregateFunction { + fun: AggregateFunction::$FUNC, + args: args.into_iter().map(|e| e.into()).collect(), + distinct, + filter: None, + } + }); expr.into() } }; @@ -182,6 +161,7 @@ scalar_function!(acos, Acos); scalar_function!(ascii, Ascii, "Returns the numeric code of the first character of the argument. In UTF8 encoding, returns the Unicode code point of the character. In other multibyte encodings, the argument must be an ASCII character."); scalar_function!(asin, Asin); scalar_function!(atan, Atan); +scalar_function!(atan2, Atan2); scalar_function!( bit_length, BitLength, @@ -194,13 +174,17 @@ scalar_function!( CharacterLength, "Returns number of characters in the string." ); +scalar_function!(length, CharacterLength); +scalar_function!(char_length, CharacterLength); scalar_function!(chr, Chr, "Returns the character with the given code."); +scalar_function!(coalesce, Coalesce); scalar_function!(cos, Cos); scalar_function!(exp, Exp); scalar_function!(floor, Floor); scalar_function!(initcap, InitCap, "Converts the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters."); scalar_function!(left, Left, "Returns first n characters in the string, or when n is negative, returns all but last |n| characters."); scalar_function!(ln, Ln); +scalar_function!(log, Log); scalar_function!(log10, Log10); scalar_function!(log2, Log2); scalar_function!(lower, Lower, "Converts the string to all lower case"); @@ -212,6 +196,8 @@ scalar_function!( "Computes the MD5 hash of the argument, with the result written in hexadecimal." ); scalar_function!(octet_length, OctetLength, "Returns number of bytes in the string. Since this version of the function accepts type character directly, it will not strip trailing spaces."); +scalar_function!(power, Power); +scalar_function!(pow, Power); scalar_function!(regexp_match, RegexpMatch); scalar_function!( regexp_replace, @@ -262,11 +248,30 @@ scalar_function!( ToHex, "Converts the number to its equivalent hexadecimal representation." ); +scalar_function!(now, Now); scalar_function!(to_timestamp, ToTimestamp); +scalar_function!(to_timestamp_millis, ToTimestampMillis); +scalar_function!(to_timestamp_micros, ToTimestampMicros); +scalar_function!(to_timestamp_seconds, ToTimestampSeconds); +scalar_function!(current_date, CurrentDate); +scalar_function!(current_time, CurrentTime); +scalar_function!(datepart, DatePart); +scalar_function!(date_part, DatePart); +scalar_function!(date_trunc, DateTrunc); +scalar_function!(datetrunc, DateTrunc); +scalar_function!(date_bin, DateBin); scalar_function!(translate, Translate, "Replaces each character in string that matches a character in the from set with the corresponding character in the to set. If from is longer than to, occurrences of the extra characters in from are deleted."); scalar_function!(trim, Trim, "Removes the longest string containing only characters in characters (a space by default) from the start, end, or both ends (BOTH is the default) of string."); scalar_function!(trunc, Trunc); scalar_function!(upper, Upper, "Converts the string to all upper case."); +scalar_function!(make_array, MakeArray); +scalar_function!(array, MakeArray); +scalar_function!(nullif, NullIf); +//scalar_function!(uuid, Uuid); +//scalar_function!(struct, Struct); +scalar_function!(from_unixtime, FromUnixtime); +scalar_function!(arrow_typeof, ArrowTypeof); +scalar_function!(random, Random); aggregate_function!(avg, Avg); aggregate_function!(count, Count); @@ -281,37 +286,55 @@ pub(crate) fn init_module(m: &PyModule) -> PyResult<()> { m.add_wrapped(wrap_pyfunction!(approx_distinct))?; m.add_wrapped(wrap_pyfunction!(alias))?; m.add_wrapped(wrap_pyfunction!(array))?; + m.add_wrapped(wrap_pyfunction!(arrow_typeof))?; m.add_wrapped(wrap_pyfunction!(ascii))?; m.add_wrapped(wrap_pyfunction!(asin))?; m.add_wrapped(wrap_pyfunction!(atan))?; + m.add_wrapped(wrap_pyfunction!(atan2))?; m.add_wrapped(wrap_pyfunction!(avg))?; m.add_wrapped(wrap_pyfunction!(bit_length))?; m.add_wrapped(wrap_pyfunction!(btrim))?; m.add_wrapped(wrap_pyfunction!(ceil))?; m.add_wrapped(wrap_pyfunction!(character_length))?; m.add_wrapped(wrap_pyfunction!(chr))?; + m.add_wrapped(wrap_pyfunction!(char_length))?; + m.add_wrapped(wrap_pyfunction!(coalesce))?; m.add_wrapped(wrap_pyfunction!(concat_ws))?; m.add_wrapped(wrap_pyfunction!(concat))?; m.add_wrapped(wrap_pyfunction!(cos))?; m.add_wrapped(wrap_pyfunction!(count))?; + m.add_wrapped(wrap_pyfunction!(current_date))?; + m.add_wrapped(wrap_pyfunction!(current_time))?; + m.add_wrapped(wrap_pyfunction!(date_bin))?; + m.add_wrapped(wrap_pyfunction!(datepart))?; + m.add_wrapped(wrap_pyfunction!(date_part))?; + m.add_wrapped(wrap_pyfunction!(datetrunc))?; + m.add_wrapped(wrap_pyfunction!(date_trunc))?; m.add_wrapped(wrap_pyfunction!(digest))?; m.add_wrapped(wrap_pyfunction!(exp))?; m.add_wrapped(wrap_pyfunction!(floor))?; + m.add_wrapped(wrap_pyfunction!(from_unixtime))?; m.add_wrapped(wrap_pyfunction!(in_list))?; m.add_wrapped(wrap_pyfunction!(initcap))?; m.add_wrapped(wrap_pyfunction!(left))?; + m.add_wrapped(wrap_pyfunction!(length))?; m.add_wrapped(wrap_pyfunction!(ln))?; + m.add_wrapped(wrap_pyfunction!(log))?; m.add_wrapped(wrap_pyfunction!(log10))?; m.add_wrapped(wrap_pyfunction!(log2))?; m.add_wrapped(wrap_pyfunction!(lower))?; m.add_wrapped(wrap_pyfunction!(lpad))?; m.add_wrapped(wrap_pyfunction!(ltrim))?; m.add_wrapped(wrap_pyfunction!(max))?; + m.add_wrapped(wrap_pyfunction!(make_array))?; m.add_wrapped(wrap_pyfunction!(md5))?; m.add_wrapped(wrap_pyfunction!(min))?; m.add_wrapped(wrap_pyfunction!(now))?; + m.add_wrapped(wrap_pyfunction!(nullif))?; m.add_wrapped(wrap_pyfunction!(octet_length))?; m.add_wrapped(wrap_pyfunction!(order_by))?; + m.add_wrapped(wrap_pyfunction!(power))?; + m.add_wrapped(wrap_pyfunction!(pow))?; m.add_wrapped(wrap_pyfunction!(random))?; m.add_wrapped(wrap_pyfunction!(regexp_match))?; m.add_wrapped(wrap_pyfunction!(regexp_replace))?; @@ -332,15 +355,20 @@ pub(crate) fn init_module(m: &PyModule) -> PyResult<()> { m.add_wrapped(wrap_pyfunction!(sqrt))?; m.add_wrapped(wrap_pyfunction!(starts_with))?; m.add_wrapped(wrap_pyfunction!(strpos))?; + //m.add_wrapped(wrap_pyfunction!(struct))?; m.add_wrapped(wrap_pyfunction!(substr))?; m.add_wrapped(wrap_pyfunction!(sum))?; m.add_wrapped(wrap_pyfunction!(tan))?; m.add_wrapped(wrap_pyfunction!(to_hex))?; m.add_wrapped(wrap_pyfunction!(to_timestamp))?; + m.add_wrapped(wrap_pyfunction!(to_timestamp_millis))?; + m.add_wrapped(wrap_pyfunction!(to_timestamp_micros))?; + m.add_wrapped(wrap_pyfunction!(to_timestamp_seconds))?; m.add_wrapped(wrap_pyfunction!(translate))?; m.add_wrapped(wrap_pyfunction!(trim))?; m.add_wrapped(wrap_pyfunction!(trunc))?; m.add_wrapped(wrap_pyfunction!(upper))?; + //m.add_wrapped(wrap_pyfunction!(uuid))?; m.add_wrapped(wrap_pyfunction!(window))?; Ok(()) } diff --git a/python/src/udaf.rs b/python/src/udaf.rs index 42c388ed8..e96925b91 100644 --- a/python/src/udaf.rs +++ b/python/src/udaf.rs @@ -25,7 +25,7 @@ use datafusion::arrow::pyarrow::{PyArrowConvert, PyArrowType}; use datafusion::common::ScalarValue; use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::{ - self, Accumulator, AccumulatorFunctionImplementation, AggregateState, AggregateUDF, + self, Accumulator, AccumulatorFunctionImplementation, AggregateUDF, }; use crate::expression::PyExpr; @@ -43,13 +43,9 @@ impl RustAccumulator { } impl Accumulator for RustAccumulator { - fn state(&self) -> Result> { - let py_result: PyResult> = - Python::with_gil(|py| self.accum.as_ref(py).call_method0("state")?.extract()); - match py_result { - Ok(r) => Ok(r.into_iter().map(AggregateState::Scalar).collect()), - Err(e) => Err(DataFusionError::Execution(format!("{}", e))), - } + fn state(&self) -> Result> { + Python::with_gil(|py| self.accum.as_ref(py).call_method0("state")?.extract()) + .map_err(|e| DataFusionError::Execution(format!("{e}"))) } fn evaluate(&self) -> Result { From 1a23bb6d91a8523aa78949abee54146e400572c9 Mon Sep 17 00:00:00 2001 From: yangzhong Date: Sun, 29 Jan 2023 18:53:12 +0800 Subject: [PATCH 3/4] Skip ut of test_window_lead due to https://github.com/apache/arrow-datafusion-python/issues/135 --- python/ballista/tests/test_dataframe.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/ballista/tests/test_dataframe.py b/python/ballista/tests/test_dataframe.py index f134199af..e8f2fe950 100644 --- a/python/ballista/tests/test_dataframe.py +++ b/python/ballista/tests/test_dataframe.py @@ -145,6 +145,9 @@ def test_join(): assert table.to_pydict() == expected +@pytest.mark.skip( + reason="https://github.com/apache/arrow-datafusion-python/issues/135" +) def test_window_lead(df): df = df.select( column("a"), From a7283521ce30f0a8d7918c54dbbb9f12bc544d6f Mon Sep 17 00:00:00 2001 From: yangzhong Date: Mon, 30 Jan 2023 01:06:14 +0800 Subject: [PATCH 4/4] Fix 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/src/client.rs | 13 ++- 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 | 23 ++--- 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 | 55 +++++------ 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 | 19 ++-- .../scheduler_server/query_stage_scheduler.rs | 6 +- .../scheduler/src/state/backend/cluster.rs | 37 +++---- 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 | 36 +++++-- benchmarks/src/bin/nyctaxi.rs | 8 +- benchmarks/src/bin/tpch.rs | 97 ++++++++----------- 45 files changed, 359 insertions(+), 424 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 c935c5f8e..3269592de 100644 --- a/ballista/client/src/context.rs +++ b/ballista/client/src/context.rs @@ -94,7 +94,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 @@ -111,7 +111,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; @@ -170,7 +170,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; @@ -260,7 +260,7 @@ impl BallistaContext { .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)) })? .into_optimized_plan()?; match plan { @@ -430,14 +430,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/src/client.rs b/ballista/core/src/client.rs index 460254c42..a9e616e81 100644 --- a/ballista/core/src/client.rs +++ b/ballista/core/src/client.rs @@ -56,16 +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 - )) + "Error connecting to Ballista scheduler or executor at {addr}: {e:?}" + )) })?; let flight_client = FlightServiceClient::new(connection); debug!("BallistaClient connected OK"); @@ -115,7 +114,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 +122,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 289b2492a..e22ae9d63 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 59e696746..88408fa38 100644 --- a/ballista/core/src/serde/mod.rs +++ b/ballista/core/src/serde/mod.rs @@ -64,7 +64,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()) } @@ -124,8 +124,7 @@ impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { let ballista_plan: protobuf::BallistaPhysicalPlanNode = protobuf::BallistaPhysicalPlanNode::decode(buf).map_err(|e| { DataFusionError::Internal(format!( - "Could not deserialize BallistaPhysicalPlanNode: {}", - e + "Could not deserialize BallistaPhysicalPlanNode: {e}", )) })?; @@ -165,8 +164,7 @@ impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { .map(|l| { l.clone().try_into().map_err(|e| { DataFusionError::Internal(format!( - "Fail to get partition location due to {:?}", - e + "Fail to get partition location due to {e:?}", )) }) }) @@ -212,8 +210,7 @@ impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { None => None, other => { return Err(DataFusionError::Internal(format!( - "physical_plan::to_proto() invalid partitioning for ShuffleWriterExec: {:?}", - other + "physical_plan::to_proto() invalid partitioning for ShuffleWriterExec: {other:?}", ))); } }; @@ -231,8 +228,7 @@ impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { proto.encode(buf).map_err(|e| { DataFusionError::Internal(format!( - "failed to encode shuffle writer execution plan: {:?}", - e + "failed to encode shuffle writer execution plan: {e:?}", )) })?; @@ -246,8 +242,7 @@ impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { .map(|l| { l.clone().try_into().map_err(|e| { DataFusionError::Internal(format!( - "Fail to get partition location due to {:?}", - e + "Fail to get partition location due to {e:?}", )) }) }) @@ -264,8 +259,7 @@ impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { }; proto.encode(buf).map_err(|e| { DataFusionError::Internal(format!( - "failed to encode shuffle reader execution plan: {:?}", - e + "failed to encode shuffle reader execution plan: {e:?}", )) })?; @@ -283,8 +277,7 @@ impl PhysicalExtensionCodec for BallistaPhysicalExtensionCodec { }; proto.encode(buf).map_err(|e| { DataFusionError::Internal(format!( - "failed to encode unresolved shuffle execution plan: {:?}", - e + "failed to encode unresolved shuffle execution plan: {e:?}", )) })?; diff --git a/ballista/core/src/serde/scheduler/to_proto.rs b/ballista/core/src/serde/scheduler/to_proto.rs index 3d7ee9464..c94be6a2b 100644 --- a/ballista/core/src/serde/scheduler/to_proto.rs +++ b/ballista/core/src/serde/scheduler/to_proto.rs @@ -113,8 +113,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 e11b4e552..9b1210047 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 17e4271a8..fc99eff01 100644 --- a/ballista/executor/src/execution_loop.rs +++ b/ballista/executor/src/execution_loop.rs @@ -165,8 +165,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 aee3d0cfa..a27355773 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..01a2eb7e0 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 b8d01d423..46187e9a6 100644 --- a/ballista/scheduler/src/flight_sql.rs +++ b/ballista/scheduler/src/flight_sql.rs @@ -130,7 +130,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 @@ -138,7 +138,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); @@ -152,7 +152,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) { @@ -161,13 +161,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}" )))? } } @@ -180,7 +179,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) } @@ -192,14 +191,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))? } @@ -207,7 +206,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))? } @@ -287,7 +286,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 }; @@ -316,7 +315,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 }; @@ -339,8 +338,7 @@ impl FlightSqlServiceImpl { Ok(plan.clone()) } else { Err(Status::internal(format!( - "Statement handle not found: {}", - handle + "Statement handle not found: {handle}" )))? } } @@ -363,7 +361,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) } @@ -373,13 +371,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) })?; @@ -502,8 +499,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()..]; @@ -529,7 +525,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) @@ -554,7 +550,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(), @@ -590,9 +586,8 @@ impl FlightSqlService for FlightSqlServiceImpl { .await .map_err(|e| { Status::internal(format!( - "Error connecting to Ballista scheduler or executor at {}: {:?}", - addr, e - )) + "Error connecting to Ballista scheduler or executor at {addr}: {e:?}" + )) })?; let mut flight_client = FlightServiceClient::new(connection); let buf = action.encode_to_vec(); @@ -601,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))) } @@ -629,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?; @@ -663,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) } @@ -675,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) } @@ -851,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 6c07c6477..87aaad775 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(( @@ -637,8 +636,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 91a161cc1..ecfa81a31 100644 --- a/ballista/scheduler/src/scheduler_server/grpc.rs +++ b/ballista/scheduler/src/scheduler_server/grpc.rs @@ -111,7 +111,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) })?; @@ -121,7 +121,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) })?; @@ -212,7 +212,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) })?; @@ -248,7 +248,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) })?; @@ -309,14 +309,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(&state, &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 b7468594f..3c9a651a9 100644 --- a/ballista/scheduler/src/scheduler_server/mod.rs +++ b/ballista/scheduler/src/scheduler_server/mod.rs @@ -216,8 +216,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..5a901cca4 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,11 @@ mod tests { false }; - assert!(received, "Did not receive heartbeat for executor {}", i); + assert!( + received, + "{}", + format!("Did not receive heartbeat for executor {i}") + ); } Ok(()) @@ -654,14 +651,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 +684,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 8105b0ac8..f4d8477d8 100644 --- a/ballista/scheduler/src/state/execution_graph.rs +++ b/ballista/scheduler/src/state/execution_graph.rs @@ -346,7 +346,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. @@ -421,8 +421,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); } @@ -571,8 +570,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}" ))); } } @@ -610,8 +608,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}" ))); } } @@ -636,8 +633,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}" ))); } } @@ -675,7 +671,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![]; @@ -756,14 +752,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}" ))); } } @@ -875,7 +869,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 { @@ -915,7 +909,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()?; @@ -1412,7 +1406,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 b49d7214e..f1181286a 100644 --- a/ballista/scheduler/src/state/execution_graph/execution_stage.rs +++ b/ballista/scheduler/src/state/execution_graph/execution_stage.rs @@ -954,7 +954,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 cf7a4b963..4edb717a6 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 { subgraph cluster0 { @@ -504,7 +500,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 { stage_3_0 [shape=box, label="ShuffleWriter [48 partitions]"] @@ -532,7 +528,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 { subgraph cluster0 { @@ -596,7 +592,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 { stage_4_0 [shape=box, label="ShuffleWriter [48 partitions]"] 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 1a5d50e4e..0ce591453 100644 --- a/ballista/scheduler/src/state/mod.rs +++ b/ballista/scheduler/src/state/mod.rs @@ -378,8 +378,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" ))) } } @@ -729,7 +728,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 c802c6f61..908b07c83 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,11 @@ 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, + "{}", + format!("Expected submitted event for job {job_id}") + ); } pub fn assert_no_submitted_event(job_id: &str, collector: &TestMetricsCollector) { @@ -690,7 +694,11 @@ 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, + "{}", + format!("Expected no submitted event for job {job_id}") + ); } pub fn assert_completed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -699,7 +707,11 @@ 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, + "{}", + format!("Expected completed event for job {job_id}") + ); } pub fn assert_cancelled_event(job_id: &str, collector: &TestMetricsCollector) { @@ -708,7 +720,11 @@ 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, + "{}", + format!("Expected cancelled event for job {job_id}") + ); } pub fn assert_failed_event(job_id: &str, collector: &TestMetricsCollector) { @@ -717,5 +733,9 @@ 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, + "{}", + format!("Expected failed event for job {job_id}") + ); } diff --git a/benchmarks/src/bin/nyctaxi.rs b/benchmarks/src/bin/nyctaxi.rs index d02b02cff..cd0cb994c 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?; @@ -123,7 +123,7 @@ async fn execute_sql(ctx: &SessionContext, sql: &str, debug: bool) -> Result<()> let plan = session_state.create_logical_plan(sql).await?; let plan = session_state.optimize(&plan)?; if debug { - println!("Optimized logical plan:\n{:?}", plan); + println!("Optimized logical plan:\n{plan:?}"); } let physical_plan = session_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 3798b14dc..e6f65708c 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -284,7 +284,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) @@ -305,7 +305,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() @@ -365,12 +365,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(); @@ -393,16 +393,16 @@ 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(); let plan = df.clone().into_optimized_plan()?; if opt.debug { - println!("=== Optimized logical plan ===\n{:?}\n", plan); + println!("=== Optimized logical plan ===\n{plan:?}\n"); } 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; @@ -451,10 +451,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( @@ -463,7 +460,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; @@ -477,7 +474,7 @@ async fn loadtest_ballista(opt: BallistaLoadtestOpt) -> Result<()> { &config, ) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?, + .map_err(|e| DataFusionError::Execution(format!("{e:?}")))?, ); } @@ -525,12 +522,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!( @@ -547,7 +544,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(()) } @@ -556,7 +553,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( @@ -584,13 +581,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")?; @@ -598,30 +594,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}'" ))) } } @@ -630,16 +623,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}" ))) } } @@ -648,8 +640,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 { @@ -662,12 +654,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( @@ -697,7 +688,7 @@ async fn create_logical_plans( .into_iter() .collect::>, tokio::task::JoinError>>( ) - .map_err(|e| DataFusionError::Internal(format!("{:?}", e)))? + .map_err(|e| DataFusionError::Internal(format!("{e:?}")))? .into_iter() .collect() } @@ -708,12 +699,12 @@ async fn execute_query( debug: bool, ) -> Result> { if debug { - println!("=== Logical plan ===\n{:?}\n", plan); + println!("=== Logical plan ===\n{plan:?}\n"); } let session_state = ctx.state(); let plan = session_state.optimize(plan)?; if debug { - println!("=== Optimized logical plan ===\n{:?}\n", plan); + println!("=== Optimized logical plan ===\n{plan:?}\n"); } let physical_plan = session_state.create_physical_plan(&plan).await?; if debug { @@ -785,8 +776,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}" ))) } }; @@ -797,8 +787,7 @@ async fn convert_tbl(opt: ConvertOpt) -> Result<()> { } other => { return Err(DataFusionError::NotImplemented(format!( - "Invalid output format: {}", - other + "Invalid output format: {other}" ))) } } @@ -819,7 +808,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'|') @@ -828,7 +817,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); @@ -836,7 +825,7 @@ async fn get_table( (Arc::new(format), path, DEFAULT_CSV_EXTENSION) } "parquet" => { - let path = format!("{}/{}", path, table); + let path = format!("{path}/{table}"); let format = ParquetFormat::default().with_enable_pruning(Some(true)); (Arc::new(format), path, DEFAULT_PARQUET_EXTENSION) @@ -1035,8 +1024,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) @@ -1616,8 +1605,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" ); @@ -1639,8 +1628,8 @@ mod tests { ) .unwrap(); assert_eq!( - format!("{:?}", physical_plan), - format!("{:?}", round_trip), + format!("{physical_plan:?}"), + format!("{round_trip:?}"), "physical plan round trip failed" ); }