From 4bae189c95356edd1636fb6b8be99e27019e4539 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Wed, 2 Jul 2025 14:45:01 -0400 Subject: [PATCH 1/6] WIP for explain analyze --- Cargo.toml | 10 +- scripts/launch_tpch_cluster.sh | 70 ++++++----- src/analyze.rs | 162 +++++++++++++++++++++++++ src/codec.rs | 54 ++++----- src/flight.rs | 29 ++--- src/flight_handlers.rs | 188 +++++++++++++++++------------ src/lib.rs | 15 ++- src/physical.rs | 27 ++++- src/planning.rs | 19 ++- src/processor_service.rs | 204 ++++++++++++++++++++++---------- src/proto/datafusion_ray.proto | 45 ++++--- src/proto/generated/protobuf.rs | 61 ++++++---- src/stage_reader.rs | 117 +++++++++++++++--- src/vocab.rs | 11 +- 14 files changed, 704 insertions(+), 308 deletions(-) create mode 100644 src/analyze.rs diff --git a/Cargo.toml b/Cargo.toml index 49846aa..9964a90 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -39,11 +39,8 @@ arrow-flight = { version = "55", features = ["flight-sql-experimental"] } async-stream = "0.3" bytes = "1.5" clap = { version = "4.4", features = ["derive"] } -datafusion = { git = "https://github.com/apache/datafusion", branch = "main", features = [ - "pyarrow", - "avro", -] } -datafusion-proto = { git = "https://github.com/apache/datafusion", branch = "main" } +datafusion = "48.0.0" +datafusion-proto = "48.0.0" env_logger = "0.11" futures = "0.3" itertools = "0.14" @@ -95,4 +92,5 @@ tonic-build = { version = "0.12", default-features = false, features = [ url = "2" [dev-dependencies] -tempfile = "3.20" \ No newline at end of file +tempfile = "3.20" + diff --git a/scripts/launch_tpch_cluster.sh b/scripts/launch_tpch_cluster.sh index f069e71..04c654f 100755 --- a/scripts/launch_tpch_cluster.sh +++ b/scripts/launch_tpch_cluster.sh @@ -38,20 +38,20 @@ DEFAULT_LOG_PATH="." # Parse named arguments for arg in "$@"; do case "$arg" in - num_workers=*) - NUM_WORKERS="${arg#*=}" - ;; - tpch_file_path=*) - TPCH_DATA_DIR="${arg#*=}" - ;; - log_file_path=*) - LOG_DIR="${arg#*=}" - ;; - *) - echo "Error: Unknown argument '$arg'" - echo "Usage: $0 [num_workers=N] [tpch_file_path=PATH] [log_file_path=PATH]" - exit 1 - ;; + num_workers=*) + NUM_WORKERS="${arg#*=}" + ;; + tpch_file_path=*) + TPCH_DATA_DIR="${arg#*=}" + ;; + log_file_path=*) + LOG_DIR="${arg#*=}" + ;; + *) + echo "Error: Unknown argument '$arg'" + echo "Usage: $0 [num_workers=N] [tpch_file_path=PATH] [log_file_path=PATH]" + exit 1 + ;; esac done @@ -71,13 +71,14 @@ if [ ! -f "./target/release/distributed-datafusion" ]; then echo "Binary not found, building release version..." echo "This may take a few minutes on first run..." if [ -f "./build.sh" ]; then - ./build.sh --release + ./build.sh #--release else - cargo build --release + cargo build #--release fi - + # Verify the build was successful - if [ ! -f "./target/release/distributed-datafusion" ]; then + #if [ ! -f "./target/release/distributed-datafusion" ]; then + if [ ! -f "./target/debug/distributed-datafusion" ]; then echo "Error: Failed to build distributed-datafusion binary" exit 1 fi @@ -98,8 +99,8 @@ if [ ! -d "$LOG_DIR" ]; then fi # Verify required parquet files exist -required_files=("customer.parquet" "lineitem.parquet" "nation.parquet" "orders.parquet" - "part.parquet" "partsupp.parquet" "region.parquet" "supplier.parquet") +required_files=("customer.parquet" "lineitem.parquet" "nation.parquet" "orders.parquet" + "part.parquet" "partsupp.parquet" "region.parquet" "supplier.parquet") for file in "${required_files[@]}"; do if [ ! -f "${TPCH_DATA_DIR}/${file}" ]; then @@ -141,12 +142,13 @@ trap cleanup SIGINT SIGTERM EXIT # Start workers echo "Starting workers..." -for ((i=0; i "$LOG_FILE" 2>&1 & + #env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" ./target/release/distributed-datafusion --mode worker --port $PORT >"$LOG_FILE" 2>&1 & + env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" ./target/debug/distributed-datafusion --mode worker --port $PORT >"$LOG_FILE" 2>&1 & WORKER_PIDS[$i]=$! WORKER_ADDRESSES[$i]="${WORKER_NAME}/localhost:${PORT}" done @@ -156,13 +158,17 @@ echo "Waiting for workers to initialize..." sleep 2 # Construct worker addresses string for proxy -WORKER_ADDRESSES_STR=$(IFS=,; echo "${WORKER_ADDRESSES[*]}") +WORKER_ADDRESSES_STR=$( + IFS=, + echo "${WORKER_ADDRESSES[*]}" +) # Start proxy echo "Starting proxy on port 20200..." echo "Connecting to workers: $WORKER_ADDRESSES_STR" PROXY_LOG="${LOG_DIR}/proxy.log" -env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" DFRAY_WORKER_ADDRESSES="$WORKER_ADDRESSES_STR" ./target/release/distributed-datafusion --mode proxy --port 20200 > "$PROXY_LOG" 2>&1 & +#env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" DFRAY_WORKER_ADDRESSES="$WORKER_ADDRESSES_STR" ./target/release/distributed-datafusion --mode proxy --port 20200 >"$PROXY_LOG" 2>&1 & +env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" DFRAY_WORKER_ADDRESSES="$WORKER_ADDRESSES_STR" ./target/debug/distributed-datafusion --mode proxy --port 20200 >"$PROXY_LOG" 2>&1 & PROXY_PID=$! echo @@ -170,23 +176,23 @@ echo "TPC-H cluster is now running!" echo echo "Cluster Information:" echo " Proxy: localhost:20200" -for ((i=0; i, +} + +impl DistributedAnalyzeExec { + pub fn new(input: Arc, verbose: bool, show_statistics: bool) -> Self { + Self { + input, + verbose, + show_statistics, + } + } + + pub fn annotated_plan(&self) -> String { + DisplayableExecutionPlan::with_metrics(self.input.as_ref()) + .set_show_statistics(self.show_statistics) + .indent(self.verbose) + .to_string() + } +} + +impl DisplayAs for DistributedAnalyzeExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!( + f, + "DistributedAnalyzeExec[verbose = {}, show stats = {}]", + self.verbose, self.show_statistics + ) + } +} + +impl ExecutionPlan for DistributedAnalyzeExec { + fn name(&self) -> &str { + "DistributedAnalyzeExec" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn properties(&self) -> &PlanProperties { + self.input.properties() + } + + fn children(&self) -> Vec<&std::sync::Arc> { + vec![&self.input] + } + + fn with_new_children( + self: std::sync::Arc, + children: Vec>, + ) -> Result> { + // TODO: generalize this + assert_eq!(children.len(), 1); + Ok(Arc::new(Self::new( + children[0].clone(), + self.verbose, + self.show_statistics, + ))) + } + + fn execute( + &self, + partition: usize, + context: std::sync::Arc, + ) -> Result { + self.input.execute(partition, context) + } +} + +#[derive(Debug)] +pub struct DistributedAnalyzeRootExec { + /// Control how much extra to print + pub(crate) verbose: bool, + /// If statistics should be displayed + pub(crate) show_statistics: bool, + /// The input plan (the plan being analyzed) + pub(crate) input: Arc, +} + +impl DistributedAnalyzeRootExec { + pub fn new(input: Arc, verbose: bool, show_statistics: bool) -> Self { + Self { + input, + verbose, + show_statistics, + } + } + + pub fn annotated_plan(&self) -> String { + DisplayableExecutionPlan::with_metrics(self.input.as_ref()) + .set_show_statistics(self.show_statistics) + .indent(self.verbose) + .to_string() + } +} + +impl DisplayAs for DistributedAnalyzeRootExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!( + f, + "DistributedAnalyzeExec[verbose = {}, show stats = {}]", + self.verbose, self.show_statistics + ) + } +} + +impl ExecutionPlan for DistributedAnalyzeRootExec { + fn name(&self) -> &str { + "DistributedAnalyzeExec" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn properties(&self) -> &PlanProperties { + self.input.properties() + } + + fn children(&self) -> Vec<&std::sync::Arc> { + vec![&self.input] + } + + fn with_new_children( + self: std::sync::Arc, + children: Vec>, + ) -> Result> { + // TODO: generalize this + assert_eq!(children.len(), 1); + Ok(Arc::new(Self::new( + children[0].clone(), + self.verbose, + self.show_statistics, + ))) + } + + fn execute( + &self, + partition: usize, + context: std::sync::Arc, + ) -> Result { + self.input.execute(partition, context) + } +} diff --git a/src/codec.rs b/src/codec.rs index 3171912..1dcd910 100644 --- a/src/codec.rs +++ b/src/codec.rs @@ -2,32 +2,28 @@ use std::sync::Arc; use arrow::datatypes::Schema; use datafusion::{ - common::{Result, internal_datafusion_err, internal_err}, + common::{internal_datafusion_err, internal_err, Result}, datasource::source::DataSourceExec, execution::FunctionRegistry, - physical_plan::{ExecutionPlan, displayable}, + physical_plan::{displayable, ExecutionPlan}, }; use datafusion_proto::{ physical_plan::{ - DefaultPhysicalExtensionCodec, - PhysicalExtensionCodec, - from_proto::parse_protobuf_partitioning, - to_proto::serialize_partitioning, + from_proto::parse_protobuf_partitioning, to_proto::serialize_partitioning, + DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, }, protobuf, }; use prost::Message; use crate::{ + analyze::DistributedAnalyzeExec, isolator::PartitionIsolatorExec, logging::trace, max_rows::MaxRowsExec, protobuf::{ - DfRayExecNode, - DfRayStageReaderExecNode, - MaxRowsExecNode, - PartitionIsolatorExecNode, - df_ray_exec_node::Payload, + df_ray_exec_node::Payload, DfRayExecNode, DfRayStageReaderExecNode, + DistributedAnalyzeExecNode, MaxRowsExecNode, PartitionIsolatorExecNode, }, stage_reader::DFRayStageReaderExec, }; @@ -95,15 +91,18 @@ impl PhysicalExtensionCodec for DFRayCodec { ))) } } - Payload::NumpangExec(_) => { - Err(internal_datafusion_err!( - "NumpangExec not supported in open source version" - )) - } - Payload::ContextExec(_) => { - Err(internal_datafusion_err!( - "ContextExec not supported in open source version" - )) + Payload::DistributedAnalyzeExec(distributed_analyze_exec_node) => { + if inputs.len() != 1 { + Err(internal_datafusion_err!( + "DistributedAnalyzeExec requires one input" + )) + } else { + Ok(Arc::new(DistributedAnalyzeExec::new( + inputs[0].clone(), + distributed_analyze_exec_node.verbose, + distributed_analyze_exec_node.show_statistics, + ))) + } } } } else { @@ -133,7 +132,6 @@ impl PhysicalExtensionCodec for DFRayCodec { Payload::StageReaderExec(pb) } else if let Some(pi) = node.as_any().downcast_ref::() { let pb = PartitionIsolatorExecNode { - dummy: 0.0, partition_count: pi.partition_count as u64, }; @@ -143,8 +141,12 @@ impl PhysicalExtensionCodec for DFRayCodec { max_rows: max.max_rows as u64, }; Payload::MaxRowsExec(pb) - } else if let Some(_exec) = node.as_any().downcast_ref::() { - return internal_err!("DataSourceExec encoding not supported in open source version"); + } else if let Some(exec) = node.as_any().downcast_ref::() { + let pb = DistributedAnalyzeExecNode { + verbose: exec.verbose, + show_statistics: exec.show_statistics, + }; + Payload::DistributedAnalyzeExec(pb) } else { return internal_err!("Not supported node to encode to proto"); }; @@ -169,16 +171,14 @@ mod test { use arrow::datatypes::DataType; use datafusion::{ - physical_plan::{Partitioning, displayable}, + physical_plan::{displayable, Partitioning}, prelude::SessionContext, }; use datafusion_proto::physical_plan::AsExecutionPlan; use super::*; use crate::{ - isolator::PartitionIsolatorExec, - max_rows::MaxRowsExec, - stage_reader::DFRayStageReaderExec, + isolator::PartitionIsolatorExec, max_rows::MaxRowsExec, stage_reader::DFRayStageReaderExec, }; fn create_test_schema() -> Arc { diff --git a/src/flight.rs b/src/flight.rs index 7dd9df7..62d959a 100644 --- a/src/flight.rs +++ b/src/flight.rs @@ -18,21 +18,9 @@ use std::sync::Arc; use arrow_flight::{ - Action, - ActionType, - Criteria, - Empty, - FlightData, - FlightDescriptor, - FlightInfo, - HandshakeRequest, - HandshakeResponse, - PollInfo, - PutResult, - SchemaResult, - Ticket, - flight_service_server::FlightService, - sql::server::FlightSqlService, + flight_service_server::FlightService, sql::server::FlightSqlService, Action, ActionType, + Criteria, Empty, FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse, + PollInfo, PutResult, SchemaResult, Ticket, }; use futures::stream::BoxStream; use tonic::{Request, Response, Status, Streaming}; @@ -46,7 +34,7 @@ pub type DoActionStream = BoxStream<'static, Result) -> Result, Status>; async fn do_action(&self, request: Request) - -> Result, Status>; + -> Result, Status>; } pub struct FlightServ { @@ -68,7 +56,8 @@ impl FlightService for FlightServ { request: Request, ) -> Result, Status> { self.handler.do_get(request).await.inspect_err(|e| { - error!("Error in do_get: {:?}", e); + error!("Error in do_get: {:#?}", e); + log::error!("LOG Error in do_get: {:#?}", e); }) } @@ -119,7 +108,7 @@ impl FlightService for FlightServ { request: Request, ) -> Result, Status> { self.handler.do_action(request).await.inspect_err(|e| { - error!("Error in do_action: {:?}", e); + error!("Error in do_action: {:#?}", e); }) } @@ -172,7 +161,7 @@ impl FlightSqlService for FlightSqlServ { .get_flight_info_statement(query, request) .await .inspect_err(|e| { - error!("Error in do_flight_info_statement: {:?}", e); + error!("Error in do_flight_info_statement: {:#?}", e); }) } async fn do_get_statement( @@ -184,7 +173,7 @@ impl FlightSqlService for FlightSqlServ { .do_get_statement(ticket, request) .await .inspect_err(|e| { - error!("Error in do_get_statement: {:?}", e); + error!("Error in do_get_statement: {:#?}", e); }) } } diff --git a/src/flight_handlers.rs b/src/flight_handlers.rs index 67c80bd..59f9bf1 100644 --- a/src/flight_handlers.rs +++ b/src/flight_handlers.rs @@ -2,15 +2,13 @@ use std::sync::Arc; use arrow::datatypes::{Schema, SchemaRef}; use arrow_flight::{ - FlightEndpoint, - FlightInfo, - Ticket, encode::FlightDataEncoderBuilder, error::FlightError, sql::{ProstMessageExt, TicketStatementQuery}, + FlightEndpoint, FlightInfo, Ticket, }; -use datafusion::{ - physical_plan::{ExecutionPlan, Partitioning, coalesce_partitions::CoalescePartitionsExec}, +use datafusion::physical_plan::{ + coalesce_partitions::CoalescePartitionsExec, ExecutionPlan, Partitioning, }; use futures::TryStreamExt; use prost::Message; @@ -18,10 +16,10 @@ use tonic::{Response, Status}; use crate::{ explain::DistributedExplainExec, + logging::{debug, trace}, + planning::{add_ctx_extentions, get_ctx}, protobuf::{DistributedExplainExecNode, TicketStatementData}, query_planner::QueryPlanner, - planning::{add_ctx_extentions, get_ctx}, - logging::{debug, trace}, result::Result, stage_reader::DFRayStageReaderExec, util::{display_plan_with_partition_counts, get_addrs}, @@ -55,9 +53,11 @@ impl FlightRequestHandler { query_id, stage_id: final_stage_id, stage_addrs: Some(final_addrs.into()), - schema: Some(schema.try_into().map_err(|e| { - Status::internal(format!("Could not convert schema {e:?}")) - })?), + schema: Some( + schema + .try_into() + .map_err(|e| Status::internal(format!("Could not convert schema {e:?}")))?, + ), explain_data, }; @@ -76,10 +76,13 @@ impl FlightRequestHandler { } /// Handle EXPLAIN query requests by preparing plans for visualization. - /// - /// EXPLAIN queries return comprehensive plan information including logical, physical, + /// + /// EXPLAIN queries return comprehensive plan information including logical, physical, /// distributed plan, and execution stages for analysis and debugging purposes. - pub async fn handle_explain_request(&self, query: &str) -> Result, Status> { + pub async fn handle_explain_request( + &self, + query: &str, + ) -> Result, Status> { let plans = self .planner .prepare_explain(query) @@ -88,14 +91,12 @@ impl FlightRequestHandler { debug!("get flight info: EXPLAIN query id {}", plans.query_id); - let explain_data = plans.explain_data.map(|data| { - DistributedExplainExecNode { - schema: data.schema().as_ref().try_into().ok(), - logical_plan: data.logical_plan().to_string(), - physical_plan: data.physical_plan().to_string(), - distributed_plan: data.distributed_plan().to_string(), - distributed_stages: data.distributed_stages().to_string(), - } + let explain_data = plans.explain_data.map(|data| DistributedExplainExecNode { + schema: data.schema().as_ref().try_into().ok(), + logical_plan: data.logical_plan().to_string(), + physical_plan: data.physical_plan().to_string(), + distributed_plan: data.distributed_plan().to_string(), + distributed_stages: data.distributed_stages().to_string(), }); let flight_info = self.create_flight_info_response( @@ -103,7 +104,7 @@ impl FlightRequestHandler { plans.worker_addresses, plans.final_stage_id, plans.schema, - explain_data + explain_data, )?; trace!("get_flight_info_statement done for EXPLAIN"); @@ -111,8 +112,8 @@ impl FlightRequestHandler { } /// Handle query requests by preparing execution plans and stages. - /// - /// Query focus on execution readiness, returning only the essential + /// + /// Query focus on execution readiness, returning only the essential /// metadata needed to execute the distributed query plan. pub async fn handle_query_request(&self, query: &str) -> Result, Status> { let query_plan = self @@ -128,7 +129,7 @@ impl FlightRequestHandler { query_plan.worker_addresses, query_plan.final_stage_id, query_plan.schema, - None // Regular queries don't have explain data + None, // Regular queries don't have explain data )?; trace!("get_flight_info_statement done"); @@ -136,16 +137,17 @@ impl FlightRequestHandler { } /// Handle execution of EXPLAIN statement queries. - /// + /// /// This function does not execute the plan but returns all plans we want to display to the user. pub async fn handle_explain_statement_execution( &self, tsd: TicketStatementData, remote_addr: &str, ) -> Result, Status> { - let explain_data = tsd.explain_data.as_ref() - .ok_or_else(|| Status::internal("No explain_data in TicketStatementData for EXPLAIN query"))?; - + let explain_data = tsd.explain_data.as_ref().ok_or_else(|| { + Status::internal("No explain_data in TicketStatementData for EXPLAIN query") + })?; + let schema: Schema = explain_data .schema .as_ref() @@ -171,14 +173,18 @@ impl FlightRequestHandler { // Create dummy addresses for EXPLAIN execution let mut dummy_addrs = std::collections::HashMap::new(); let mut partition_addrs = std::collections::HashMap::new(); - partition_addrs.insert(0u64, vec![("explain_local".to_string(), "local".to_string())]); + partition_addrs.insert( + 0u64, + vec![("explain_local".to_string(), "local".to_string())], + ); dummy_addrs.insert(0u64, partition_addrs); - self.execute_plan_and_build_stream(explain_plan, tsd.query_id, dummy_addrs).await + self.execute_plan_and_build_stream(explain_plan, tsd.query_id, dummy_addrs) + .await } /// Handle execution of regular statement queries - /// + /// /// This function executes the plan and returns the results to the client. pub async fn handle_regular_statement_execution( &self, @@ -206,9 +212,13 @@ impl FlightRequestHandler { // Validate that addrs contains exactly one stage self.validate_single_stage_addrs(&addrs, tsd.stage_id)?; - let stage_partition_addrs = addrs.get(&tsd.stage_id) - .ok_or_else(|| Status::internal(format!("No partition addresses found for stage_id {}", tsd.stage_id)))?; - + let stage_partition_addrs = addrs.get(&tsd.stage_id).ok_or_else(|| { + Status::internal(format!( + "No partition addresses found for stage_id {}", + tsd.stage_id + )) + })?; + let plan = Arc::new( DFRayStageReaderExec::try_new( Partitioning::UnknownPartitioning(stage_partition_addrs.len()), @@ -227,11 +237,16 @@ impl FlightRequestHandler { display_plan_with_partition_counts(&plan) ); - self.execute_plan_and_build_stream(plan, tsd.query_id, addrs).await + self.execute_plan_and_build_stream(plan, tsd.query_id, addrs) + .await } /// Validate that addresses contain exactly one stage with the expected stage_id - pub fn validate_single_stage_addrs(&self, addrs: &Addrs, expected_stage_id: u64) -> Result<(), Status> { + pub fn validate_single_stage_addrs( + &self, + addrs: &Addrs, + expected_stage_id: u64, + ) -> Result<(), Status> { if addrs.len() != 1 { return Err(Status::internal(format!( "Expected exactly one stage in addrs, got {}", @@ -248,7 +263,7 @@ impl FlightRequestHandler { } /// Execute a plan and build the response stream. - /// + /// /// This function handles the common execution logic for both regular queries (which return data) /// and EXPLAIN queries (which return plan information as text). pub async fn execute_plan_and_build_stream( @@ -287,14 +302,11 @@ impl FlightRequestHandler { #[cfg(test)] mod tests { use super::*; - use std::collections::HashMap; - use crate::{ - test_utils::explain_test_helpers::{ - create_explain_ticket_statement_data, - create_test_flight_handler, - verify_explain_stream_results - }, + use crate::test_utils::explain_test_helpers::{ + create_explain_ticket_statement_data, create_test_flight_handler, + verify_explain_stream_results, }; + use std::collections::HashMap; // ////////////////////////////////////////////////////////////// // Test helper functions @@ -304,15 +316,17 @@ mod tests { fn create_test_addrs() -> Addrs { let mut addrs = HashMap::new(); let mut stage_addrs = HashMap::new(); - stage_addrs.insert(1u64, vec![ - ("worker1".to_string(), "localhost:8001".to_string()), - ("worker2".to_string(), "localhost:8002".to_string()), - ]); + stage_addrs.insert( + 1u64, + vec![ + ("worker1".to_string(), "localhost:8001".to_string()), + ("worker2".to_string(), "localhost:8002".to_string()), + ], + ); addrs.insert(1u64, stage_addrs); addrs } - // ////////////////////////////////////////////////////////////// // Unit tests // ////////////////////////////////////////////////////////////// @@ -330,20 +344,26 @@ mod tests { #[test] fn test_validate_single_stage_addrs_multiple_stages() { let handler = create_test_flight_handler(); - + // Create addresses with multiple stages let mut addrs = HashMap::new(); let mut stage1_addrs = HashMap::new(); - stage1_addrs.insert(1u64, vec![("worker1".to_string(), "localhost:8001".to_string())]); + stage1_addrs.insert( + 1u64, + vec![("worker1".to_string(), "localhost:8001".to_string())], + ); let mut stage2_addrs = HashMap::new(); - stage2_addrs.insert(2u64, vec![("worker2".to_string(), "localhost:8002".to_string())]); + stage2_addrs.insert( + 2u64, + vec![("worker2".to_string(), "localhost:8002".to_string())], + ); addrs.insert(1u64, stage1_addrs); addrs.insert(2u64, stage2_addrs); // Should fail with multiple stages let result = handler.validate_single_stage_addrs(&addrs, 1); assert!(result.is_err()); - + if let Err(status) = result { assert!(status.message().contains("Expected exactly one stage")); } @@ -357,9 +377,11 @@ mod tests { // Should fail when looking for non-existent stage let result = handler.validate_single_stage_addrs(&addrs, 999); assert!(result.is_err()); - + if let Err(status) = result { - assert!(status.message().contains("No addresses found for stage_id 999")); + assert!(status + .message() + .contains("No addresses found for stage_id 999")); } } @@ -371,7 +393,7 @@ mod tests { // Should fail with empty addresses let result = handler.validate_single_stage_addrs(&addrs, 1); assert!(result.is_err()); - + if let Err(status) = result { assert!(status.message().contains("Expected exactly one stage")); } @@ -385,35 +407,38 @@ mod tests { async fn test_handle_explain_request() { let handler = create_test_flight_handler(); let query = "EXPLAIN SELECT 1 as test_col, 'hello' as text_col"; - + let result = handler.handle_explain_request(query).await; assert!(result.is_ok()); - + let response = result.unwrap(); let flight_info = response.into_inner(); - + // Verify FlightInfo structure assert!(!flight_info.schema.is_empty()); assert_eq!(flight_info.endpoint.len(), 1); assert!(flight_info.endpoint[0].ticket.is_some()); - + // Verify that ticket has content (encoded TicketStatementData) let ticket = flight_info.endpoint[0].ticket.as_ref().unwrap(); assert!(!ticket.ticket.is_empty()); - - println!("✓ FlightInfo created successfully with {} schema bytes and ticket with {} bytes", - flight_info.schema.len(), ticket.ticket.len()); + + println!( + "✓ FlightInfo created successfully with {} schema bytes and ticket with {} bytes", + flight_info.schema.len(), + ticket.ticket.len() + ); } #[tokio::test] async fn test_handle_explain_request_invalid_query() { let handler = create_test_flight_handler(); - + // Test with EXPLAIN ANALYZE (should fail) let query = "EXPLAIN ANALYZE SELECT 1"; let result = handler.handle_explain_request(query).await; assert!(result.is_err()); - + let error = result.unwrap_err(); assert_eq!(error.code(), tonic::Code::Internal); assert!(error.message().contains("Could not prepare EXPLAIN query")); @@ -422,21 +447,23 @@ mod tests { #[tokio::test] async fn test_handle_explain_statement_execution() { let handler = create_test_flight_handler(); - + // First prepare an EXPLAIN query to get the ticket data structure let query = "EXPLAIN SELECT 1 as test_col"; let plans = handler.planner.prepare_explain(query).await.unwrap(); - + // Create the TicketStatementData that would be sent to do_get_statement let tsd = create_explain_ticket_statement_data(plans); - + // Test the execution - let result = handler.handle_explain_statement_execution(tsd, "test_remote").await; + let result = handler + .handle_explain_statement_execution(tsd, "test_remote") + .await; assert!(result.is_ok()); - + let response = result.unwrap(); let stream = response.into_inner(); - + // Use shared verification function verify_explain_stream_results(stream).await; } @@ -444,7 +471,7 @@ mod tests { #[tokio::test] async fn test_handle_explain_statement_execution_missing_explain_data() { let handler = create_test_flight_handler(); - + // Create TicketStatementData without explain_data (should fail) let tsd = TicketStatementData { query_id: "test_query".to_string(), @@ -453,13 +480,18 @@ mod tests { schema: None, explain_data: None, }; - - let result = handler.handle_explain_statement_execution(tsd, "test_remote").await; + + let result = handler + .handle_explain_statement_execution(tsd, "test_remote") + .await; assert!(result.is_err()); - + if let Err(error) = result { assert_eq!(error.code(), tonic::Code::Internal); - assert!(error.message().contains("No explain_data in TicketStatementData")); + assert!(error + .message() + .contains("No explain_data in TicketStatementData")); } } -} \ No newline at end of file +} + diff --git a/src/lib.rs b/src/lib.rs index 3b6cd2f..0a9f6b3 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -22,6 +22,7 @@ use crate::logging::info; mod proto; pub use proto::generated::protobuf; +pub mod analyze; pub mod codec; pub mod explain; pub mod flight; @@ -72,14 +73,12 @@ fn setup_memory_logging() { #[cfg(not(target_env = "msvc"))] { let mut mem_tracker = MemTracker::new(); - std::thread::spawn(move || { - loop { - let dt = 0.5; - std::thread::sleep(std::time::Duration::from_millis((dt * 1000.0) as u64)); - mem_tracker.update(dt); - if mem_tracker.rate_above_reporting_threshold() { - info!("{}", mem_tracker.status()); - } + std::thread::spawn(move || loop { + let dt = 0.5; + std::thread::sleep(std::time::Duration::from_millis((dt * 1000.0) as u64)); + mem_tracker.update(dt); + if mem_tracker.rate_above_reporting_threshold() { + info!("{}", mem_tracker.status()); } }); } diff --git a/src/physical.rs b/src/physical.rs index 7764b04..6ecca14 100644 --- a/src/physical.rs +++ b/src/physical.rs @@ -22,14 +22,15 @@ use datafusion::{ error::Result, physical_optimizer::PhysicalOptimizerRule, physical_plan::{ - ExecutionPlan, - joins::NestedLoopJoinExec, - repartition::RepartitionExec, - sorts::sort::SortExec, + analyze::AnalyzeExec, joins::NestedLoopJoinExec, repartition::RepartitionExec, + sorts::sort::SortExec, ExecutionPlan, }, }; -use crate::{logging::info, stage::DFRayStageExec, util::display_plan_with_partition_counts}; +use crate::{ + analyze::DistributedAnalyzeExec, logging::info, stage::DFRayStageExec, + util::display_plan_with_partition_counts, +}; /// This optimizer rule walks up the physical plan tree /// and inserts RayStageExec nodes where appropriate to denote where we will @@ -67,6 +68,8 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { display_plan_with_partition_counts(&plan) ); + let maybe_analyze_plan = plan.as_any().downcast_ref::(); + let mut stage_counter = 0; let up = |plan: Arc| { @@ -74,6 +77,18 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { || plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some() { + let plan = if maybe_analyze_plan.is_some() { + let definitely_analyze_plan = maybe_analyze_plan.cloned().unwrap(); + Arc::new(DistributedAnalyzeExec::new( + plan.clone(), + definitely_analyze_plan.verbose(), + definitely_analyze_plan.show_statistics(), + )) as Arc + } else { + plan + }; + + // insert a stage marker here so we know where to break up the physical plan later let stage = Arc::new(DFRayStageExec::new(plan, stage_counter)); stage_counter += 1; Ok(Transformed::yes(stage as Arc)) @@ -82,7 +97,7 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { } }; - let plan = plan.transform_up(up)?.data; + let plan = plan.clone().transform_up(up)?.data; let final_plan = Arc::new(DFRayStageExec::new(plan, stage_counter)) as Arc; diff --git a/src/planning.rs b/src/planning.rs index 40d1265..13eed20 100644 --- a/src/planning.rs +++ b/src/planning.rs @@ -4,13 +4,13 @@ use std::{ sync::{Arc, LazyLock}, }; -use anyhow::{Context, anyhow}; +use anyhow::{anyhow, Context}; use arrow_flight::Action; use datafusion::{ common::tree_node::{Transformed, TreeNode}, config::ConfigOptions, datasource::{ - file_format::{FileFormat, csv::CsvFormat, json::JsonFormat, parquet::ParquetFormat}, + file_format::{csv::CsvFormat, json::JsonFormat, parquet::ParquetFormat, FileFormat}, listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}, }, error::DataFusionError, @@ -18,13 +18,9 @@ use datafusion::{ logical_expr::LogicalPlan, physical_optimizer::PhysicalOptimizerRule, physical_plan::{ - ExecutionPlan, + coalesce_batches::CoalesceBatchesExec, displayable, joins::NestedLoopJoinExec, + repartition::RepartitionExec, sorts::sort::SortExec, ExecutionPlan, ExecutionPlanProperties, - coalesce_batches::CoalesceBatchesExec, - displayable, - joins::NestedLoopJoinExec, - repartition::RepartitionExec, - sorts::sort::SortExec, }, prelude::{SQLOptions, SessionConfig, SessionContext}, }; @@ -42,7 +38,7 @@ use crate::{ stage::DFRayStageExec, stage_reader::{DFRayStageReaderExec, QueryId}, util::{display_plan_with_partition_counts, get_client, physical_plan_to_bytes, wait_for}, - vocab::{Addrs, CtxName, CtxPartitionGroup, CtxStageAddrs}, + vocab::{Addrs, CtxAnnotatedOutputs, CtxName, CtxPartitionGroup, CtxStageAddrs}, }; #[derive(Debug)] @@ -150,6 +146,7 @@ pub fn add_ctx_extentions( config.set_extension(Arc::new(CtxStageAddrs(stage_addrs))); config.set_extension(Arc::new(QueryId(query_id.to_owned()))); config.set_extension(Arc::new(CtxName(ctx_name.to_owned()))); + config.set_extension(Arc::new(CtxAnnotatedOutputs::default())); if let Some(pg) = partition_group { // this only matters if the plan includes an PartitionIsolatorExec, which looks @@ -419,7 +416,9 @@ async fn try_distribute_stages(stage_datas: &[StageData]) -> Result<()> { for stage_data in stage_datas { trace!( "Distributing stage_id {}, pg: {:?} to worker: {:?}", - stage_data.stage_id, stage_data.partition_group, stage_data.assigned_addr + stage_data.stage_id, + stage_data.partition_group, + stage_data.assigned_addr ); // populate its child stages diff --git a/src/processor_service.rs b/src/processor_service.rs index c447c1f..6c843ae 100644 --- a/src/processor_service.rs +++ b/src/processor_service.rs @@ -21,43 +21,40 @@ use std::{ time::{Duration, SystemTime}, }; -use anyhow::{Context, anyhow}; +use anyhow::{anyhow, Context}; use arrow::array::RecordBatch; use arrow_flight::{ - Action, - Ticket, - encode::FlightDataEncoderBuilder, - error::FlightError, - flight_service_server::FlightServiceServer, + encode::FlightDataEncoderBuilder, error::FlightError, + flight_service_server::FlightServiceServer, Action, FlightData, Ticket, }; use async_stream::stream; use datafusion::{ - physical_plan::{ExecutionPlan, ExecutionPlanProperties, displayable}, + physical_plan::{displayable, EmptyRecordBatchStream, ExecutionPlan, ExecutionPlanProperties}, prelude::SessionContext, }; -use futures::{Stream, TryStreamExt}; +use futures::{StreamExt, TryStreamExt}; use parking_lot::{Mutex, RwLock}; use prost::Message; use tokio::{ net::TcpListener, - sync::mpsc::{Receiver, Sender, channel}, + sync::mpsc::{channel, Receiver, Sender}, }; -use tonic::{Request, Response, Status, async_trait, transport::Server}; +use tonic::{async_trait, transport::Server, Request, Response, Status}; use crate::{ + analyze::DistributedAnalyzeExec, flight::{FlightHandler, FlightServ}, logging::{debug, error, info, trace}, planning::{add_ctx_extentions, get_ctx}, - protobuf::{FlightTicketData, StageData}, + protobuf::{ + AnnotatedTaskOutput, AnnotatedTaskOutputs, FlightDataMetadata, FlightTicketData, StageData, + }, result::{DFRayError, Result}, util::{ - bytes_to_physical_plan, - display_plan_with_partition_counts, - get_addrs, - register_object_store_for_paths_in_plan, - reporting_stream, + bytes_to_physical_plan, display_plan_with_partition_counts, get_addrs, + register_object_store_for_paths_in_plan, reporting_stream, }, - vocab::{Addrs, CtxName}, + vocab::{Addrs, CtxAnnotatedOutputs, CtxName}, }; #[derive(Eq, PartialEq, Hash, Clone, Debug)] @@ -231,12 +228,12 @@ impl DfRayProcessorHandler { Ok(ctx) } - fn make_stream( + fn get_ctx_and_plan( &self, query_id: &str, stage_id: u64, partition: u64, - ) -> Result> + Send + 'static, Status> { + ) -> Result<(SessionContext, Arc)> { let key = PlanKey { query_id: query_id.to_string(), stage_id, @@ -245,22 +242,10 @@ impl DfRayProcessorHandler { let (ctx, plan) = { let mut _guard = self.plans.write(); - let (plan_key, mut plan_vec) = _guard - .remove_entry(&key) - .ok_or_else(|| { - Status::internal(format!( - "{}, No plan found for plan key {:?}", - self.name, key, - )) - }) - .inspect_err(|e| { - error!( - "{}, No plan found for plan key {:?},{e:?} have keys {:?}", - self.name, - key, - _guard.keys().map(|k| format!("{k:?}")) - ); - })?; + let (plan_key, mut plan_vec) = _guard.remove_entry(&key).context(format!( + "{}, No plan found for plan key {:?}", + self.name, key, + ))?; trace!( "{} found {} plans for plan key {:?}", self.name, @@ -273,32 +258,123 @@ impl DfRayProcessorHandler { } (ctx, plan) }; - trace!( - "make_stream for plan {}", - displayable(plan.as_ref()).indent(true) - ); + Ok((ctx, plan)) + } - let task_ctx = ctx.task_ctx(); + /// we want to send any additional FlightDataMetadata that we discover while + /// executing the plan further downstream to consumers of our stream. + /// + /// We may discover a FlightDataMetadata as an additional payload incoming + /// on streming requests. The [`RayStageReader`] will look for these and add + /// them to an extension on the context. + /// + /// Our job is to send all record batches from our stream over our flight response + /// but when the stream is exhausted, we'll send an additional message containing the + /// metadata. + /// + /// The reason we have to do it at the end is that the metadata may include an annodated + /// plan with metrics which will only be available after the stream has been + /// fully consumed. - let ctx_name = task_ctx - .session_config() - .get_extension::() - .ok_or_else(|| { - Status::internal(format!("{}, CtxName not set in session config", self.name)) - })? - .0 - .clone(); + fn make_stream( + &self, + ctx: SessionContext, + plan: Arc, + stage_id: u64, + partition: u64, + ) -> Result { + let task_ctx = ctx.task_ctx(); + // the RecordBatchStream from our plan let stream = plan .execute(partition as usize, task_ctx) - .inspect_err(|e| error!("Could not get partition stream from plan {e:?}")) - .map(|s| reporting_stream(&format!("{ctx_name} s:{stage_id} p:{partition}"), s)) - .map_err(|e| Status::internal(format!("Could not get partition stream from plan {e}")))? + .inspect_err(|e| error!("Could not get partition stream from plan {e:#?}"))? .map_err(|e| FlightError::from_external_error(Box::new(e))); info!("{} plans held {}", self.name, self.plans.read().len()); - Ok(stream) + let mut flight_data_stream = FlightDataEncoderBuilder::new().build(stream); + let name = self.name.clone(); + + #[allow(unused_assignments)] // clippy can't understand our assignment to done in the macro + let out_stream = async_stream::stream! { + let mut done = false; + while !done { + + match (done, flight_data_stream.next().await) { + (false, None) => { + // no more data so now we yield our additional FlightDataMetadata if required + debug!("stream exhausted, yielding FlightDataMetadata"); + let task_outputs = ctx.state().config() + .get_extension::() + .unwrap_or(Arc::new(CtxAnnotatedOutputs::default())) + .0 + .clone(); + + + if let Some(analyze) = plan.as_any().downcast_ref::() { + let annotated_plan = analyze.annotated_plan(); + debug!("sending annotated plan: {}", annotated_plan); + + let output = AnnotatedTaskOutput { + plan: annotated_plan, + host: None, + stage_id, + partition_group: vec![partition], + }; + task_outputs.lock().push(output); + } + + let meta = FlightDataMetadata { + annotated_task_outputs: Some(AnnotatedTaskOutputs { + outputs: task_outputs.lock().clone(), + }), + }; + + + let fake_batch = RecordBatch::new_empty(plan.schema()); + + + let mut fde = FlightDataEncoderBuilder::new() + //.with_schema(plan.schema()) + .with_metadata(meta.encode_to_vec().into()).build(futures::stream::once(async {Ok(fake_batch)})).map_err(|e| { + FlightError::from_external_error(Box::new(e)) + }); + + let flight_data = fde + .next() + .await + .ok_or_else(|| { + Status::internal(format!( + "{}, No FlightDataMetadata from our fde", + name + )) + })??; + + yield Ok(flight_data); + done = true; + + }, + (false, Some(Err(e))) => { + yield Err(Status::internal(format!( + "Unexpected error getting flight data stream: {e:?}", + ))); + done = true; + }, + (false, Some(Ok(flight_data))) => { + // we have a flight data, so we yield it + // decode this data for output + yield Ok(flight_data); + }, + (true, _) => { + // we are done, so we don't yield anything + debug!("{} stream done for stage {} partition {}", name, stage_id, partition); + }, + } + } + }; + + Ok(Box::pin(out_stream)) } } @@ -334,19 +410,25 @@ impl FlightHandler for DfRayProcessorHandler { ); let name = self.name.clone(); - let stream = self - .make_stream(&ftd.query_id, ftd.stage_id, ftd.partition) + let (ctx, plan) = self + .get_ctx_and_plan(&ftd.query_id, ftd.stage_id, ftd.partition) .map_err(|e| { - Status::internal(format!("{name} Unexpected error making stream {e:?}")) + Status::internal(format!( + "{name} Could not find plan for query_id {} stage {} partition {}: {e:?}", + ftd.query_id, ftd.stage_id, ftd.partition + )) })?; - let out_stream = FlightDataEncoderBuilder::new() - .build(stream) - .map_err(move |e| { - Status::internal(format!("{name} Unexpected error building stream {e:?}")) - }); + let do_get_stream = self + .make_stream(ctx, plan, ftd.stage_id, ftd.partition) + .map_err(|e| { + Status::internal(format!( + "{name} Could not make stream for query_id {} stage {} partition {}: {e:?}", + ftd.query_id, ftd.stage_id, ftd.partition + )) + })?; - Ok(Response::new(Box::pin(out_stream))) + Ok(Response::new(do_get_stream)) } async fn do_action( diff --git a/src/proto/datafusion_ray.proto b/src/proto/datafusion_ray.proto index bf4e243..ba237dc 100644 --- a/src/proto/datafusion_ray.proto +++ b/src/proto/datafusion_ray.proto @@ -18,30 +18,22 @@ message DFRayStageReaderExecNode { uint64 stage_id = 3; } -// the simplicity of the decoder in src/codec.rs currently requires a different byte -// representation per message. Hence the dummy fields. -// -// I'll come back to this and sort it out. Its not super critical as the plans are -// only exchanged at the start of queries, not during execution. - message MaxRowsExecNode { uint64 max_rows = 1; } message PartitionIsolatorExecNode { - float dummy = 1; uint64 partition_count = 2; } -message NumpangScanExecNode { - datafusion.FileScanExecConf base_conf = 1; +message DistributedAnalyzeExecNode { + // how much data to show + bool verbose = 1; + // if statistics should be displayed + bool show_statistics = 2; } -message ContextScanExecNode { - datafusion.FileScanExecConf base_conf = 1; - repeated datafusion.LogicalExprNode filters = 2; - datafusion_common.Schema schema = 3; -} +message DistributedAnalyzeRootExecNode {} message DistributedExplainExecNode { datafusion_common.Schema schema = 1; @@ -56,11 +48,30 @@ message DFRayExecNode { DFRayStageReaderExecNode stage_reader_exec = 1; MaxRowsExecNode max_rows_exec = 2; PartitionIsolatorExecNode isolator_exec = 3; - NumpangScanExecNode numpang_exec = 4; - ContextScanExecNode context_exec = 5; + DistributedAnalyzeExecNode distributed_analyze_exec = 4; + //DistributedAnalyzeRootExecNode distributed_analyze_root_exec = 5; } } +message AnnotatedTaskOutput { + // the output of the explain analyze + string plan = 1; + // the host who executed this stage + Host host = 2; + // the stage id that was executed + uint64 stage_id = 3; + // the partitions that were executed by this stage + repeated uint64 partition_group = 4; +} + +message AnnotatedTaskOutputs { + repeated AnnotatedTaskOutput outputs = 1; +} + +message FlightDataMetadata { + AnnotatedTaskOutputs annotated_task_outputs = 1; +} + message FlightTicketData { string query_id = 1; uint64 stage_id = 2; @@ -79,7 +90,7 @@ message TicketStatementData { // the schema of the final stage datafusion_common.Schema schema = 4; // For EXPLAIN queries, store the explain plan data directly - optional DistributedExplainExecNode explain_data = 5; + DistributedExplainExecNode explain_data = 5; } message Host { diff --git a/src/proto/generated/protobuf.rs b/src/proto/generated/protobuf.rs index 270653a..f6b5e0e 100644 --- a/src/proto/generated/protobuf.rs +++ b/src/proto/generated/protobuf.rs @@ -18,29 +18,20 @@ pub struct MaxRowsExecNode { } #[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct PartitionIsolatorExecNode { - #[prost(float, tag = "1")] - pub dummy: f32, #[prost(uint64, tag = "2")] pub partition_count: u64, } -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct NumpangScanExecNode { - #[prost(message, optional, tag = "1")] - pub base_conf: ::core::option::Option< - ::datafusion_proto::protobuf::FileScanExecConf, - >, -} -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ContextScanExecNode { - #[prost(message, optional, tag = "1")] - pub base_conf: ::core::option::Option< - ::datafusion_proto::protobuf::FileScanExecConf, - >, - #[prost(message, repeated, tag = "2")] - pub filters: ::prost::alloc::vec::Vec<::datafusion_proto::protobuf::LogicalExprNode>, - #[prost(message, optional, tag = "3")] - pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, +#[derive(Clone, Copy, PartialEq, ::prost::Message)] +pub struct DistributedAnalyzeExecNode { + /// how much data to show + #[prost(bool, tag = "1")] + pub verbose: bool, + /// if statistics should be displayed + #[prost(bool, tag = "2")] + pub show_statistics: bool, } +#[derive(Clone, Copy, PartialEq, ::prost::Message)] +pub struct DistributedAnalyzeRootExecNode {} #[derive(Clone, PartialEq, ::prost::Message)] pub struct DistributedExplainExecNode { #[prost(message, optional, tag = "1")] @@ -56,7 +47,7 @@ pub struct DistributedExplainExecNode { } #[derive(Clone, PartialEq, ::prost::Message)] pub struct DfRayExecNode { - #[prost(oneof = "df_ray_exec_node::Payload", tags = "1, 2, 3, 4, 5")] + #[prost(oneof = "df_ray_exec_node::Payload", tags = "1, 2, 3, 4")] pub payload: ::core::option::Option, } /// Nested message and enum types in `DFRayExecNode`. @@ -69,13 +60,37 @@ pub mod df_ray_exec_node { MaxRowsExec(super::MaxRowsExecNode), #[prost(message, tag = "3")] IsolatorExec(super::PartitionIsolatorExecNode), + /// DistributedAnalyzeRootExecNode distributed_analyze_root_exec = 5; #[prost(message, tag = "4")] - NumpangExec(super::NumpangScanExecNode), - #[prost(message, tag = "5")] - ContextExec(super::ContextScanExecNode), + DistributedAnalyzeExec(super::DistributedAnalyzeExecNode), } } #[derive(Clone, PartialEq, ::prost::Message)] +pub struct AnnotatedTaskOutput { + /// the output of the explain analyze + #[prost(string, tag = "1")] + pub plan: ::prost::alloc::string::String, + /// the host who executed this stage + #[prost(message, optional, tag = "2")] + pub host: ::core::option::Option, + /// the stage id that was executed + #[prost(uint64, tag = "3")] + pub stage_id: u64, + /// the partitions that were executed by this stage + #[prost(uint64, repeated, tag = "4")] + pub partition_group: ::prost::alloc::vec::Vec, +} +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AnnotatedTaskOutputs { + #[prost(message, repeated, tag = "1")] + pub outputs: ::prost::alloc::vec::Vec, +} +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlightDataMetadata { + #[prost(message, optional, tag = "1")] + pub annotated_task_outputs: ::core::option::Option, +} +#[derive(Clone, PartialEq, ::prost::Message)] pub struct FlightTicketData { #[prost(string, tag = "1")] pub query_id: ::prost::alloc::string::String, diff --git a/src/stage_reader.rs b/src/stage_reader.rs index b8118b8..59cd948 100644 --- a/src/stage_reader.rs +++ b/src/stage_reader.rs @@ -1,6 +1,10 @@ use std::{fmt::Formatter, sync::Arc}; -use arrow_flight::Ticket; +use arrow_flight::{ + decode::{DecodedPayload, FlightRecordBatchStream}, + error::FlightError, + Ticket, +}; use datafusion::{ arrow::datatypes::SchemaRef, common::{internal_datafusion_err, internal_err}, @@ -8,23 +12,20 @@ use datafusion::{ execution::SendableRecordBatchStream, physical_expr::EquivalenceProperties, physical_plan::{ - DisplayAs, - DisplayFormatType, - ExecutionPlan, - Partitioning, - PlanProperties, execution_plan::{Boundedness, EmissionType}, stream::RecordBatchStreamAdapter, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, }, + prelude::{SessionConfig, SessionContext}, }; -use futures::{StreamExt, stream::TryStreamExt}; +use futures::{stream::TryStreamExt, StreamExt}; use prost::Message; use crate::{ - logging::trace, - protobuf::FlightTicketData, - util::{CombinedRecordBatchStream, get_client}, - vocab::{CtxName, CtxStageAddrs}, + logging::{error, trace}, + protobuf::{FlightDataMetadata, FlightTicketData}, + util::{get_client, CombinedRecordBatchStream}, + vocab::{CtxAnnotatedOutputs, CtxName, CtxStageAddrs}, }; pub(crate) struct QueryId(pub String); @@ -170,6 +171,7 @@ impl ExecutionPlan for DFRayStageReaderExec { let schema = self.schema.clone(); let num_clients = clients.len(); + let ctx_name_capture = ctx_name.clone(); let stream = async_stream::stream! { let mut error = false; @@ -177,19 +179,22 @@ impl ExecutionPlan for DFRayStageReaderExec { let mut streams = vec![]; for (i, mut client) in clients.into_iter().enumerate() { let name = name.clone(); - trace!("{name} Getting flight stream {}/{}",i+1, num_clients); + trace!("{name} - {ctx_name_capture} Getting flight stream {}/{}",i+1, num_clients); match client.do_get(ticket.clone()).await { Ok(flight_stream) => { - trace!("{name} Got flight stream. headers:{:?}", flight_stream.headers()); - let rbr_stream = RecordBatchStreamAdapter::new(schema.clone(), - flight_stream - .map_err(move |e| internal_datafusion_err!("{name} Error consuming flight stream from {}: {e}", client.destination))); + trace!("{name} - {ctx_name_capture} Got flight stream. headers:{:?}", flight_stream.headers()); + let rbr_stream = make_flight_metadata_saver_stream( + ctx_name_capture.clone(), + context.session_config(), + schema.clone(), + flight_stream, + ); - streams.push(Box::pin(rbr_stream) as SendableRecordBatchStream); + streams.push(rbr_stream); }, Err(e) => { error = true; - yield internal_err!("{name} Error getting flight stream from {}: {e}", client.destination); + yield internal_err!("{name} - {ctx_name_capture} Error getting flight stream from {}: {e}", client.destination); } } } @@ -209,3 +214,79 @@ impl ExecutionPlan for DFRayStageReaderExec { ))) } } + +fn make_flight_metadata_saver_stream( + name: String, + config: &SessionConfig, + schema: SchemaRef, + stream: FlightRecordBatchStream, +) -> SendableRecordBatchStream { + let mut decoder = stream.into_inner(); + + let task_outputs = config + .get_extension::() + .unwrap_or(Arc::new(CtxAnnotatedOutputs::default())) + .0 + .clone(); + + let name_capture = name.clone(); + + #[allow(unused_assignments)] // clippy can't understand our assignment to done in the macro + let new_stream = async_stream::stream! { + let mut done = false; + while !done { + match (done, decoder.next().await) { + (false, Some(Ok(flight_data))) => { + let app_metadata_bytes = flight_data.app_metadata(); + + if !app_metadata_bytes.is_empty() { + trace!("{name} Received trailing metadata from flight stream"); + // decode the metadata + match FlightDataMetadata::decode(&app_metadata_bytes[..]) + .map(|fdm| fdm.annotated_task_outputs.unwrap_or_default()) { + Ok(outputs) => { + trace!("{name} Decoded flight data metadata annotated outputs: {:?}", outputs); + task_outputs.lock().extend(outputs.outputs); + } + Err(e) => { + yield Err(FlightError::DecodeError(format!( + "{name} Failed to decode flight data metadata: {e:#?}" + ))); + done = true; + } + } + + // ok we consumed the last flight message and extracted the trailing + // metadata, we don't want to yield this payload as there are no records in + // it + done = true; + } else { + yield Ok(flight_data.inner); + } + }, + (false, Some(Err(e))) => { + yield Err(e); + } + (false, None) => { + // we should not reach this block, because we decide we have reach the end when + // we receive a valid message with trailing data + error!("{name} Unexpected arrival in block we do not expect!"); + done = true; + } + (true,_) => { + // already done, ignore any further data + trace!("{name} Ignoring further data from flight stream, already done"); + } + } + } + }; + + let trailing_stream = + FlightRecordBatchStream::new_from_flight_data(new_stream).map_err(move |e| { + internal_datafusion_err!( + "{name_capture} Failed to consume from our flight data stream: {e:#?}" + ) + }); + + Box::pin(RecordBatchStreamAdapter::new(schema, trailing_stream)) +} diff --git a/src/vocab.rs b/src/vocab.rs index 1879fdb..969332f 100644 --- a/src/vocab.rs +++ b/src/vocab.rs @@ -1,4 +1,8 @@ -use std::collections::HashMap; +use std::{collections::HashMap, sync::Arc}; + +use parking_lot::Mutex; + +use crate::protobuf::AnnotatedTaskOutput; /// a map of stage_id, partition to a list (name,endpoint address) that can /// serve this (stage_id, and partition). It is assumed that to consume a @@ -15,4 +19,7 @@ pub(crate) struct CtxStageAddrs(pub Addrs); pub(crate) struct CtxName(pub String); /// used to hold a partition group as an extension for datafusion SessionContext -pub struct CtxPartitionGroup(pub Vec); +pub(crate) struct CtxPartitionGroup(pub Vec); + +#[derive(Default)] +pub(crate) struct CtxAnnotatedOutputs(pub Arc>>); From 7dc07772853df791335831a4ee3db7a7c63a0e15 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Tue, 8 Jul 2025 11:30:50 -0400 Subject: [PATCH 2/6] refactor to change all (String,String) to Host, explain analyze --- build.rs | 8 +- scripts/launch_tpch_cluster.sh | 4 +- src/analyze.rs | 129 ++++++++++++++-- src/codec.rs | 24 ++- src/flight_handlers.rs | 35 +++-- src/friendly.rs | 6 +- src/k8s.rs | 254 -------------------------------- src/lib.rs | 2 +- src/main.rs | 9 +- src/physical.rs | 102 ++++++++++++- src/planning.rs | 95 +++++------- src/processor_service.rs | 191 ++++++++++++++---------- src/proto/datafusion_ray.proto | 132 ----------------- src/proto/generated/protobuf.rs | 16 +- src/proto/mod.rs | 7 +- src/proxy_service.rs | 118 +++++++++------ src/query_planner.rs | 6 +- src/result.rs | 8 +- src/stage_reader.rs | 11 +- src/util.rs | 80 +++++----- src/vocab.rs | 17 ++- 21 files changed, 578 insertions(+), 676 deletions(-) delete mode 100644 src/k8s.rs delete mode 100644 src/proto/datafusion_ray.proto diff --git a/build.rs b/build.rs index a82e403..87dc74f 100644 --- a/build.rs +++ b/build.rs @@ -32,16 +32,16 @@ fn main() -> Result<(), String> { // We don't include the proto files in releases so that downstreams // do not need to have PROTOC included - if Path::new("src/proto/datafusion_ray.proto").exists() { + if Path::new("src/proto/distributed_datafusion.proto").exists() { println!("cargo:rerun-if-changed=src/proto/datafusion_common.proto"); println!("cargo:rerun-if-changed=src/proto/datafusion.proto"); - println!("cargo:rerun-if-changed=src/proto/datafusion_ray.proto"); + println!("cargo:rerun-if-changed=src/proto/distributed_datafusion.proto"); tonic_build::configure() .extern_path(".datafusion", "::datafusion_proto::protobuf") .extern_path(".datafusion_common", "::datafusion_proto::protobuf") - .compile_protos(&["src/proto/datafusion_ray.proto"], &["src/proto"]) + .compile_protos(&["src/proto/distributed_datafusion.proto"], &["src/proto"]) .map_err(|e| format!("protobuf compilation failed: {e}"))?; - let generated_source_path = out.join("datafusion_ray.protobuf.rs"); + let generated_source_path = out.join("distributed_datafusion.protobuf.rs"); let code = std::fs::read_to_string(generated_source_path).unwrap(); let mut file = std::fs::OpenOptions::new() .write(true) diff --git a/scripts/launch_tpch_cluster.sh b/scripts/launch_tpch_cluster.sh index 04c654f..5cbf662 100755 --- a/scripts/launch_tpch_cluster.sh +++ b/scripts/launch_tpch_cluster.sh @@ -148,7 +148,7 @@ for ((i = 0; i < NUM_WORKERS; i++)); do LOG_FILE="${LOG_DIR}/${WORKER_NAME}.log" echo " Starting $WORKER_NAME on port $PORT..." #env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" ./target/release/distributed-datafusion --mode worker --port $PORT >"$LOG_FILE" 2>&1 & - env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" ./target/debug/distributed-datafusion --mode worker --port $PORT >"$LOG_FILE" 2>&1 & + env RUST_BACKTRACE=1 DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" ./target/debug/distributed-datafusion --mode worker --port $PORT >"$LOG_FILE" 2>&1 & WORKER_PIDS[$i]=$! WORKER_ADDRESSES[$i]="${WORKER_NAME}/localhost:${PORT}" done @@ -168,7 +168,7 @@ echo "Starting proxy on port 20200..." echo "Connecting to workers: $WORKER_ADDRESSES_STR" PROXY_LOG="${LOG_DIR}/proxy.log" #env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" DFRAY_WORKER_ADDRESSES="$WORKER_ADDRESSES_STR" ./target/release/distributed-datafusion --mode proxy --port 20200 >"$PROXY_LOG" 2>&1 & -env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" DFRAY_WORKER_ADDRESSES="$WORKER_ADDRESSES_STR" ./target/debug/distributed-datafusion --mode proxy --port 20200 >"$PROXY_LOG" 2>&1 & +env RUST_BACKTRACE=1 DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" DFRAY_WORKER_ADDRESSES="$WORKER_ADDRESSES_STR" ./target/debug/distributed-datafusion --mode proxy --port 20200 >"$PROXY_LOG" 2>&1 & PROXY_PID=$! echo diff --git a/src/analyze.rs b/src/analyze.rs index a65b401..77438ce 100644 --- a/src/analyze.rs +++ b/src/analyze.rs @@ -1,13 +1,24 @@ use std::{fmt::Formatter, sync::Arc}; +use arrow::{ + array::{RecordBatch, StringBuilder}, + datatypes::{DataType, Field, Schema, SchemaRef}, +}; use datafusion::{ - error::Result, + error::{DataFusionError, Result}, execution::SendableRecordBatchStream, + physical_expr::EquivalenceProperties, physical_plan::{ - display::DisplayableExecutionPlan, DisplayAs, DisplayFormatType, ExecutionPlan, + display::DisplayableExecutionPlan, + execution_plan::{Boundedness, EmissionType}, + stream::RecordBatchStreamAdapter, + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, PlanProperties, }, }; +use futures::StreamExt; + +use crate::{logging::debug, protobuf::AnnotatedTaskOutput, vocab::CtxAnnotatedOutputs}; #[derive(Debug)] pub struct DistributedAnalyzeExec { @@ -93,30 +104,37 @@ pub struct DistributedAnalyzeRootExec { pub(crate) show_statistics: bool, /// The input plan (the plan being analyzed) pub(crate) input: Arc, + /// our plan properties + properties: PlanProperties, } impl DistributedAnalyzeRootExec { pub fn new(input: Arc, verbose: bool, show_statistics: bool) -> Self { + let field_a = Field::new("Task", DataType::Utf8, false); + let field_b = Field::new("Plan", DataType::Utf8, false); + let schema = Arc::new(Schema::new(vec![field_a, field_b])); + + let properties = PlanProperties::new( + EquivalenceProperties::new(schema.clone()), + Partitioning::UnknownPartitioning(1), + EmissionType::Final, + Boundedness::Bounded, + ); + Self { input, verbose, show_statistics, + properties, } } - - pub fn annotated_plan(&self) -> String { - DisplayableExecutionPlan::with_metrics(self.input.as_ref()) - .set_show_statistics(self.show_statistics) - .indent(self.verbose) - .to_string() - } } impl DisplayAs for DistributedAnalyzeRootExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { write!( f, - "DistributedAnalyzeExec[verbose = {}, show stats = {}]", + "DistributedAnalyzeRootExec[verbose = {}, show stats = {}]", self.verbose, self.show_statistics ) } @@ -124,7 +142,7 @@ impl DisplayAs for DistributedAnalyzeRootExec { impl ExecutionPlan for DistributedAnalyzeRootExec { fn name(&self) -> &str { - "DistributedAnalyzeExec" + "DistributedAnalyzeRootExec" } fn as_any(&self) -> &dyn std::any::Any { @@ -132,7 +150,7 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { } fn properties(&self) -> &PlanProperties { - self.input.properties() + &self.properties } fn children(&self) -> Vec<&std::sync::Arc> { @@ -157,6 +175,91 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { partition: usize, context: std::sync::Arc, ) -> Result { - self.input.execute(partition, context) + let input_capture = self.input.clone(); + let show_statistics_capture = self.show_statistics; + let verbose_capture = self.verbose; + let fmt_plan = move || -> String { + DisplayableExecutionPlan::with_metrics(input_capture.as_ref()) + .set_show_statistics(show_statistics_capture) + .indent(verbose_capture) + .to_string() + }; + + let task_outputs = context + .session_config() + .get_extension::() + .unwrap_or(Arc::new(CtxAnnotatedOutputs::default())) + .0 + .clone(); + + assert!( + partition == 0, + "DistributedAnalyzeRootExec expects only partition 0" + ); + + let mut input_stream = self.input.execute(partition, context)?; + + let schema_clone = self.schema().clone(); + + let output = async move { + // consume input, and we do not have to send it downstream as we are the + // root of the distributed analyze so we can discard the results just like + // regular AnalyzeExec + let mut done = false; + let mut total_rows = 0; + while !done { + match input_stream.next().await.transpose() { + Ok(Some(batch)) => { + // we consume the batch, yum. + debug!("consumed {} ", batch.num_rows()); + total_rows += batch.num_rows(); + } + Ok(None) => done = true, + Err(e) => { + return Err(e); + } + } + } + let annotated_plan = fmt_plan(); + let toutput = AnnotatedTaskOutput { + plan: annotated_plan, + host: None, + stage_id: 0, + partition_group: vec![0], + }; + + let mut tasks = task_outputs.lock(); + tasks.push(toutput); + + let mut task_builder = StringBuilder::with_capacity(1, 1024); + let mut plan_builder = StringBuilder::with_capacity(1, 1024); + task_builder.append_value("Task"); + plan_builder.append_value("Plan with Metrics"); + + for task_output in tasks.iter() { + task_builder.append_value(format!( + "Task: Stage {}, Partitions {:?}", + task_output.stage_id, task_output.partition_group + )); + plan_builder.append_value(&task_output.plan); + } + + RecordBatch::try_new( + schema_clone, + vec![ + Arc::new(task_builder.finish()), + Arc::new(plan_builder.finish()), + ], + ) + .map_err(DataFusionError::from) + .inspect(|batch| { + debug!("returning record batch {:?}", batch); + }) + }; + + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(output), + ))) } } diff --git a/src/codec.rs b/src/codec.rs index 1dcd910..2b732a8 100644 --- a/src/codec.rs +++ b/src/codec.rs @@ -17,13 +17,14 @@ use datafusion_proto::{ use prost::Message; use crate::{ - analyze::DistributedAnalyzeExec, + analyze::{DistributedAnalyzeExec, DistributedAnalyzeRootExec}, isolator::PartitionIsolatorExec, logging::trace, max_rows::MaxRowsExec, protobuf::{ df_ray_exec_node::Payload, DfRayExecNode, DfRayStageReaderExecNode, - DistributedAnalyzeExecNode, MaxRowsExecNode, PartitionIsolatorExecNode, + DistributedAnalyzeExecNode, DistributedAnalyzeRootExecNode, MaxRowsExecNode, + PartitionIsolatorExecNode, }, stage_reader::DFRayStageReaderExec, }; @@ -104,6 +105,19 @@ impl PhysicalExtensionCodec for DFRayCodec { ))) } } + Payload::DistributedAnalyzeRootExec(distributed_analyze_root_exec_node) => { + if inputs.len() != 1 { + Err(internal_datafusion_err!( + "DistributedAnalyzeRootExec requires one input" + )) + } else { + Ok(Arc::new(DistributedAnalyzeRootExec::new( + inputs[0].clone(), + distributed_analyze_root_exec_node.verbose, + distributed_analyze_root_exec_node.show_statistics, + ))) + } + } } } else { internal_err!("cannot decode proto extension in dfray codec") @@ -147,6 +161,12 @@ impl PhysicalExtensionCodec for DFRayCodec { show_statistics: exec.show_statistics, }; Payload::DistributedAnalyzeExec(pb) + } else if let Some(exec) = node.as_any().downcast_ref::() { + let pb = DistributedAnalyzeRootExecNode { + verbose: exec.verbose, + show_statistics: exec.show_statistics, + }; + Payload::DistributedAnalyzeRootExec(pb) } else { return internal_err!("Not supported node to encode to proto"); }; diff --git a/src/flight_handlers.rs b/src/flight_handlers.rs index 59f9bf1..673857d 100644 --- a/src/flight_handlers.rs +++ b/src/flight_handlers.rs @@ -173,10 +173,7 @@ impl FlightRequestHandler { // Create dummy addresses for EXPLAIN execution let mut dummy_addrs = std::collections::HashMap::new(); let mut partition_addrs = std::collections::HashMap::new(); - partition_addrs.insert( - 0u64, - vec![("explain_local".to_string(), "local".to_string())], - ); + partition_addrs.insert(0u64, vec![]); dummy_addrs.insert(0u64, partition_addrs); self.execute_plan_and_build_stream(explain_plan, tsd.query_id, dummy_addrs) @@ -302,9 +299,12 @@ impl FlightRequestHandler { #[cfg(test)] mod tests { use super::*; - use crate::test_utils::explain_test_helpers::{ - create_explain_ticket_statement_data, create_test_flight_handler, - verify_explain_stream_results, + use crate::{ + test_utils::explain_test_helpers::{ + create_explain_ticket_statement_data, create_test_flight_handler, + verify_explain_stream_results, + }, + vocab::Host, }; use std::collections::HashMap; @@ -319,8 +319,14 @@ mod tests { stage_addrs.insert( 1u64, vec![ - ("worker1".to_string(), "localhost:8001".to_string()), - ("worker2".to_string(), "localhost:8002".to_string()), + Host { + name: "worker1".to_string(), + addr: "localhost:8001".to_string(), + }, + Host { + name: "worker2".to_string(), + addr: "localhost:8002".to_string(), + }, ], ); addrs.insert(1u64, stage_addrs); @@ -350,12 +356,18 @@ mod tests { let mut stage1_addrs = HashMap::new(); stage1_addrs.insert( 1u64, - vec![("worker1".to_string(), "localhost:8001".to_string())], + vec![Host { + name: "worker1".to_string(), + addr: "localhost:8001".to_string(), + }], ); let mut stage2_addrs = HashMap::new(); stage2_addrs.insert( 2u64, - vec![("worker2".to_string(), "localhost:8002".to_string())], + vec![Host { + name: "worker1".to_string(), + addr: "localhost:8002".to_string(), + }], ); addrs.insert(1u64, stage1_addrs); addrs.insert(2u64, stage2_addrs); @@ -494,4 +506,3 @@ mod tests { } } } - diff --git a/src/friendly.rs b/src/friendly.rs index 6d6fda0..17dda48 100644 --- a/src/friendly.rs +++ b/src/friendly.rs @@ -1,7 +1,7 @@ use std::sync::{Mutex, OnceLock}; use anyhow::anyhow; -use rand::{Rng, thread_rng}; +use rand::{thread_rng, Rng}; use crate::result::Result; @@ -1619,9 +1619,11 @@ impl Friendly { .lock() .map_err(|e| anyhow!("Failed to lock adjectives: {}", e))?; + let suffix = rng.gen_range(0..1000); + let index = rng.gen_range(0..adjectives.len()); let adj = adjectives.remove(index); - Ok(format!("{}-{}", adj, name)) + Ok(format!("{}-{}-{:0>4}", adj, name, suffix)) } } diff --git a/src/k8s.rs b/src/k8s.rs deleted file mode 100644 index 0ee3567..0000000 --- a/src/k8s.rs +++ /dev/null @@ -1,254 +0,0 @@ -use std::{ - collections::HashMap, - sync::{Arc, OnceLock}, -}; - -use anyhow::{Context, anyhow}; -use futures::{StreamExt, TryStreamExt}; -use k8s_openapi::api::{apps::v1::Deployment, core::v1::Pod}; -use kube::{ - Client, - api::{Api, ResourceExt, WatchEvent, WatchParams}, -}; -use parking_lot::RwLock; - -use crate::{ - logging::{debug, error, trace}, - result::Result, -}; - -static WORKER_DISCOVERY: OnceLock> = OnceLock::new(); - -pub fn get_worker_addresses() -> Result> { - match WORKER_DISCOVERY.get_or_init(WorkerDiscovery::new) { - Ok(wd) => { - let worker_addrs = wd.get_addresses(); - debug!( - "Worker addresses found:\n{}", - worker_addrs - .iter() - .map(|(name, addr)| format!("{}: {}", name, addr)) - .collect::>() - .join("\n") - ); - Ok(worker_addrs) - }, - Err(e) => Err(anyhow!("Failed to initialize WorkerDiscovery: {}", e).into()), - } -} - -struct WorkerDiscovery { - addresses: Arc>>, -} - -impl WorkerDiscovery { - pub fn new() -> Result { - let wd = WorkerDiscovery { - addresses: Arc::new(RwLock::new(HashMap::new())), - }; - wd.start()?; - Ok(wd) - } - - fn get_addresses(&self) -> Vec<(String, String)> { - let guard = self.addresses.read(); - guard - .iter() - .map(|(_ip, (name, addr))| (name.clone(), addr.clone())) - .collect() - } - - fn start(&self) -> Result<()> { - let worker_addrs_env = std::env::var("DFRAY_WORKER_ADDRESSES"); - let worker_deployment_env = std::env::var("DFRAY_WORKER_DEPLOYMENT"); - let worker_deployment_namespace_env = std::env::var("DFRAY_WORKER_DEPLOYMENT_NAMESPACE"); - - if worker_addrs_env.is_ok() { - // if the env var is set, use it - self.set_worker_addresses_from_env(worker_addrs_env.unwrap().as_str()) - .context("Failed to set worker addresses from env var")?; - } else if worker_deployment_namespace_env.is_ok() && worker_deployment_env.is_ok() { - let addresses = self.addresses.clone(); - let deployment = worker_deployment_env.unwrap(); - let namespace = worker_deployment_namespace_env.unwrap(); - tokio::spawn(async move { - match watch_deployment_hosts_continuous(addresses, &deployment, &namespace).await { - Ok(_) => {} - Err(e) => error!("Error starting worker watcher: {:?}", e), - } - }); - } else { - // if neither env var is set, return an error - return Err(anyhow!( - "Either DFRAY_WORKER_ADDRESSES or both DFRAY_WORKER_DEPLOYMENT and \ - DFRAY_WORKER_DEPLOYMENT_NAMESPACE must be set" - ) - .into()); - } - Ok(()) - } - - fn set_worker_addresses_from_env(&self, env_str: &str) -> Result<()> { - // get addresss from an env var where addresses are split by comans - // and in the form of name/address,name/address - let mut guard = self.addresses.write(); - - for addr in env_str.split(',') { - let parts: Vec<&str> = addr.split('/').collect(); - if parts.len() != 2 { - return Err(anyhow!("Invalid worker address format: {addr}").into()); - } - let name = parts[0].to_string(); - let address = parts[1].to_string(); - guard.insert(address.clone(), (name, address)); - } - Ok(()) - } -} -/// Continuously watch for changes to pods in a Kubernetes deployment and call a -/// handler function whenever the list of hosts changes. -/// -/// # Arguments -/// * `deployment_name` - Name of the deployment -/// * `namespace` - Kubernetes namespace where the deployment is located -/// * `handler` - A function to call when the host list changes -/// -/// # Returns -/// This function runs indefinitely until an error occurs -/// -/// # Errors -/// Returns an error if there's an issue connecting to the Kubernetes API -/// or if the deployment or its pods cannot be found -async fn watch_deployment_hosts_continuous( - addresses: Arc>>, - deployment_name: &str, - namespace: &str, -) -> Result<()> { - debug!( - "Starting to watch deployment {} in namespace {}", - deployment_name, namespace - ); - // Initialize the Kubernetes client - let client = Client::try_default() - .await - .context("Failed to create Kubernetes client")?; - - // Access the Deployments API - let deployments: Api = Api::namespaced(client.clone(), namespace); - - // Get the specific deployment - let deployment = deployments - .get(deployment_name) - .await - .context(format!("Failed to get deployment {}", deployment_name))?; - - // Extract the selector labels from the deployment - let selector = deployment - .spec - .as_ref() - .and_then(|spec| spec.selector.match_labels.as_ref()) - .context("Deployment has no selector labels")?; - - // Convert selector to a string format for the label selector - let label_selector = selector - .iter() - .map(|(k, v)| format!("{}={}", k, v)) - .collect::>() - .join(","); - - // Access the Pods API - let pods: Api = Api::namespaced(client, namespace); - - debug!( - "Watching deployment {} in namespace {} with label selector: {}", - deployment_name, namespace, label_selector - ); - - let wp = WatchParams::default().labels(&label_selector); - - // Start watching for pod changes - let mut watcher = pods - .watch(&wp, "0") - .await - .context("could not build watcher")? - .boxed(); - - while let Some(event_result) = watcher - .try_next() - .await - .context("could not get next event from watcher")? - { - match &event_result { - WatchEvent::Added(pod) | WatchEvent::Modified(pod) => { - trace!( - "Pod event: {:?}, added or modified: {:#?}", - event_result, pod - ); - if let Some(Some(_ip)) = pod.status.as_ref().map(|s| s.pod_ip.as_ref()) { - let (pod_ip, name_str, host_str) = get_worker_info_from_pod(pod)?; - debug!( - "Pod {} has IP address {}, name {}, host {}", - pod.name_any(), - pod_ip, - name_str, - host_str - ); - addresses.write().insert(pod_ip, (name_str, host_str)); - } else { - trace!("Pod {} has no IP address, skipping", pod.name_any()); - } - } - WatchEvent::Deleted(pod) => { - debug!("Pod deleted: {}", pod.name_any()); - if let Some(status) = &pod.status { - if let Some(pod_ip) = &status.pod_ip { - if !pod_ip.is_empty() { - debug!("Removing pod IP: {}", pod_ip); - addresses.write().remove(pod_ip); - } - } - } - } - WatchEvent::Bookmark(_) => {} - WatchEvent::Error(e) => { - eprintln!("Watch error: {}", e); - } - } - } - - Ok(()) -} - -fn get_worker_info_from_pod(pod: &Pod) -> Result<(String, String, String)> { - let status = pod.status.as_ref().context("Pod has no status")?; - let pod_ip = status.pod_ip.as_ref().context("Pod has no IP address")?; - - // filter on container name - let port = pod - .spec - .as_ref() - .and_then(|spec| { - spec.containers - .iter() - .find(|c| c.name == "dfray-worker") - .and_then(|c| { - c.ports - .as_ref() - .and_then(|ports| ports.iter().next().map(|p| p.container_port)) - }) - }) - .ok_or_else(|| { - anyhow::anyhow!( - "No could not find container port for container named dfray-worker found in pod {}", - pod.name_any() - ) - })?; - - if pod_ip.is_empty() { - Err(anyhow::anyhow!("Pod {} has no IP address", pod.name_any()).into()) - } else { - let host_str = format!("{}:{}", pod_ip, port); - let name_str = format!("{}:{}", pod_ip, port); // for now - Ok((pod_ip.to_owned(), name_str, host_str)) - } -} diff --git a/src/lib.rs b/src/lib.rs index 0a9f6b3..323c00c 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -29,7 +29,6 @@ pub mod flight; pub mod flight_handlers; pub mod friendly; pub mod isolator; -pub mod k8s; pub mod logging; pub mod max_rows; pub mod physical; @@ -43,6 +42,7 @@ pub mod stage_reader; pub mod test_utils; pub mod util; pub mod vocab; +pub mod worker_discovery; #[cfg(not(target_env = "msvc"))] use tikv_jemallocator::Jemalloc; diff --git a/src/main.rs b/src/main.rs index 0c08ed5..5fea6d9 100644 --- a/src/main.rs +++ b/src/main.rs @@ -1,10 +1,8 @@ use anyhow::Result; use clap::Parser; use distributed_datafusion::{ - friendly::new_friendly_name, - processor_service::DFRayProcessorService, - proxy_service::DFRayProxyService, - setup, + friendly::new_friendly_name, processor_service::DFRayProcessorService, + proxy_service::DFRayProxyService, setup, }; #[derive(Parser)] @@ -40,8 +38,7 @@ async fn main() -> Result<()> { service.serve().await?; } "worker" => { - let mut service = DFRayProcessorService::new(new_friendly_name()?, args.port); - service.start_up().await?; + let service = DFRayProcessorService::new(new_friendly_name()?, args.port).await?; service.serve().await?; } _ => { diff --git a/src/physical.rs b/src/physical.rs index 6ecca14..cff362b 100644 --- a/src/physical.rs +++ b/src/physical.rs @@ -22,13 +22,16 @@ use datafusion::{ error::Result, physical_optimizer::PhysicalOptimizerRule, physical_plan::{ - analyze::AnalyzeExec, joins::NestedLoopJoinExec, repartition::RepartitionExec, - sorts::sort::SortExec, ExecutionPlan, + analyze::AnalyzeExec, coalesce_partitions::CoalescePartitionsExec, + joins::NestedLoopJoinExec, repartition::RepartitionExec, sorts::sort::SortExec, + ExecutionPlan, }, }; use crate::{ - analyze::DistributedAnalyzeExec, logging::info, stage::DFRayStageExec, + analyze::{DistributedAnalyzeExec, DistributedAnalyzeRootExec}, + logging::info, + stage::DFRayStageExec, util::display_plan_with_partition_counts, }; @@ -50,7 +53,6 @@ impl Default for DFRayStageOptimizerRule { Self::new() } } - impl DFRayStageOptimizerRule { pub fn new() -> Self { Self {} @@ -92,6 +94,24 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { let stage = Arc::new(DFRayStageExec::new(plan, stage_counter)); stage_counter += 1; Ok(Transformed::yes(stage as Arc)) + } else if let Some(definitely_analize_plan) = + plan.as_any().downcast_ref::() + { + // we need to replace this with a DistributedAnalyzeRootExec so that we can + // discoard the output and send back the plans for each task. + + // add a coalesce partitions exec to ensure that we have a single partition + let child = Arc::new(CoalescePartitionsExec::new( + definitely_analize_plan.input().clone(), + )) as Arc; + + let new_plan = Arc::new(DistributedAnalyzeRootExec::new( + child, + definitely_analize_plan.verbose(), + definitely_analize_plan.show_statistics(), + )) as Arc; + + Ok(Transformed::yes(new_plan as Arc)) } else { Ok(Transformed::no(plan)) } @@ -116,3 +136,77 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { true } } + +#[cfg(test)] +mod tests { + use super::*; + use datafusion::arrow::array::{Int32Array, StringArray}; + use datafusion::arrow::datatypes::{DataType, Field, Schema}; + use datafusion::arrow::record_batch::RecordBatch; + use datafusion::catalog::memory::DataSourceExec; + use datafusion::execution::context::SessionContext; + use datafusion::physical_plan::displayable; + use std::sync::Arc; + + #[tokio::test] + async fn test_optimize_with_explain_analyze() { + // Create a session context + let ctx = SessionContext::new(); + + // Define a schema for the in-memory table + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + // Create some data for the table + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + ], + ) + .unwrap(); + + // Register the in-memory table + ctx.register_batch("test_table", batch).unwrap(); + + // Run the EXPLAIN ANALYZE query + let df = ctx + .sql("EXPLAIN ANALYZE SELECT * FROM test_table") + .await + .unwrap(); + // get the physical plan from the dataframe + let physical_plan = df.create_physical_plan().await.unwrap(); + + // Apply the optimizer + let optimizer = DFRayStageOptimizerRule::new(); + let optimized_physical_plan = optimizer + .optimize(physical_plan.clone(), &Default::default()) + .unwrap(); + + let data_source = ctx + .table_provider("test_table") + .await + .unwrap() + .scan(&ctx.state(), None, &[], None) + .await + .unwrap(); + + let coalesce = Arc::new(CoalescePartitionsExec::new(data_source)); + + let analyze = Arc::new(DistributedAnalyzeRootExec::new(coalesce, false, false)); + + let target_plan = DFRayStageExec::new( + analyze, 0, // stage counter + ); + + assert_eq!( + displayable(optimized_physical_plan.as_ref()) + .indent(true) + .to_string(), + displayable(&target_plan).indent(true).to_string() + ); + } +} diff --git a/src/planning.rs b/src/planning.rs index 13eed20..0078f44 100644 --- a/src/planning.rs +++ b/src/planning.rs @@ -33,12 +33,12 @@ use crate::{ logging::{debug, error, info, trace}, max_rows::MaxRowsExec, physical::DFRayStageOptimizerRule, - protobuf::{Host, Hosts, PartitionAddrs, StageAddrs, StageData}, result::{DFRayError, Result}, stage::DFRayStageExec, stage_reader::{DFRayStageReaderExec, QueryId}, util::{display_plan_with_partition_counts, get_client, physical_plan_to_bytes, wait_for}, vocab::{Addrs, CtxAnnotatedOutputs, CtxName, CtxPartitionGroup, CtxStageAddrs}, + vocab::{Host, Hosts, PartitionAddrs, StageAddrs, StageData}, }; #[derive(Debug)] @@ -96,7 +96,7 @@ static STATE: LazyLock> = LazyLock::new(|| { pub fn get_ctx() -> Result { match &*STATE { - Ok(ctx) => Ok(SessionContext::new_with_state(ctx.clone())), + Ok(state) => Ok(SessionContext::new_with_state(state.clone())), Err(e) => Err(anyhow!("Context initialization failed: {}", e).into()), } } @@ -341,6 +341,12 @@ pub async fn execution_planning( let distributed_plan_clone = Arc::clone(&distributed_plan); distributed_plan.transform_up(up)?; + let txt = stages + .iter() + .map(|stage| format!("{}", display_plan_with_partition_counts(&stage.plan))) + .join(",\n"); + trace!("stages before fix:\n{}", txt); + // add coalesce and max rows to last stage let mut last_stage = stages.pop().ok_or(anyhow!("No stages found"))?; @@ -362,7 +368,7 @@ pub async fn execution_planning( .iter() .map(|stage| format!("{}", display_plan_with_partition_counts(&stage.plan))) .join(",\n"); - trace!("stages:{}", txt); + trace!("stages:\n{}", txt); Ok((distributed_plan_clone, stages)) } @@ -373,18 +379,22 @@ pub async fn execution_planning( pub async fn distribute_stages( query_id: &str, stages: Vec, - worker_addrs: Vec<(String, String)>, + worker_addrs: Vec, ) -> Result { // map of worker name to address // FIXME: use types over tuples of strings, as we can accidently swap them and // not know - let mut workers: HashMap = worker_addrs.iter().cloned().collect(); + // a map of worker name to host + let mut workers: HashMap = worker_addrs + .iter() + .map(|host| (host.name.clone(), host.clone())) + .collect(); for attempt in 0..3 { // all stages to workers let (stage_datas, final_addrs) = - assign_to_workers(query_id, &stages, workers.iter().collect())?; + assign_to_workers(query_id, &stages, workers.values().collect())?; // we retry this a few times to ensure that the workers are ready // and can accept the stages @@ -396,7 +406,7 @@ pub async fn distribute_stages( worker {bad_worker}. Retrying..." ); // if we cannot communicate with a worker, we remove it from the list of workers - workers.remove(&bad_worker); + workers.remove(&bad_worker.name); } Err(e) => return Err(e), } @@ -418,7 +428,7 @@ async fn try_distribute_stages(stage_datas: &[StageData]) -> Result<()> { "Distributing stage_id {}, pg: {:?} to worker: {:?}", stage_data.stage_id, stage_data.partition_group, - stage_data.assigned_addr + stage_data.assigned_host ); // populate its child stages @@ -428,28 +438,17 @@ async fn try_distribute_stages(stage_datas: &[StageData]) -> Result<()> { stage_datas, )?); - let mut client = match get_client( - &stage_data - .assigned_addr - .as_ref() - .context("Assigned stage address is missing")? - .name, - &stage_data - .assigned_addr - .as_ref() - .context("Assigned stage address is missing")? - .addr, - ) { + let host = stage_data + .assigned_host + .clone() + .context("Assigned host is missing for stage data")?; + + let mut client = match get_client(&host) { Ok(client) => client, Err(e) => { error!("Couldn't not communicate with worker {e:#?}"); return Err(DFRayError::WorkerCommunicationError( - stage_data - .assigned_addr - .as_ref() - .cloned() - .unwrap() // we know we can unwrap it as we checked a few lines up - .name, + host.clone(), // here )); } }; @@ -488,7 +487,7 @@ async fn try_distribute_stages(stage_datas: &[StageData]) -> Result<()> { fn assign_to_workers( query_id: &str, stages: &[DFRayStage], - worker_addrs: Vec<(&String, &String)>, + worker_addrs: Vec<&Host>, ) -> Result<(Vec, Addrs)> { let mut stage_datas = vec![]; let mut worker_idx = 0; @@ -502,10 +501,7 @@ fn assign_to_workers( for partition_group in stage.partition_groups.iter() { let plan_bytes = physical_plan_to_bytes(stage.plan.clone())?; - let addr = Host { - name: worker_addrs[worker_idx].0.to_string(), - addr: worker_addrs[worker_idx].1.to_string(), - }; + let host = worker_addrs[worker_idx].clone(); worker_idx = (worker_idx + 1) % worker_addrs.len(); if stage.stage_id as isize > max_stage_id { @@ -521,7 +517,7 @@ fn assign_to_workers( .or_default() .entry(*part) .or_default() - .push((addr.name.clone(), addr.addr.clone())); + .push(host.clone()); } } @@ -534,7 +530,7 @@ fn assign_to_workers( stage_addrs: None, // will be calculated and filled in later num_output_partitions: stage.plan.output_partitioning().partition_count() as u64, full_partitions: stage.full_partitions, - assigned_addr: Some(addr), + assigned_host: Some(host), }; stage_datas.push(stage_data); } @@ -567,17 +563,12 @@ fn get_stage_addrs_from_stages( .iter() .filter(|s| s.stage_id == stage_id) .map(|s| { - Ok(Host { - name: s - .assigned_addr - .clone() - .context("assigned address missing")? - .name, - addr: s - .assigned_addr - .clone() - .context("assigned address missing")? - .addr, + s.assigned_host.clone().ok_or_else(|| { + anyhow!( + "Assigned address is missing for stage_id: {}", + stage_id + ) + .into() }) }) .collect::>>()?; @@ -587,18 +578,10 @@ fn get_stage_addrs_from_stages( .insert(*part, Hosts { hosts }); } else { // we have the full partition, so this is the only address required - let host = Host { - name: stage - .assigned_addr - .clone() - .context("Assigned address is missing")? - .name, - addr: stage - .assigned_addr - .clone() - .context("Assigned address is missing")? - .addr, - }; + let host = stage + .assigned_host + .clone() + .context("Assigned address is missing")?; partition_addrs .partition_addrs .insert(*part, Hosts { hosts: vec![host] }); diff --git a/src/processor_service.rs b/src/processor_service.rs index 6c843ae..a4714a9 100644 --- a/src/processor_service.rs +++ b/src/processor_service.rs @@ -17,6 +17,7 @@ use std::{ collections::HashMap, + ops::DerefMut, sync::Arc, time::{Duration, SystemTime}, }; @@ -47,7 +48,8 @@ use crate::{ logging::{debug, error, info, trace}, planning::{add_ctx_extentions, get_ctx}, protobuf::{ - AnnotatedTaskOutput, AnnotatedTaskOutputs, FlightDataMetadata, FlightTicketData, StageData, + AnnotatedTaskOutput, AnnotatedTaskOutputs, FlightDataMetadata, FlightTicketData, Host, + StageData, }, result::{DFRayError, Result}, util::{ @@ -72,6 +74,8 @@ type PlanVec = Vec<(SystemTime, SessionContext, Arc)>; struct DfRayProcessorHandler { /// our name, useful for logging name: String, + //// our address string, also useful for logging + pub(crate) addr: String, /// our map of query_id -> (session ctx, execution plan) #[allow(clippy::type_complexity)] plans: Arc>>, @@ -79,7 +83,7 @@ struct DfRayProcessorHandler { } impl DfRayProcessorHandler { - pub fn new(name: String) -> Self { + pub fn new(name: String, addr: String) -> Self { let plans: Arc>> = Arc::new(RwLock::new(HashMap::new())); let done = Arc::new(Mutex::new(false)); @@ -135,10 +139,16 @@ impl DfRayProcessorHandler { } }); - Self { name, plans, done } + Self { + name, + addr, + plans, + done, + } } #[allow(dead_code)] + /// shutdown pub fn all_done(&self) { *self.done.lock() = true; } @@ -289,10 +299,26 @@ impl DfRayProcessorHandler { let stream = plan .execute(partition as usize, task_ctx) .inspect_err(|e| error!("Could not get partition stream from plan {e:#?}"))? + .inspect(|batch| { + trace!("producing maybe batch {:?}", batch); + }) .map_err(|e| FlightError::from_external_error(Box::new(e))); info!("{} plans held {}", self.name, self.plans.read().len()); + fn find_analyze(plan: &dyn ExecutionPlan) -> Option<&DistributedAnalyzeExec> { + if let Some(target) = plan.as_any().downcast_ref::() { + Some(target) + } else { + for child in plan.children() { + if let Some(target) = find_analyze(child.as_ref()) { + return Some(target); + } + } + None + } + } + let mut flight_data_stream = FlightDataEncoderBuilder::new().build(stream); let name = self.name.clone(); @@ -312,7 +338,7 @@ impl DfRayProcessorHandler { .clone(); - if let Some(analyze) = plan.as_any().downcast_ref::() { + if let Some(analyze) = find_analyze(plan.as_ref()) { let annotated_plan = analyze.annotated_plan(); debug!("sending annotated plan: {}", annotated_plan); @@ -363,12 +389,12 @@ impl DfRayProcessorHandler { }, (false, Some(Ok(flight_data))) => { // we have a flight data, so we yield it - // decode this data for output + trace!("received normal flight data, yielding"); yield Ok(flight_data); }, (true, _) => { // we are done, so we don't yield anything - debug!("{} stream done for stage {} partition {}", name, stage_id, partition); + error!("{} we should not arrive at this block!. stage {} partition {}", name, stage_id, partition); }, } } @@ -376,6 +402,58 @@ impl DfRayProcessorHandler { Ok(Box::pin(out_stream)) } + fn do_action_get_host(&self) -> Result, Status> { + let addr = self.addr.clone(); + let name = self.name.clone(); + + let out_stream = Box::pin(stream! { + yield Ok::<_, tonic::Status>(arrow_flight::Result { + body: Host{ + addr, + name, + }.encode_to_vec().into() + }); + }) as crate::flight::DoActionStream; + + Ok(Response::new(out_stream)) + } + + async fn do_action_add_plan( + &self, + action: Action, + ) -> Result, Status> { + let stage_data = StageData::decode(action.body.as_ref()).map_err(|e| { + Status::internal(format!( + "{}, Unexpected error decoding StageData: {e:?}", + self.name + )) + })?; + + let addrs = stage_data + .stage_addrs + .as_ref() + .context("stage addrs not present") + .map_err(DFRayError::from) + .and_then(get_addrs) + .map_err(|e| Status::internal(format!("{}, {e}", self.name)))?; + + self.add_plan( + stage_data.query_id, + stage_data.stage_id, + addrs, + stage_data.partition_group, + stage_data.full_partitions, + &stage_data.plan_bytes, + ) + .await + .map_err(|e| Status::internal(format!("{}, Could not add plan: {e:?}", self.name)))?; + + let out_stream = Box::pin(stream! { + yield Ok::<_, tonic::Status>(arrow_flight::Result::default()); + }) as crate::flight::DoActionStream; + + Ok(Response::new(out_stream)) + } } #[async_trait] @@ -440,45 +518,27 @@ impl FlightHandler for DfRayProcessorHandler { let action_type = action.r#type.as_str(); trace!("{} received action: {}", self.name, action_type); - if action_type != "add_plan" { - return Err(Status::unimplemented(format!( + if action_type == "add_plan" { + self.do_action_add_plan(action).await + } else if action_type == "get_host" { + self.do_action_get_host() + } else { + Err(Status::unimplemented(format!( "{}, Unimplemented action: {}", self.name, action_type - ))); + ))) } + } +} - let stage_data = StageData::decode(action.body.as_ref()).map_err(|e| { - Status::internal(format!( - "{}, Unexpected error decoding StageData: {e:?}", - self.name - )) - })?; +pub async fn start_up(port: usize) -> Result { + let my_host_str = format!("0.0.0.0:{}", port); - let addrs = stage_data - .stage_addrs - .as_ref() - .context("stage addrs not present") - .map_err(DFRayError::from) - .and_then(get_addrs) - .map_err(|e| Status::internal(format!("{}, {e}", self.name)))?; - - self.add_plan( - stage_data.query_id, - stage_data.stage_id, - addrs, - stage_data.partition_group, - stage_data.full_partitions, - &stage_data.plan_bytes, - ) + let listener = TcpListener::bind(&my_host_str) .await - .map_err(|e| Status::internal(format!("{}, Could not add plan: {e:?}", self.name)))?; - - let out_stream = Box::pin(stream! { - yield Ok::<_, tonic::Status>(arrow_flight::Result::default()); - }) as crate::flight::DoActionStream; + .context("Could not bind socket to {my_host_str}")?; - Ok(Response::new(out_stream)) - } + Ok(listener) } /// DFRayProcessorService is a Arrow Flight service that serves streams of @@ -488,26 +548,30 @@ impl FlightHandler for DfRayProcessorHandler { pub struct DFRayProcessorService { #[allow(dead_code)] name: String, - listener: Option, + listener: TcpListener, handler: Arc, - addr: Option, + addr: String, all_done_tx: Arc>>, all_done_rx: Option>, port: usize, } impl DFRayProcessorService { - pub fn new(name: String, port: usize) -> Self { + pub async fn new(name: String, port: usize) -> Result { let name = format!("[{}]", name); - let listener = None; - let addr = None; let (all_done_tx, all_done_rx) = channel(1); let all_done_tx = Arc::new(Mutex::new(all_done_tx)); - let handler = Arc::new(DfRayProcessorHandler::new(name.clone())); + let listener = start_up(port).await?; - Self { + let addr = format!("{}", listener.local_addr().unwrap()); + + info!("DFRayProcessorService bound to {addr}"); + + let handler = Arc::new(DfRayProcessorHandler::new(name.clone(), addr.clone())); + + Ok(Self { name, listener, handler, @@ -515,35 +579,12 @@ impl DFRayProcessorService { all_done_tx, all_done_rx: Some(all_done_rx), port, - } + }) } - pub async fn start_up(&mut self) -> Result<()> { - let my_host_str = format!("0.0.0.0:{}", self.port); - - self.listener = TcpListener::bind(&my_host_str) - .await - .map(Some) - .context("Could not bind socket to {my_host_str}")?; - - self.addr = Some(format!( - "{}", - self.listener.as_ref().unwrap().local_addr().unwrap() - )); - - info!( - "DFRayProcessorService bound to {}", - self.addr.as_ref().unwrap() - ); - - Ok(()) - } /// get the address of the listing socket for this service - pub fn addr(&self) -> Result { - let addr = self.addr.clone().ok_or(anyhow!( - "DFRayProxyService not started yet, no address available" - ))?; - Ok(addr) + pub fn addr(&self) -> String { + self.addr.clone() } pub async fn all_done(&self) -> Result<()> { @@ -556,8 +597,8 @@ impl DFRayProcessorService { Ok(()) } - /// start the service - pub async fn serve(&mut self) -> Result<()> { + /// start the service, consuming self + pub async fn serve(mut self) -> Result<()> { let mut all_done_rx = self.all_done_rx.take().unwrap(); let signal = async move { @@ -574,12 +615,10 @@ impl DFRayProcessorService { let svc = FlightServiceServer::new(flight_serv); - let listener = self.listener.take().unwrap(); - Server::builder() .add_service(svc) .serve_with_incoming_shutdown( - tokio_stream::wrappers::TcpListenerStream::new(listener), + tokio_stream::wrappers::TcpListenerStream::new(self.listener), signal, ) .await diff --git a/src/proto/datafusion_ray.proto b/src/proto/datafusion_ray.proto deleted file mode 100644 index ba237dc..0000000 --- a/src/proto/datafusion_ray.proto +++ /dev/null @@ -1,132 +0,0 @@ -syntax = "proto3"; - -package datafusion_ray.protobuf; - -import "datafusion.proto"; -import "datafusion_common.proto"; - -option java_multiple_files = true; -option java_outer_classname = "RayDataFusionProto"; -option java_package = "datafusion_ray.protobuf"; - -message DFRayStageReaderExecNode { - // schema of the stage we will consume - datafusion_common.Schema schema = 1; - // properties of the stage we will consume - datafusion.Partitioning partitioning = 2; - // stage to read from - uint64 stage_id = 3; -} - -message MaxRowsExecNode { - uint64 max_rows = 1; -} - -message PartitionIsolatorExecNode { - uint64 partition_count = 2; -} - -message DistributedAnalyzeExecNode { - // how much data to show - bool verbose = 1; - // if statistics should be displayed - bool show_statistics = 2; -} - -message DistributedAnalyzeRootExecNode {} - -message DistributedExplainExecNode { - datafusion_common.Schema schema = 1; - string logical_plan = 2; - string physical_plan = 3; - string distributed_plan = 4; - string distributed_stages = 5; -} - -message DFRayExecNode { - oneof payload { - DFRayStageReaderExecNode stage_reader_exec = 1; - MaxRowsExecNode max_rows_exec = 2; - PartitionIsolatorExecNode isolator_exec = 3; - DistributedAnalyzeExecNode distributed_analyze_exec = 4; - //DistributedAnalyzeRootExecNode distributed_analyze_root_exec = 5; - } -} - -message AnnotatedTaskOutput { - // the output of the explain analyze - string plan = 1; - // the host who executed this stage - Host host = 2; - // the stage id that was executed - uint64 stage_id = 3; - // the partitions that were executed by this stage - repeated uint64 partition_group = 4; -} - -message AnnotatedTaskOutputs { - repeated AnnotatedTaskOutput outputs = 1; -} - -message FlightDataMetadata { - AnnotatedTaskOutputs annotated_task_outputs = 1; -} - -message FlightTicketData { - string query_id = 1; - uint64 stage_id = 2; - uint64 partition = 3; - // name of the requestor, useful in logs for debugging - string requestor_name = 4; -} - -message TicketStatementData { - // identity of the query we want to consume - string query_id = 1; - // the stage id of the final stage - uint64 stage_id = 2; - // host holding the final stage, the root of the query plan - StageAddrs stage_addrs = 3; - // the schema of the final stage - datafusion_common.Schema schema = 4; - // For EXPLAIN queries, store the explain plan data directly - DistributedExplainExecNode explain_data = 5; -} - -message Host { - string addr = 1; - string name = 2; -} - -message Hosts { - repeated Host hosts = 1; -} - -message PartitionAddrs { - map partition_addrs = 1; -} - -message StageAddrs { - // map of stage id to stage address - map stage_addrs = 1; -} - -message StageData { - string query_id = 1; - - uint64 stage_id = 2; - - repeated uint64 partition_group = 3; - - bytes plan_bytes = 4; - - repeated uint64 child_stage_ids = 5; - - StageAddrs stage_addrs = 6; - - uint64 num_output_partitions = 7; - - bool full_partitions = 8; - - Host assigned_addr = 9; -} diff --git a/src/proto/generated/protobuf.rs b/src/proto/generated/protobuf.rs index f6b5e0e..4045c9a 100644 --- a/src/proto/generated/protobuf.rs +++ b/src/proto/generated/protobuf.rs @@ -31,7 +31,14 @@ pub struct DistributedAnalyzeExecNode { pub show_statistics: bool, } #[derive(Clone, Copy, PartialEq, ::prost::Message)] -pub struct DistributedAnalyzeRootExecNode {} +pub struct DistributedAnalyzeRootExecNode { + /// how much data to show + #[prost(bool, tag = "1")] + pub verbose: bool, + /// if statistics should be displayed + #[prost(bool, tag = "2")] + pub show_statistics: bool, +} #[derive(Clone, PartialEq, ::prost::Message)] pub struct DistributedExplainExecNode { #[prost(message, optional, tag = "1")] @@ -47,7 +54,7 @@ pub struct DistributedExplainExecNode { } #[derive(Clone, PartialEq, ::prost::Message)] pub struct DfRayExecNode { - #[prost(oneof = "df_ray_exec_node::Payload", tags = "1, 2, 3, 4")] + #[prost(oneof = "df_ray_exec_node::Payload", tags = "1, 2, 3, 4, 5")] pub payload: ::core::option::Option, } /// Nested message and enum types in `DFRayExecNode`. @@ -60,9 +67,10 @@ pub mod df_ray_exec_node { MaxRowsExec(super::MaxRowsExecNode), #[prost(message, tag = "3")] IsolatorExec(super::PartitionIsolatorExecNode), - /// DistributedAnalyzeRootExecNode distributed_analyze_root_exec = 5; #[prost(message, tag = "4")] DistributedAnalyzeExec(super::DistributedAnalyzeExecNode), + #[prost(message, tag = "5")] + DistributedAnalyzeRootExec(super::DistributedAnalyzeRootExecNode), } } #[derive(Clone, PartialEq, ::prost::Message)] @@ -162,5 +170,5 @@ pub struct StageData { #[prost(bool, tag = "8")] pub full_partitions: bool, #[prost(message, optional, tag = "9")] - pub assigned_addr: ::core::option::Option, + pub assigned_host: ::core::option::Option, } diff --git a/src/proto/mod.rs b/src/proto/mod.rs index 2602738..a3ddc5f 100644 --- a/src/proto/mod.rs +++ b/src/proto/mod.rs @@ -16,7 +16,7 @@ // under the License. use crate::{ - protobuf::{Host, Hosts, StageAddrs}, + protobuf::{Hosts, StageAddrs}, vocab::Addrs, }; @@ -29,10 +29,7 @@ impl From for StageAddrs { for (stage_id, partition_addrs) in value { for (part, part_addrs) in partition_addrs { - let host_addrs = part_addrs - .into_iter() - .map(|(name, addr)| Host { name, addr }) - .collect(); + let host_addrs = part_addrs.into_iter().collect(); stage_addrs .stage_addrs .entry(stage_id) diff --git a/src/proxy_service.rs b/src/proxy_service.rs index 39964a0..8947682 100644 --- a/src/proxy_service.rs +++ b/src/proxy_service.rs @@ -17,30 +17,27 @@ use std::sync::Arc; -use anyhow::{Context, anyhow}; +use anyhow::{anyhow, Context}; use arrow_flight::{ - FlightDescriptor, - FlightInfo, - Ticket, - flight_service_server::FlightServiceServer, + flight_service_server::FlightServiceServer, FlightDescriptor, FlightInfo, Ticket, }; use parking_lot::Mutex; use prost::Message; use tokio::{ net::TcpListener, - sync::mpsc::{Receiver, Sender, channel}, + sync::mpsc::{channel, Receiver, Sender}, }; -use tonic::{Request, Response, Status, async_trait, transport::Server}; +use tonic::{async_trait, transport::Server, Request, Response, Status}; use crate::{ explain::is_explain_query, flight::{FlightSqlHandler, FlightSqlServ}, flight_handlers::FlightRequestHandler, - k8s::get_worker_addresses, logging::{debug, info, trace}, protobuf::TicketStatementData, query_planner::QueryPlanner, result::Result, + worker_discovery::get_worker_addresses, }; pub struct DfRayProxyHandler { @@ -67,7 +64,9 @@ impl FlightSqlHandler for DfRayProxyHandler { let is_explain = is_explain_query(&query.query); if is_explain { - self.flight_handler.handle_explain_request(&query.query).await + self.flight_handler + .handle_explain_request(&query.query) + .await } else { self.flight_handler.handle_query_request(&query.query).await } @@ -90,9 +89,13 @@ impl FlightSqlHandler for DfRayProxyHandler { debug!("request for ticket: {:?} from {}", tsd, remote_addr); if tsd.explain_data.is_some() { - self.flight_handler.handle_explain_statement_execution(tsd, &remote_addr).await + self.flight_handler + .handle_explain_statement_execution(tsd, &remote_addr) + .await } else { - self.flight_handler.handle_regular_statement_execution(tsd, &remote_addr).await + self.flight_handler + .handle_regular_statement_execution(tsd, &remote_addr) + .await } } } @@ -210,75 +213,77 @@ mod tests { use super::*; // Test-specific imports use arrow_flight::{ - FlightDescriptor, - Ticket, sql::{CommandStatementQuery, TicketStatementQuery}, + FlightDescriptor, Ticket, }; use prost::Message; use tonic::Request; - - use crate::{ - test_utils::explain_test_helpers::{ - create_explain_ticket_statement_data, - create_test_proxy_handler, - verify_explain_stream_results - }, + + use crate::test_utils::explain_test_helpers::{ + create_explain_ticket_statement_data, create_test_proxy_handler, + verify_explain_stream_results, }; #[tokio::test] async fn test_get_flight_info_statement_explain() { - let handler = create_test_proxy_handler(); - + // Test EXPLAIN query let command = CommandStatementQuery { query: "EXPLAIN SELECT 1 as test_col".to_string(), transaction_id: None, }; - + let request = Request::new(FlightDescriptor::new_cmd(vec![])); let result = handler.get_flight_info_statement(command, request).await; - + assert!(result.is_ok()); let response = result.unwrap(); let flight_info = response.into_inner(); - + // Verify FlightInfo structure assert!(!flight_info.schema.is_empty()); assert_eq!(flight_info.endpoint.len(), 1); assert!(flight_info.endpoint[0].ticket.is_some()); - + // Verify that ticket has content (encoded TicketStatementData) let ticket = flight_info.endpoint[0].ticket.as_ref().unwrap(); assert!(!ticket.ticket.is_empty()); - - println!("✓ FlightInfo created successfully with {} schema bytes and ticket with {} bytes", - flight_info.schema.len(), ticket.ticket.len()); + + println!( + "✓ FlightInfo created successfully with {} schema bytes and ticket with {} bytes", + flight_info.schema.len(), + ticket.ticket.len() + ); } #[tokio::test] async fn test_do_get_statement_explain() { - let handler = create_test_proxy_handler(); - + // First prepare an EXPLAIN query to get proper ticket data let query = "EXPLAIN SELECT 1 as test_col"; - let plans = handler.flight_handler.planner.prepare_explain(query).await.unwrap(); - + let plans = handler + .flight_handler + .planner + .prepare_explain(query) + .await + .unwrap(); + let tsd = create_explain_ticket_statement_data(plans); - + // Create the ticket let ticket_query = TicketStatementQuery { statement_handle: tsd.encode_to_vec().into(), }; - + let request = Request::new(Ticket::new(vec![])); let result = handler.do_get_statement(ticket_query, request).await; - + assert!(result.is_ok()); let response = result.unwrap(); let stream = response.into_inner(); - + // Use shared verification function verify_explain_stream_results(stream).await; } @@ -287,35 +292,54 @@ mod tests { async fn test_compare_explain_flight_info_responses() { let handler = create_test_proxy_handler(); let query = "EXPLAIN SELECT 1 as test_col"; - + // Get FlightInfo from handle_explain_request - let result1 = handler.flight_handler.handle_explain_request(query).await.unwrap(); + let result1 = handler + .flight_handler + .handle_explain_request(query) + .await + .unwrap(); let flight_info1 = result1.into_inner(); - + // Get FlightInfo from get_flight_info_statement let command = CommandStatementQuery { query: query.to_string(), transaction_id: None, }; let request = Request::new(FlightDescriptor::new_cmd(vec![])); - let result2 = handler.get_flight_info_statement(command, request).await.unwrap(); + let result2 = handler + .get_flight_info_statement(command, request) + .await + .unwrap(); let flight_info2 = result2.into_inner(); - + // Compare FlightInfo responses (structure should be identical) assert_eq!(flight_info1.schema.len(), flight_info2.schema.len()); // Same schema size assert_eq!(flight_info1.endpoint.len(), flight_info2.endpoint.len()); // Same number of endpoints assert_eq!(flight_info1.endpoint.len(), 1); // Both should have exactly one endpoint - + // Both should have tickets with content let ticket1 = flight_info1.endpoint[0].ticket.as_ref().unwrap(); let ticket2 = flight_info2.endpoint[0].ticket.as_ref().unwrap(); assert!(!ticket1.ticket.is_empty()); assert!(!ticket2.ticket.is_empty()); - + println!("✓ Both tests produce FlightInfo with identical structure:"); - println!(" - Schema bytes: {} vs {}", flight_info1.schema.len(), flight_info2.schema.len()); - println!(" - Endpoints: {} vs {}", flight_info1.endpoint.len(), flight_info2.endpoint.len()); - println!(" - Ticket bytes: {} vs {}", ticket1.ticket.len(), ticket2.ticket.len()); + println!( + " - Schema bytes: {} vs {}", + flight_info1.schema.len(), + flight_info2.schema.len() + ); + println!( + " - Endpoints: {} vs {}", + flight_info1.endpoint.len(), + flight_info2.endpoint.len() + ); + println!( + " - Ticket bytes: {} vs {}", + ticket1.ticket.len(), + ticket2.ticket.len() + ); } // TODO: Add tests for regular (non-explain) queries diff --git a/src/query_planner.rs b/src/query_planner.rs index 57d1f65..668b1a4 100644 --- a/src/query_planner.rs +++ b/src/query_planner.rs @@ -10,7 +10,7 @@ use datafusion::{ use crate::{ explain::{DistributedExplainExec, is_explain_query}, - k8s::get_worker_addresses, + worker_discovery::get_worker_addresses, logging::debug, planning::{ distribute_stages, @@ -162,7 +162,7 @@ impl QueryPlanner { // Create dummy addresses for EXPLAIN (no real workers needed) let mut dummy_addrs = std::collections::HashMap::new(); let mut partition_addrs = std::collections::HashMap::new(); - partition_addrs.insert(0u64, vec![("explain_local".to_string(), "local".to_string())]); + partition_addrs.insert(0u64, vec![]); dummy_addrs.insert(0u64, partition_addrs); Ok(QueryPlan { @@ -329,4 +329,4 @@ mod tests { } } } -} \ No newline at end of file +} diff --git a/src/result.rs b/src/result.rs index 43536ff..e549257 100644 --- a/src/result.rs +++ b/src/result.rs @@ -1,5 +1,8 @@ +use arrow_flight::error::FlightError; use thiserror::Error; +use crate::vocab::Host; + #[derive(Debug, Error)] pub enum DFRayError { #[error("Internal Arrow error: {0}")] @@ -7,10 +10,13 @@ pub enum DFRayError { #[error("Internal DataFusion error: {0}")] DataFusionError(#[from] datafusion::error::DataFusionError), + #[error("Arrow Flight error: {0}")] + FlightError(#[from] FlightError), + #[error("Failed to communicate with worker: {0}")] // the name of the worker // improve this as we swallow the error here - WorkerCommunicationError(String), + WorkerCommunicationError(Host), #[error(transparent)] Other(#[from] anyhow::Error), diff --git a/src/stage_reader.rs b/src/stage_reader.rs index 59cd948..7214670 100644 --- a/src/stage_reader.rs +++ b/src/stage_reader.rs @@ -152,7 +152,7 @@ impl ExecutionPlan for DFRayStageReaderExec { )) })?? .iter() - .map(|(name, addr)| get_client(name, addr)) + .map(|host| get_client(host)) .collect::>>()?; trace!("got clients. {name} num clients: {}", clients.len()); @@ -194,7 +194,7 @@ impl ExecutionPlan for DFRayStageReaderExec { }, Err(e) => { error = true; - yield internal_err!("{name} - {ctx_name_capture} Error getting flight stream from {}: {e}", client.destination); + yield internal_err!("{name} - {ctx_name_capture} Error getting flight stream from {}: {e}", client.host); } } } @@ -252,7 +252,6 @@ fn make_flight_metadata_saver_stream( yield Err(FlightError::DecodeError(format!( "{name} Failed to decode flight data metadata: {e:#?}" ))); - done = true; } } @@ -261,10 +260,13 @@ fn make_flight_metadata_saver_stream( // it done = true; } else { + // just normal data, yield to consumer + trace!("received normal data"); yield Ok(flight_data.inner); } }, (false, Some(Err(e))) => { + // propagate this error yield Err(e); } (false, None) => { @@ -275,10 +277,11 @@ fn make_flight_metadata_saver_stream( } (true,_) => { // already done, ignore any further data - trace!("{name} Ignoring further data from flight stream, already done"); + error!("{name} Unreachable block. flight stream already done"); } } } + trace!("Done with flight stream {name} - no more data to yield"); }; let trailing_stream = diff --git a/src/util.rs b/src/util.rs index 066be30..b9907d2 100644 --- a/src/util.rs +++ b/src/util.rs @@ -15,7 +15,7 @@ use arrow::{ error::ArrowError, ipc::{convert::fb_to_schema, root_as_message}, }; -use arrow_flight::{FlightClient, FlightData, Ticket, decode::FlightRecordBatchStream}; +use arrow_flight::{decode::FlightRecordBatchStream, FlightClient, FlightData, Ticket}; use async_stream::stream; use bytes::Bytes; use datafusion::{ @@ -25,22 +25,16 @@ use datafusion::{ }, datasource::{physical_plan::FileScanConfig, source::DataSourceExec}, error::DataFusionError, - execution::{RecordBatchStream, SendableRecordBatchStream, object_store::ObjectStoreUrl}, + execution::{object_store::ObjectStoreUrl, RecordBatchStream, SendableRecordBatchStream}, physical_plan::{ - ExecutionPlan, - ExecutionPlanProperties, - displayable, - stream::RecordBatchStreamAdapter, + displayable, stream::RecordBatchStreamAdapter, ExecutionPlan, ExecutionPlanProperties, }, prelude::SessionContext, }; use datafusion_proto::physical_plan::AsExecutionPlan; -use futures::{Stream, StreamExt, stream::BoxStream}; +use futures::{stream::BoxStream, Stream, StreamExt}; use object_store::{ - ObjectStore, - aws::AmazonS3Builder, - gcp::GoogleCloudStorageBuilder, - http::HttpBuilder, + aws::AmazonS3Builder, gcp::GoogleCloudStorageBuilder, http::HttpBuilder, ObjectStore, }; use parking_lot::RwLock; use prost::Message; @@ -57,7 +51,7 @@ use crate::{ protobuf::StageAddrs, result::Result, stage_reader::DFRayStageReaderExec, - vocab::Addrs, + vocab::{Addrs, Host}, }; struct Spawner { @@ -160,11 +154,7 @@ pub fn get_addrs(stage_addrs: &StageAddrs) -> Result { for (stage_id, partition_addrs) in stage_addrs.stage_addrs.iter() { let mut stage_addrs = HashMap::new(); for (partition, hosts) in partition_addrs.partition_addrs.iter() { - let mut host_addrs = vec![]; - for host in &hosts.hosts { - host_addrs.push((host.name.clone(), host.addr.clone())); - } - stage_addrs.insert(*partition, host_addrs); + stage_addrs.insert(*partition, hosts.hosts.clone()); } addrs.insert(*stage_id, stage_addrs); } @@ -306,10 +296,8 @@ pub fn reporting_stream( } pub struct ProcessorClient { - /// The name of the processor we are connecting to - pub destination: String, - /// the address we are connecting to - addr: String, + /// the host we are connecting to + pub(crate) host: Host, /// The flight client to the processor inner: FlightClient, /// the channel cache in the factory @@ -318,14 +306,12 @@ pub struct ProcessorClient { impl ProcessorClient { pub fn new( - name: String, + host: Host, inner: FlightClient, channels: Arc>>, - addr: String, ) -> Self { Self { - destination: name, - addr, + host, inner, channels, } @@ -338,8 +324,8 @@ impl ProcessorClient { let stream = self.inner.do_get(ticket).await .inspect_err(|e| { error!("Error in do_get for processor {}: {e:?}. - Considering this channel poisoned and removing it from ProcessorClientFactory cache", self.destination); - self.channels.write().remove(&self.addr); + Considering this channel poisoned and removing it from ProcessorClientFactory cache", self.host); + self.channels.write().remove(&self.host.addr); })?; Ok(stream) @@ -354,9 +340,9 @@ impl ProcessorClient { "Error in do_action for processor {}: {e:?}. Considering this channel poisoned and removing it from ProcessorClientFactory \ cache", - self.destination + self.host ); - self.channels.write().remove(&self.addr); + self.channels.write().remove(&self.host.addr); })?; Ok(result) @@ -374,7 +360,7 @@ impl ProcessorClientFactory { } } - pub fn get_client(&self, name: &str, addr: &str) -> Result { + pub fn get_client(&self, host: &Host) -> Result { // ideally we want to reuse channels as Tonic encourages cloning them when // you can. This could would allow us to lazily create channels and keep // them around so that on subsequent requests to the same address, we won't @@ -397,18 +383,18 @@ impl ProcessorClientFactory { // higher number of processors, even 2. I'm going to leave in the // functionality for cached channels for now - let url = format!("http://{addr}"); + let url = format!("http://{}", host.addr); - let maybe_chan = self.channels.read().get(addr).cloned(); + let maybe_chan = self.channels.read().get(&host.addr).cloned(); let chan = match maybe_chan { Some(chan) => { - debug!("ProcessorFactory using cached channel for {addr}"); + debug!("ProcessorFactory using cached channel for {host}"); chan } None => { - let addr_c = addr.to_owned(); + let host_str = host.to_string(); let fut = async move { - trace!("ProcessorFactory connecting to {addr_c}"); + trace!("ProcessorFactory connecting to {host_str}"); Channel::from_shared(url.clone()) .map_err(|e| { internal_datafusion_err!("ProcessorFactory invalid url {e:#?}") @@ -427,30 +413,31 @@ impl ProcessorClientFactory { "ProcessorFactory Cannot wait for channel connect future {e:#?}" ) })??; - trace!("ProcessorFactory connected to {addr}"); - self.channels.write().insert(addr.to_string(), chan.clone()); + trace!("ProcessorFactory connected to {host}"); + self.channels + .write() + .insert(host.addr.to_string(), chan.clone()); chan } }; - debug!("ProcessorFactory have channel now for {addr}"); + debug!("ProcessorFactory have channel now for {host}"); let flight_client = FlightClient::new(chan); - debug!("ProcessorFactory made flight client for {addr}"); + debug!("ProcessorFactory made flight client for {host}"); Ok(ProcessorClient::new( - name.to_owned(), + host.clone(), flight_client, self.channels.clone(), - addr.to_owned(), )) } } static FACTORY: OnceLock = OnceLock::new(); -pub fn get_client(name: &str, addr: &str) -> Result { +pub fn get_client(host: &Host) -> Result { let factory = FACTORY.get_or_init(ProcessorClientFactory::new); - factory.get_client(name, addr) + factory.get_client(host) } /// Copied from datafusion_physical_plan::union as its useful and not public @@ -487,7 +474,10 @@ impl Stream for CombinedRecordBatchStream { let stream = self.entries.get_mut(idx).unwrap(); match Pin::new(stream).poll_next(cx) { - Ready(Some(val)) => return Ready(Some(val)), + Ready(Some(val)) => { + trace!("Combined stream got {:?}", val); + return Ready(Some(val)); + } Ready(None) => { // Remove the entry self.entries.swap_remove(idx); @@ -529,7 +519,7 @@ fn print_node(plan: &Arc, indent: usize, output: &mut String) "[ output_partitions: {}]{:>indent$}{}", plan.output_partitioning().partition_count(), "", - displayable(plan.as_ref()).one_line(), + displayable(plan.as_ref()).set_show_schema(true).one_line(), indent = indent )); diff --git a/src/vocab.rs b/src/vocab.rs index 969332f..8af334f 100644 --- a/src/vocab.rs +++ b/src/vocab.rs @@ -1,8 +1,13 @@ -use std::{collections::HashMap, sync::Arc}; +use std::{collections::HashMap, fmt::Display, sync::Arc}; use parking_lot::Mutex; -use crate::protobuf::AnnotatedTaskOutput; +pub use crate::protobuf::AnnotatedTaskOutput; +pub use crate::protobuf::Host; +pub use crate::protobuf::Hosts; +pub use crate::protobuf::PartitionAddrs; +pub use crate::protobuf::StageAddrs; +pub use crate::protobuf::StageData; /// a map of stage_id, partition to a list (name,endpoint address) that can /// serve this (stage_id, and partition). It is assumed that to consume a @@ -10,7 +15,7 @@ use crate::protobuf::AnnotatedTaskOutput; /// merge the results. /// /// This is on a per query basis -pub type Addrs = HashMap>>; +pub type Addrs = HashMap>>; /// used to hold an Addrs as an extenstion for datafusion SessionContext pub(crate) struct CtxStageAddrs(pub Addrs); @@ -23,3 +28,9 @@ pub(crate) struct CtxPartitionGroup(pub Vec); #[derive(Default)] pub(crate) struct CtxAnnotatedOutputs(pub Arc>>); + +impl Display for Host { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}: {}", self.name, self.addr) + } +} From 116964d7757c9220f5e4fabfbd041789d8804b18 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 10 Jul 2025 09:45:13 -0400 Subject: [PATCH 3/6] remove conditional logic for explain from planning and execution paths --- scripts/launch_tpch_cluster.sh | 4 +- src/analyze.rs | 67 ++++- src/codec.rs | 1 - src/explain.rs | 103 ++++--- src/flight_handlers.rs | 508 -------------------------------- src/isolator.rs | 20 +- src/lib.rs | 4 +- src/main.rs | 3 +- src/physical.rs | 33 +-- src/planning.rs | 166 +++++++---- src/processor_service.rs | 296 ++++++++++++------- src/proto/generated/protobuf.rs | 18 +- src/proxy_service.rs | 384 ++++++++++++------------ src/query_planner.rs | 319 ++++---------------- src/stage_reader.rs | 21 +- src/test_utils.rs | 113 ------- src/util.rs | 11 + src/vocab.rs | 9 +- 18 files changed, 686 insertions(+), 1394 deletions(-) delete mode 100644 src/flight_handlers.rs delete mode 100644 src/test_utils.rs diff --git a/scripts/launch_tpch_cluster.sh b/scripts/launch_tpch_cluster.sh index 5cbf662..6cdf045 100755 --- a/scripts/launch_tpch_cluster.sh +++ b/scripts/launch_tpch_cluster.sh @@ -150,12 +150,12 @@ for ((i = 0; i < NUM_WORKERS; i++)); do #env DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" ./target/release/distributed-datafusion --mode worker --port $PORT >"$LOG_FILE" 2>&1 & env RUST_BACKTRACE=1 DATAFUSION_RAY_LOG_LEVEL="$DATAFUSION_RAY_LOG_LEVEL" DFRAY_TABLES="$DFRAY_TABLES" ./target/debug/distributed-datafusion --mode worker --port $PORT >"$LOG_FILE" 2>&1 & WORKER_PIDS[$i]=$! - WORKER_ADDRESSES[$i]="${WORKER_NAME}/localhost:${PORT}" + WORKER_ADDRESSES[$i]="localhost:${PORT}" done # Give workers time to start echo "Waiting for workers to initialize..." -sleep 2 +sleep 5 # Construct worker addresses string for proxy WORKER_ADDRESSES_STR=$( diff --git a/src/analyze.rs b/src/analyze.rs index 77438ce..20ba5c5 100644 --- a/src/analyze.rs +++ b/src/analyze.rs @@ -2,7 +2,7 @@ use std::{fmt::Formatter, sync::Arc}; use arrow::{ array::{RecordBatch, StringBuilder}, - datatypes::{DataType, Field, Schema, SchemaRef}, + datatypes::{DataType, Field, Schema}, }; use datafusion::{ error::{DataFusionError, Result}, @@ -12,13 +12,16 @@ use datafusion::{ display::DisplayableExecutionPlan, execution_plan::{Boundedness, EmissionType}, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, - PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, }, }; use futures::StreamExt; -use crate::{logging::debug, protobuf::AnnotatedTaskOutput, vocab::CtxAnnotatedOutputs}; +use crate::{ + logging::{debug, trace}, + protobuf::AnnotatedTaskOutput, + vocab::{CtxAnnotatedOutputs, CtxHost, CtxPartitionGroup, CtxStageId}, +}; #[derive(Debug)] pub struct DistributedAnalyzeExec { @@ -197,6 +200,40 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { "DistributedAnalyzeRootExec expects only partition 0" ); + let host = context + .session_config() + .get_extension::() + .ok_or_else(|| { + DataFusionError::Internal( + "CtxHost not set in session config for DistributedAnalyzeRootExec".to_string(), + ) + })? + .0 + .clone(); + + let stage_id = context + .session_config() + .get_extension::() + .ok_or_else(|| { + DataFusionError::Internal( + "CtxStageId not set in session config for DistributedAnalyzeRootExec" + .to_string(), + ) + })? + .0; + + let partition_group = context + .session_config() + .get_extension::() + .ok_or_else(|| { + DataFusionError::Internal( + "CtxPartitionGroup not set in session config for DistributedAnalyzeRootExec" + .to_string(), + ) + })? + .0 + .clone(); + let mut input_stream = self.input.execute(partition, context)?; let schema_clone = self.schema().clone(); @@ -206,13 +243,11 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { // root of the distributed analyze so we can discard the results just like // regular AnalyzeExec let mut done = false; - let mut total_rows = 0; while !done { match input_stream.next().await.transpose() { Ok(Some(batch)) => { // we consume the batch, yum. - debug!("consumed {} ", batch.num_rows()); - total_rows += batch.num_rows(); + trace!("consumed {} ", batch.num_rows()); } Ok(None) => done = true, Err(e) => { @@ -223,14 +258,16 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { let annotated_plan = fmt_plan(); let toutput = AnnotatedTaskOutput { plan: annotated_plan, - host: None, - stage_id: 0, - partition_group: vec![0], + host: Some(host), + stage_id, + partition_group, }; let mut tasks = task_outputs.lock(); tasks.push(toutput); + tasks.sort_by_key(|t| (t.stage_id, t.partition_group.clone())); + let mut task_builder = StringBuilder::with_capacity(1, 1024); let mut plan_builder = StringBuilder::with_capacity(1, 1024); task_builder.append_value("Task"); @@ -238,8 +275,14 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { for task_output in tasks.iter() { task_builder.append_value(format!( - "Task: Stage {}, Partitions {:?}", - task_output.stage_id, task_output.partition_group + "Task: Stage {}, Partitions {:?}\nHost: {}", + task_output.stage_id, + task_output.partition_group, + task_output + .host + .as_ref() + .map(|h| h.to_string()) + .unwrap_or("Unknown".to_string()) )); plan_builder.append_value(&task_output.plan); } diff --git a/src/codec.rs b/src/codec.rs index 2b732a8..390baf4 100644 --- a/src/codec.rs +++ b/src/codec.rs @@ -3,7 +3,6 @@ use std::sync::Arc; use arrow::datatypes::Schema; use datafusion::{ common::{internal_datafusion_err, internal_err, Result}, - datasource::source::DataSourceExec, execution::FunctionRegistry, physical_plan::{displayable, ExecutionPlan}, }; diff --git a/src/explain.rs b/src/explain.rs index 4be4101..1c7a406 100644 --- a/src/explain.rs +++ b/src/explain.rs @@ -15,27 +15,29 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::Formatter, - sync::Arc, -}; +use std::{any::Any, fmt::Formatter, sync::Arc}; -use arrow::{ - array::StringArray, - datatypes::SchemaRef, - record_batch::RecordBatch, -}; +use anyhow::Context; +use arrow::{array::StringArray, datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion::{ execution::TaskContext, + logical_expr::LogicalPlan, + physical_expr::EquivalenceProperties, physical_plan::{ + displayable, execution_plan::{Boundedness, EmissionType}, memory::MemoryStream, - ExecutionPlan, Partitioning, - PlanProperties, DisplayAs, DisplayFormatType, - SendableRecordBatchStream, displayable, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, + SendableRecordBatchStream, }, - physical_expr::EquivalenceProperties, + prelude::SessionContext, +}; + +use crate::{ + planning::{get_ctx, logical_planning}, + result::Result, + util::bytes_to_physical_plan, + vocab::DDTask, }; /// Custom distributed EXPLAIN execution plan that also returns distributed plan and stages @@ -45,7 +47,7 @@ pub struct DistributedExplainExec { logical_plan: String, physical_plan: String, distributed_plan: String, - distributed_stages: String, + distributed_tasks: String, properties: PlanProperties, } @@ -55,7 +57,7 @@ impl DistributedExplainExec { logical_plan: String, physical_plan: String, distributed_plan: String, - distributed_stages: String, + distributed_tasks: String, ) -> Self { // properties required by the ExecutionPlan trait let properties = PlanProperties::new( @@ -70,7 +72,7 @@ impl DistributedExplainExec { logical_plan, physical_plan, distributed_plan, - distributed_stages, + distributed_tasks, properties, } } @@ -87,30 +89,33 @@ impl DistributedExplainExec { &self.distributed_plan } - pub fn distributed_stages(&self) -> &str { - &self.distributed_stages + pub fn distributed_tasks(&self) -> &str { + &self.distributed_tasks } - /// Format distributed stages for display - pub fn format_distributed_stages(stages: &[crate::planning::DFRayStage]) -> String { + /// Format distributed tasks for display + pub fn format_distributed_tasks(tasks: &[DDTask]) -> Result { let mut result = String::new(); - for (i, stage) in stages.iter().enumerate() { - result.push_str(&format!("Stage {}:\n", stage.stage_id)); - result.push_str(&format!(" Partition Groups: {:?}\n", stage.partition_groups)); - result.push_str(&format!(" Full Partitions: {}\n", stage.full_partitions)); + for (i, task) in tasks.iter().enumerate() { + let plan = bytes_to_physical_plan(&SessionContext::new(), &task.plan_bytes) + .context(format!("unable to decode task plan for formatted output"))?; + + result.push_str(&format!("Stage {}:\n", task.stage_id)); + result.push_str(&format!(" Partition Group: {:?}\n", task.partition_group)); + result.push_str(&format!(" Full Partitions: {}\n", task.full_partitions)); result.push_str(" Plan:\n"); - let plan_display = format!("{}", displayable(stage.plan.as_ref()).indent(true)); + let plan_display = format!("{}", displayable(plan.as_ref()).indent(true)); for line in plan_display.lines() { result.push_str(&format!(" {}\n", line)); } - if i < stages.len() - 1 { + if i < tasks.len() - 1 { result.push('\n'); } } if result.is_empty() { - result.push_str("No distributed stages generated"); + result.push_str("No distributed tasks generated"); } - result + Ok(result) } } @@ -150,29 +155,28 @@ impl ExecutionPlan for DistributedExplainExec { _context: Arc, ) -> datafusion::error::Result { let schema = self.schema.clone(); - + // Create the result data with our 4 plan types let plan_types = StringArray::from(vec![ - "logical_plan", - "physical_plan", - "distributed_plan", - "distributed_stages" + "logical_plan", + "physical_plan", + "distributed_plan", + "distributed_tasks", ]); let plans = StringArray::from(vec![ self.logical_plan.as_str(), self.physical_plan.as_str(), self.distributed_plan.as_str(), - self.distributed_stages.as_str(), + self.distributed_tasks.as_str(), ]); - let batch = RecordBatch::try_new( - schema.clone(), - vec![Arc::new(plan_types), Arc::new(plans)], - ).map_err(|e| datafusion::error::DataFusionError::ArrowError(e, None))?; + let batch = + RecordBatch::try_new(schema.clone(), vec![Arc::new(plan_types), Arc::new(plans)]) + .map_err(|e| datafusion::error::DataFusionError::ArrowError(e, None))?; // Use MemoryStream which is designed for DataFusion execution plans let stream = MemoryStream::try_new(vec![batch], schema, None)?; - + Ok(Box::pin(stream)) } @@ -182,18 +186,19 @@ impl ExecutionPlan for DistributedExplainExec { } /// Check if this is an EXPLAIN query (but not EXPLAIN ANALYZE) -/// +/// /// This function distinguishes between: /// - EXPLAIN queries (returns true) - show plan information only /// - EXPLAIN ANALYZE queries (returns false) - execute and show runtime stats /// - Regular queries (returns false) - normal query execution -pub fn is_explain_query(query: &str) -> bool { - let query_upper = query.trim().to_uppercase(); - // Must start with "EXPLAIN" followed by whitespace or end of string - let is_explain = query_upper.starts_with("EXPLAIN") && - (query_upper.len() == 7 || query_upper.chars().nth(7).is_some_and(|c| c.is_whitespace())); - let is_explain_analyze = query_upper.starts_with("EXPLAIN ANALYZE"); - is_explain && !is_explain_analyze +pub async fn is_explain_query(sql: &str) -> Result { + let ctx = get_ctx().map_err(|e| anyhow!("Could not create context: {e}"))?; + let logical_plan = logical_planning(sql, &ctx).await?; + + match logical_plan { + LogicalPlan::Explain(_) => Ok(true), + _ => Ok(false), // Not an EXPLAIN plan + } } #[cfg(test)] @@ -223,7 +228,7 @@ mod tests { // Test edge cases assert!(!is_explain_query("")); assert!(!is_explain_query(" ")); - assert!(!is_explain_query("EXPLAINSELECT")); // No space + assert!(!is_explain_query("EXPLAINSELECT")); // No space assert!(is_explain_query("EXPLAIN")); // Just EXPLAIN } } diff --git a/src/flight_handlers.rs b/src/flight_handlers.rs deleted file mode 100644 index 673857d..0000000 --- a/src/flight_handlers.rs +++ /dev/null @@ -1,508 +0,0 @@ -use std::sync::Arc; - -use arrow::datatypes::{Schema, SchemaRef}; -use arrow_flight::{ - encode::FlightDataEncoderBuilder, - error::FlightError, - sql::{ProstMessageExt, TicketStatementQuery}, - FlightEndpoint, FlightInfo, Ticket, -}; -use datafusion::physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, ExecutionPlan, Partitioning, -}; -use futures::TryStreamExt; -use prost::Message; -use tonic::{Response, Status}; - -use crate::{ - explain::DistributedExplainExec, - logging::{debug, trace}, - planning::{add_ctx_extentions, get_ctx}, - protobuf::{DistributedExplainExecNode, TicketStatementData}, - query_planner::QueryPlanner, - result::Result, - stage_reader::DFRayStageReaderExec, - util::{display_plan_with_partition_counts, get_addrs}, - vocab::Addrs, -}; - -/// Handler for Arrow Flight SQL requests and responses -pub struct FlightRequestHandler { - pub planner: QueryPlanner, -} - -impl FlightRequestHandler { - pub fn new(planner: QueryPlanner) -> Self { - Self { planner } - } - - /// Create a FlightInfo response with the given ticket data - pub fn create_flight_info_response( - &self, - query_id: String, - final_addrs: Addrs, - final_stage_id: u64, - schema: SchemaRef, - explain_data: Option, - ) -> Result { - let mut flight_info = FlightInfo::new() - .try_with_schema(&schema) - .map_err(|e| Status::internal(format!("Could not create flight info {e:?}")))?; - - let ticket_data = TicketStatementData { - query_id, - stage_id: final_stage_id, - stage_addrs: Some(final_addrs.into()), - schema: Some( - schema - .try_into() - .map_err(|e| Status::internal(format!("Could not convert schema {e:?}")))?, - ), - explain_data, - }; - - let ticket = Ticket::new( - TicketStatementQuery { - statement_handle: ticket_data.encode_to_vec().into(), - } - .as_any() - .encode_to_vec(), - ); - - let endpoint = FlightEndpoint::new().with_ticket(ticket); - flight_info = flight_info.with_endpoint(endpoint); - - Ok(flight_info) - } - - /// Handle EXPLAIN query requests by preparing plans for visualization. - /// - /// EXPLAIN queries return comprehensive plan information including logical, physical, - /// distributed plan, and execution stages for analysis and debugging purposes. - pub async fn handle_explain_request( - &self, - query: &str, - ) -> Result, Status> { - let plans = self - .planner - .prepare_explain(query) - .await - .map_err(|e| Status::internal(format!("Could not prepare EXPLAIN query {e:?}")))?; - - debug!("get flight info: EXPLAIN query id {}", plans.query_id); - - let explain_data = plans.explain_data.map(|data| DistributedExplainExecNode { - schema: data.schema().as_ref().try_into().ok(), - logical_plan: data.logical_plan().to_string(), - physical_plan: data.physical_plan().to_string(), - distributed_plan: data.distributed_plan().to_string(), - distributed_stages: data.distributed_stages().to_string(), - }); - - let flight_info = self.create_flight_info_response( - plans.query_id, - plans.worker_addresses, - plans.final_stage_id, - plans.schema, - explain_data, - )?; - - trace!("get_flight_info_statement done for EXPLAIN"); - Ok(Response::new(flight_info)) - } - - /// Handle query requests by preparing execution plans and stages. - /// - /// Query focus on execution readiness, returning only the essential - /// metadata needed to execute the distributed query plan. - pub async fn handle_query_request(&self, query: &str) -> Result, Status> { - let query_plan = self - .planner - .prepare_query(query) - .await - .map_err(|e| Status::internal(format!("Could not prepare query {e:?}")))?; - - debug!("get flight info: query id {}", query_plan.query_id); - - let flight_info = self.create_flight_info_response( - query_plan.query_id, - query_plan.worker_addresses, - query_plan.final_stage_id, - query_plan.schema, - None, // Regular queries don't have explain data - )?; - - trace!("get_flight_info_statement done"); - Ok(Response::new(flight_info)) - } - - /// Handle execution of EXPLAIN statement queries. - /// - /// This function does not execute the plan but returns all plans we want to display to the user. - pub async fn handle_explain_statement_execution( - &self, - tsd: TicketStatementData, - remote_addr: &str, - ) -> Result, Status> { - let explain_data = tsd.explain_data.as_ref().ok_or_else(|| { - Status::internal("No explain_data in TicketStatementData for EXPLAIN query") - })?; - - let schema: Schema = explain_data - .schema - .as_ref() - .ok_or_else(|| Status::internal("No schema in ExplainData"))? - .try_into() - .map_err(|e| Status::internal(format!("Cannot convert schema {e}")))?; - - let explain_plan = Arc::new(DistributedExplainExec::new( - Arc::new(schema), - explain_data.logical_plan.clone(), - explain_data.physical_plan.clone(), - explain_data.distributed_plan.clone(), - explain_data.distributed_stages.clone(), - )) as Arc; - - debug!( - "EXPLAIN request for query_id {} from {} explain plan:\n{}", - tsd.query_id, - remote_addr, - display_plan_with_partition_counts(&explain_plan) - ); - - // Create dummy addresses for EXPLAIN execution - let mut dummy_addrs = std::collections::HashMap::new(); - let mut partition_addrs = std::collections::HashMap::new(); - partition_addrs.insert(0u64, vec![]); - dummy_addrs.insert(0u64, partition_addrs); - - self.execute_plan_and_build_stream(explain_plan, tsd.query_id, dummy_addrs) - .await - } - - /// Handle execution of regular statement queries - /// - /// This function executes the plan and returns the results to the client. - pub async fn handle_regular_statement_execution( - &self, - tsd: TicketStatementData, - remote_addr: &str, - ) -> Result, Status> { - let schema: Schema = tsd - .schema - .as_ref() - .ok_or_else(|| Status::internal("No schema in TicketStatementData"))? - .try_into() - .map_err(|e| Status::internal(format!("Cannot convert schema {e}")))?; - - // Create an Addrs from the final stage information in the tsd - let stage_addrs = tsd.stage_addrs.ok_or_else(|| { - Status::internal("No stages_addrs in TicketStatementData, cannot proceed") - })?; - - let addrs: Addrs = get_addrs(&stage_addrs).map_err(|e| { - Status::internal(format!("Cannot get addresses from stage_addrs {e:?}")) - })?; - - trace!("calculated addrs: {:?}", addrs); - - // Validate that addrs contains exactly one stage - self.validate_single_stage_addrs(&addrs, tsd.stage_id)?; - - let stage_partition_addrs = addrs.get(&tsd.stage_id).ok_or_else(|| { - Status::internal(format!( - "No partition addresses found for stage_id {}", - tsd.stage_id - )) - })?; - - let plan = Arc::new( - DFRayStageReaderExec::try_new( - Partitioning::UnknownPartitioning(stage_partition_addrs.len()), - Arc::new(schema), - tsd.stage_id, - ) - // TODO: revisit this to allow for consuming a particular partition - .map(|stg| CoalescePartitionsExec::new(Arc::new(stg))) - .map_err(|e| Status::internal(format!("Unexpected error {e}")))?, - ) as Arc; - - debug!( - "request for query_id {} from {} reader plan:\n{}", - tsd.query_id, - remote_addr, - display_plan_with_partition_counts(&plan) - ); - - self.execute_plan_and_build_stream(plan, tsd.query_id, addrs) - .await - } - - /// Validate that addresses contain exactly one stage with the expected stage_id - pub fn validate_single_stage_addrs( - &self, - addrs: &Addrs, - expected_stage_id: u64, - ) -> Result<(), Status> { - if addrs.len() != 1 { - return Err(Status::internal(format!( - "Expected exactly one stage in addrs, got {}", - addrs.len() - ))); - } - if !addrs.contains_key(&expected_stage_id) { - return Err(Status::internal(format!( - "No addresses found for stage_id {} in addrs", - expected_stage_id - ))); - } - Ok(()) - } - - /// Execute a plan and build the response stream. - /// - /// This function handles the common execution logic for both regular queries (which return data) - /// and EXPLAIN queries (which return plan information as text). - pub async fn execute_plan_and_build_stream( - &self, - plan: Arc, - query_id: String, - addrs: Addrs, - ) -> Result, Status> { - let mut ctx = - get_ctx().map_err(|e| Status::internal(format!("Could not create context {e:?}")))?; - - add_ctx_extentions(&mut ctx, "proxy", &query_id, addrs, None) - .map_err(|e| Status::internal(format!("Could not add context extensions {e:?}")))?; - - // TODO: revisit this to allow for consuming a partitular partition - trace!("calling execute plan"); - let partition = 0; - let stream = plan - .execute(partition, ctx.task_ctx()) - .map_err(|e| { - Status::internal(format!( - "Error executing plan for query_id {} partition {}: {e:?}", - query_id, partition - )) - })? - .map_err(|e| FlightError::ExternalError(Box::new(e))); - - let out_stream = FlightDataEncoderBuilder::new() - .build(stream) - .map_err(move |e| Status::internal(format!("Unexpected error building stream {e:?}"))); - - Ok(Response::new(Box::pin(out_stream))) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::{ - test_utils::explain_test_helpers::{ - create_explain_ticket_statement_data, create_test_flight_handler, - verify_explain_stream_results, - }, - vocab::Host, - }; - use std::collections::HashMap; - - // ////////////////////////////////////////////////////////////// - // Test helper functions - // ////////////////////////////////////////////////////////////// - - /// Create test worker addresses - fn create_test_addrs() -> Addrs { - let mut addrs = HashMap::new(); - let mut stage_addrs = HashMap::new(); - stage_addrs.insert( - 1u64, - vec![ - Host { - name: "worker1".to_string(), - addr: "localhost:8001".to_string(), - }, - Host { - name: "worker2".to_string(), - addr: "localhost:8002".to_string(), - }, - ], - ); - addrs.insert(1u64, stage_addrs); - addrs - } - - // ////////////////////////////////////////////////////////////// - // Unit tests - // ////////////////////////////////////////////////////////////// - - #[test] - fn test_validate_single_stage_addrs_success() { - let handler = create_test_flight_handler(); - let addrs = create_test_addrs(); - - // Should succeed with valid single stage - let result = handler.validate_single_stage_addrs(&addrs, 1); - assert!(result.is_ok()); - } - - #[test] - fn test_validate_single_stage_addrs_multiple_stages() { - let handler = create_test_flight_handler(); - - // Create addresses with multiple stages - let mut addrs = HashMap::new(); - let mut stage1_addrs = HashMap::new(); - stage1_addrs.insert( - 1u64, - vec![Host { - name: "worker1".to_string(), - addr: "localhost:8001".to_string(), - }], - ); - let mut stage2_addrs = HashMap::new(); - stage2_addrs.insert( - 2u64, - vec![Host { - name: "worker1".to_string(), - addr: "localhost:8002".to_string(), - }], - ); - addrs.insert(1u64, stage1_addrs); - addrs.insert(2u64, stage2_addrs); - - // Should fail with multiple stages - let result = handler.validate_single_stage_addrs(&addrs, 1); - assert!(result.is_err()); - - if let Err(status) = result { - assert!(status.message().contains("Expected exactly one stage")); - } - } - - #[test] - fn test_validate_single_stage_addrs_wrong_stage_id() { - let handler = create_test_flight_handler(); - let addrs = create_test_addrs(); // Contains stage_id 1 - - // Should fail when looking for non-existent stage - let result = handler.validate_single_stage_addrs(&addrs, 999); - assert!(result.is_err()); - - if let Err(status) = result { - assert!(status - .message() - .contains("No addresses found for stage_id 999")); - } - } - - #[test] - fn test_validate_single_stage_addrs_empty() { - let handler = create_test_flight_handler(); - let addrs: Addrs = HashMap::new(); - - // Should fail with empty addresses - let result = handler.validate_single_stage_addrs(&addrs, 1); - assert!(result.is_err()); - - if let Err(status) = result { - assert!(status.message().contains("Expected exactly one stage")); - } - } - - // ////////////////////////////////////////////////////////////// - // Handler core function tests - // ////////////////////////////////////////////////////////////// - - #[tokio::test] - async fn test_handle_explain_request() { - let handler = create_test_flight_handler(); - let query = "EXPLAIN SELECT 1 as test_col, 'hello' as text_col"; - - let result = handler.handle_explain_request(query).await; - assert!(result.is_ok()); - - let response = result.unwrap(); - let flight_info = response.into_inner(); - - // Verify FlightInfo structure - assert!(!flight_info.schema.is_empty()); - assert_eq!(flight_info.endpoint.len(), 1); - assert!(flight_info.endpoint[0].ticket.is_some()); - - // Verify that ticket has content (encoded TicketStatementData) - let ticket = flight_info.endpoint[0].ticket.as_ref().unwrap(); - assert!(!ticket.ticket.is_empty()); - - println!( - "✓ FlightInfo created successfully with {} schema bytes and ticket with {} bytes", - flight_info.schema.len(), - ticket.ticket.len() - ); - } - - #[tokio::test] - async fn test_handle_explain_request_invalid_query() { - let handler = create_test_flight_handler(); - - // Test with EXPLAIN ANALYZE (should fail) - let query = "EXPLAIN ANALYZE SELECT 1"; - let result = handler.handle_explain_request(query).await; - assert!(result.is_err()); - - let error = result.unwrap_err(); - assert_eq!(error.code(), tonic::Code::Internal); - assert!(error.message().contains("Could not prepare EXPLAIN query")); - } - - #[tokio::test] - async fn test_handle_explain_statement_execution() { - let handler = create_test_flight_handler(); - - // First prepare an EXPLAIN query to get the ticket data structure - let query = "EXPLAIN SELECT 1 as test_col"; - let plans = handler.planner.prepare_explain(query).await.unwrap(); - - // Create the TicketStatementData that would be sent to do_get_statement - let tsd = create_explain_ticket_statement_data(plans); - - // Test the execution - let result = handler - .handle_explain_statement_execution(tsd, "test_remote") - .await; - assert!(result.is_ok()); - - let response = result.unwrap(); - let stream = response.into_inner(); - - // Use shared verification function - verify_explain_stream_results(stream).await; - } - - #[tokio::test] - async fn test_handle_explain_statement_execution_missing_explain_data() { - let handler = create_test_flight_handler(); - - // Create TicketStatementData without explain_data (should fail) - let tsd = TicketStatementData { - query_id: "test_query".to_string(), - stage_id: 0, - stage_addrs: None, - schema: None, - explain_data: None, - }; - - let result = handler - .handle_explain_statement_execution(tsd, "test_remote") - .await; - assert!(result.is_err()); - - if let Err(error) = result { - assert_eq!(error.code(), tonic::Code::Internal); - assert!(error - .message() - .contains("No explain_data in TicketStatementData")); - } - } -} diff --git a/src/isolator.rs b/src/isolator.rs index 8cb2b0c..02fd875 100644 --- a/src/isolator.rs +++ b/src/isolator.rs @@ -5,19 +5,14 @@ use datafusion::{ error::Result, execution::SendableRecordBatchStream, physical_plan::{ - DisplayAs, - DisplayFormatType, - EmptyRecordBatchStream, - ExecutionPlan, - ExecutionPlanProperties, - Partitioning, - PlanProperties, + DisplayAs, DisplayFormatType, EmptyRecordBatchStream, ExecutionPlan, + ExecutionPlanProperties, Partitioning, PlanProperties, }, }; use crate::{ logging::{error, trace}, - vocab::{CtxName, CtxPartitionGroup}, + vocab::{CtxHost, CtxPartitionGroup}, }; /// This is a simple execution plan that isolates a partition from the input @@ -113,12 +108,11 @@ impl ExecutionPlan for PartitionIsolatorExec { )); } - let ctx_name = context + let ctx_name = &context .session_config() - .get_extension::() - .ok_or_else(|| internal_datafusion_err!("CtxName not set in session config"))? - .0 - .clone(); + .get_extension::() + .map(|ctx_host| ctx_host.0.to_string()) + .unwrap_or("unknown_context_host!".to_string()); let partitions_in_input = self.input.output_partitioning().partition_count() as u64; diff --git a/src/lib.rs b/src/lib.rs index 323c00c..a68372f 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -24,9 +24,8 @@ pub use proto::generated::protobuf; pub mod analyze; pub mod codec; -pub mod explain; +//pub mod explain; pub mod flight; -pub mod flight_handlers; pub mod friendly; pub mod isolator; pub mod logging; @@ -39,7 +38,6 @@ pub mod query_planner; pub mod result; pub mod stage; pub mod stage_reader; -pub mod test_utils; pub mod util; pub mod vocab; pub mod worker_discovery; diff --git a/src/main.rs b/src/main.rs index 5fea6d9..21429c9 100644 --- a/src/main.rs +++ b/src/main.rs @@ -33,8 +33,7 @@ async fn main() -> Result<()> { match args.mode.as_str() { "proxy" => { - let mut service = DFRayProxyService::new(args.port); - service.start_up().await?; + let service = DFRayProxyService::new(new_friendly_name()?, args.port).await?; service.serve().await?; } "worker" => { diff --git a/src/physical.rs b/src/physical.rs index cff362b..78844cf 100644 --- a/src/physical.rs +++ b/src/physical.rs @@ -70,8 +70,6 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { display_plan_with_partition_counts(&plan) ); - let maybe_analyze_plan = plan.as_any().downcast_ref::(); - let mut stage_counter = 0; let up = |plan: Arc| { @@ -79,39 +77,10 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { || plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some() { - let plan = if maybe_analyze_plan.is_some() { - let definitely_analyze_plan = maybe_analyze_plan.cloned().unwrap(); - Arc::new(DistributedAnalyzeExec::new( - plan.clone(), - definitely_analyze_plan.verbose(), - definitely_analyze_plan.show_statistics(), - )) as Arc - } else { - plan - }; - // insert a stage marker here so we know where to break up the physical plan later let stage = Arc::new(DFRayStageExec::new(plan, stage_counter)); stage_counter += 1; Ok(Transformed::yes(stage as Arc)) - } else if let Some(definitely_analize_plan) = - plan.as_any().downcast_ref::() - { - // we need to replace this with a DistributedAnalyzeRootExec so that we can - // discoard the output and send back the plans for each task. - - // add a coalesce partitions exec to ensure that we have a single partition - let child = Arc::new(CoalescePartitionsExec::new( - definitely_analize_plan.input().clone(), - )) as Arc; - - let new_plan = Arc::new(DistributedAnalyzeRootExec::new( - child, - definitely_analize_plan.verbose(), - definitely_analize_plan.show_statistics(), - )) as Arc; - - Ok(Transformed::yes(new_plan as Arc)) } else { Ok(Transformed::no(plan)) } @@ -143,7 +112,7 @@ mod tests { use datafusion::arrow::array::{Int32Array, StringArray}; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::arrow::record_batch::RecordBatch; - use datafusion::catalog::memory::DataSourceExec; + use datafusion::execution::context::SessionContext; use datafusion::physical_plan::displayable; use std::sync::Arc; diff --git a/src/planning.rs b/src/planning.rs index 0078f44..efb6a1d 100644 --- a/src/planning.rs +++ b/src/planning.rs @@ -18,7 +18,8 @@ use datafusion::{ logical_expr::LogicalPlan, physical_optimizer::PhysicalOptimizerRule, physical_plan::{ - coalesce_batches::CoalesceBatchesExec, displayable, joins::NestedLoopJoinExec, + analyze::AnalyzeExec, coalesce_batches::CoalesceBatchesExec, + coalesce_partitions::CoalescePartitionsExec, displayable, joins::NestedLoopJoinExec, repartition::RepartitionExec, sorts::sort::SortExec, ExecutionPlan, ExecutionPlanProperties, }, @@ -29,6 +30,7 @@ use itertools::Itertools; use prost::Message; use crate::{ + analyze::{DistributedAnalyzeExec, DistributedAnalyzeRootExec}, isolator::PartitionIsolatorExec, logging::{debug, error, info, trace}, max_rows::MaxRowsExec, @@ -37,12 +39,14 @@ use crate::{ stage::DFRayStageExec, stage_reader::{DFRayStageReaderExec, QueryId}, util::{display_plan_with_partition_counts, get_client, physical_plan_to_bytes, wait_for}, - vocab::{Addrs, CtxAnnotatedOutputs, CtxName, CtxPartitionGroup, CtxStageAddrs}, - vocab::{Host, Hosts, PartitionAddrs, StageAddrs, StageData}, + vocab::{ + Addrs, CtxAnnotatedOutputs, CtxHost, CtxPartitionGroup, CtxStageAddrs, CtxStageId, DDTask, + Host, Hosts, PartitionAddrs, StageAddrs, + }, }; #[derive(Debug)] -pub struct DFRayStage { +pub struct DDStage { /// our stage id pub stage_id: u64, /// the physical plan of our stage @@ -56,7 +60,7 @@ pub struct DFRayStage { pub full_partitions: bool, } -impl DFRayStage { +impl DDStage { fn new( stage_id: u64, plan: Arc, @@ -134,10 +138,11 @@ async fn make_state() -> Result { pub fn add_ctx_extentions( ctx: &mut SessionContext, - ctx_name: &str, + host: &Host, query_id: &str, + stage_id: u64, stage_addrs: Addrs, - partition_group: Option>, + partition_group: Vec, ) -> Result<()> { let state = ctx.state_ref(); let mut guard = state.write(); @@ -145,16 +150,12 @@ pub fn add_ctx_extentions( config.set_extension(Arc::new(CtxStageAddrs(stage_addrs))); config.set_extension(Arc::new(QueryId(query_id.to_owned()))); - config.set_extension(Arc::new(CtxName(ctx_name.to_owned()))); + config.set_extension(Arc::new(CtxHost(host.clone()))); + config.set_extension(Arc::new(CtxStageId(stage_id))); config.set_extension(Arc::new(CtxAnnotatedOutputs::default())); - if let Some(pg) = partition_group { - // this only matters if the plan includes an PartitionIsolatorExec, which looks - // for this for this extension and will be ignored otherwise - - trace!("Adding partition group: {:?}", pg); - config.set_extension(Arc::new(CtxPartitionGroup(pg))); - } + trace!("Adding partition group: {:?}", partition_group); + config.set_extension(Arc::new(CtxPartitionGroup(partition_group))); Ok(()) } @@ -242,7 +243,7 @@ pub async fn execution_planning( physical_plan: Arc, batch_size: usize, partitions_per_worker: Option, -) -> Result<(Arc, Vec)> { +) -> Result<(Arc, Vec)> { let mut stages = vec![]; let mut partition_groups = vec![]; @@ -272,7 +273,7 @@ pub async fn execution_planning( stage_exec.stage_id, )?) as Arc; - let stage = DFRayStage::new( + let stage = DDStage::new( stage_exec.stage_id, input.clone(), partition_groups.clone(), @@ -341,16 +342,10 @@ pub async fn execution_planning( let distributed_plan_clone = Arc::clone(&distributed_plan); distributed_plan.transform_up(up)?; - let txt = stages - .iter() - .map(|stage| format!("{}", display_plan_with_partition_counts(&stage.plan))) - .join(",\n"); - trace!("stages before fix:\n{}", txt); - // add coalesce and max rows to last stage let mut last_stage = stages.pop().ok_or(anyhow!("No stages found"))?; - last_stage = DFRayStage::new( + last_stage = DDStage::new( last_stage.stage_id, Arc::new(MaxRowsExec::new( Arc::new(CoalesceBatchesExec::new(last_stage.plan, batch_size)) @@ -364,6 +359,12 @@ pub async fn execution_planning( // done fixing last stage, put it back stages.push(last_stage); + if contains_analyze(stages[stages.len() - 1].plan.as_ref()) { + // if the plan contains an analyze, we need to add the distributed analyze + // stages to the plan + add_distributed_analyze(&mut stages, false, false)?; + } + let txt = stages .iter() .map(|stage| format!("{}", display_plan_with_partition_counts(&stage.plan))) @@ -373,14 +374,74 @@ pub async fn execution_planning( Ok((distributed_plan_clone, stages)) } +fn contains_analyze(plan: &dyn ExecutionPlan) -> bool { + trace!( + "checking stage for analyze: {}", + displayable(plan).indent(false) + ); + if plan.as_any().downcast_ref::().is_some() { + true + } else { + for child in plan.children() { + if contains_analyze(child.as_ref()) { + return true; + } + } + false + } +} + +pub fn add_distributed_analyze( + stages: &mut [DDStage], + verbose: bool, + show_statistics: bool, +) -> Result<()> { + trace!("Adding distributed analyze to stages"); + let len = stages.len(); + for (i, stage) in stages.iter_mut().enumerate() { + if i == len - 1 { + let plan_without_analyze = stage + .plan + .clone() + .transform_down(|plan: Arc| { + if let Some(analyze) = plan.as_any().downcast_ref::() { + Ok(Transformed::yes(analyze.input().clone())) + } else { + Ok(Transformed::no(plan)) + } + })? + .data; + + trace!( + "plan without analyze: {}", + displayable(plan_without_analyze.as_ref()).indent(false) + ); + stage.plan = Arc::new(DistributedAnalyzeRootExec::new( + Arc::new(CoalescePartitionsExec::new(plan_without_analyze)) + as Arc, + verbose, + show_statistics, + )) as Arc; + stage.partition_groups = vec![vec![0]]; // accounting for coalesce + } else { + stage.plan = Arc::new(DistributedAnalyzeExec::new( + stage.plan.clone(), + verbose, + show_statistics, + )) as Arc; + } + } + Ok(()) +} + /// Distribute the stages to the workers, assigning each stage to a worker /// Returns an Addrs containing the addresses of the workers that will execute /// final stage only as that's all we care about from the call site pub async fn distribute_stages( query_id: &str, - stages: Vec, + stages: Vec, worker_addrs: Vec, -) -> Result { +) -> Result<(Addrs, Vec)> { // map of worker name to address // FIXME: use types over tuples of strings, as we can accidently swap them and // not know @@ -393,13 +454,13 @@ pub async fn distribute_stages( for attempt in 0..3 { // all stages to workers - let (stage_datas, final_addrs) = + let (task_datas, final_addrs) = assign_to_workers(query_id, &stages, workers.values().collect())?; // we retry this a few times to ensure that the workers are ready // and can accept the stages - match try_distribute_stages(&stage_datas).await { - Ok(_) => return Ok(final_addrs), + match try_distribute_tasks(&task_datas).await { + Ok(_) => return Ok((final_addrs, task_datas)), Err(DFRayError::WorkerCommunicationError(bad_worker)) => { error!( "distribute stages for query {query_id} attempt {attempt} failed removing \ @@ -421,27 +482,27 @@ pub async fn distribute_stages( /// try to distribute the stages to the workers, if we cannot communicate with a /// worker return it as the element in the Err -async fn try_distribute_stages(stage_datas: &[StageData]) -> Result<()> { +async fn try_distribute_tasks(task_datas: &[DDTask]) -> Result<()> { // we can use the stage data to distribute the stages to workers - for stage_data in stage_datas { + for task_data in task_datas { trace!( - "Distributing stage_id {}, pg: {:?} to worker: {:?}", - stage_data.stage_id, - stage_data.partition_group, - stage_data.assigned_host + "Distributing Task: stage_id {}, pg: {:?} to worker: {:?}", + task_data.stage_id, + task_data.partition_group, + task_data.assigned_host ); // populate its child stages - let mut stage_data = stage_data.clone(); - stage_data.stage_addrs = Some(get_stage_addrs_from_stages( + let mut stage_data = task_data.clone(); + stage_data.stage_addrs = Some(get_stage_addrs_from_tasks( &stage_data.child_stage_ids, - stage_datas, + task_datas, )?); let host = stage_data .assigned_host .clone() - .context("Assigned host is missing for stage data")?; + .context("Assigned host is missing for task data")?; let mut client = match get_client(&host) { Ok(client) => client, @@ -482,16 +543,24 @@ async fn try_distribute_stages(stage_datas: &[StageData]) -> Result<()> { } // go through our stages, and further divide them into their partition -// groups and produce a StageData for each partition group and assign it +// groups and produce a DDTask for each partition group and assign it // to a worker fn assign_to_workers( query_id: &str, - stages: &[DFRayStage], + stages: &[DDStage], worker_addrs: Vec<&Host>, -) -> Result<(Vec, Addrs)> { - let mut stage_datas = vec![]; +) -> Result<(Vec, Addrs)> { + let mut task_datas = vec![]; let mut worker_idx = 0; + trace!( + "assigning stages: {:?}", + stages + .iter() + .map(|s| format!("stage_id: {}, pgs:{:?}", s.stage_id, s.partition_groups)) + .join(",\n") + ); + // keep track of which worker has the root of the plan tree (highest stage // number) let mut max_stage_id = -1; @@ -521,7 +590,7 @@ fn assign_to_workers( } } - let stage_data = StageData { + let task_data = DDTask { query_id: query_id.to_string(), stage_id: stage.stage_id, plan_bytes, @@ -532,17 +601,14 @@ fn assign_to_workers( full_partitions: stage.full_partitions, assigned_host: Some(host), }; - stage_datas.push(stage_data); + task_datas.push(task_data); } } - Ok((stage_datas, final_addrs)) + Ok((task_datas, final_addrs)) } -fn get_stage_addrs_from_stages( - target_stage_ids: &[u64], - stages: &[StageData], -) -> Result { +fn get_stage_addrs_from_tasks(target_stage_ids: &[u64], stages: &[DDTask]) -> Result { let mut stage_addrs = StageAddrs::default(); // this can be more efficient diff --git a/src/processor_service.rs b/src/processor_service.rs index a4714a9..555ff91 100644 --- a/src/processor_service.rs +++ b/src/processor_service.rs @@ -16,8 +16,8 @@ // under the License. use std::{ - collections::HashMap, - ops::DerefMut, + collections::{HashMap, HashSet}, + fmt::Debug, sync::Arc, time::{Duration, SystemTime}, }; @@ -26,11 +26,11 @@ use anyhow::{anyhow, Context}; use arrow::array::RecordBatch; use arrow_flight::{ encode::FlightDataEncoderBuilder, error::FlightError, - flight_service_server::FlightServiceServer, Action, FlightData, Ticket, + flight_service_server::FlightServiceServer, Action, Ticket, }; use async_stream::stream; use datafusion::{ - physical_plan::{displayable, EmptyRecordBatchStream, ExecutionPlan, ExecutionPlanProperties}, + physical_plan::{ExecutionPlan, ExecutionPlanProperties}, prelude::SessionContext, }; use futures::{StreamExt, TryStreamExt}; @@ -49,47 +49,62 @@ use crate::{ planning::{add_ctx_extentions, get_ctx}, protobuf::{ AnnotatedTaskOutput, AnnotatedTaskOutputs, FlightDataMetadata, FlightTicketData, Host, - StageData, }, result::{DFRayError, Result}, util::{ bytes_to_physical_plan, display_plan_with_partition_counts, get_addrs, - register_object_store_for_paths_in_plan, reporting_stream, + register_object_store_for_paths_in_plan, start_up, }, - vocab::{Addrs, CtxAnnotatedOutputs, CtxName}, + vocab::{Addrs, CtxAnnotatedOutputs, CtxPartitionGroup, DDTask}, }; #[derive(Eq, PartialEq, Hash, Clone, Debug)] -struct PlanKey { +struct StageKey { query_id: String, stage_id: u64, - partition: u64, } -// For each plan key, we may have multiple plans that we might need to hold -// with the same key. -type PlanVec = Vec<(SystemTime, SessionContext, Arc)>; +#[derive(Clone)] +struct Task { + partitions: HashSet, + ctx: SessionContext, + plan: Arc, +} + +impl Debug for Task { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("Task") + .field("partitions", &self.partitions) + .finish() + } +} + +#[derive(Debug)] +struct StageTasks { + tasks: Vec, + insert_time: SystemTime, +} /// It only responds to the DoGet Arrow Flight method. struct DfRayProcessorHandler { /// our name, useful for logging name: String, - //// our address string, also useful for logging - pub(crate) addr: String, + /// our address string, also useful for logging + addr: String, /// our map of query_id -> (session ctx, execution plan) - #[allow(clippy::type_complexity)] - plans: Arc>>, + stages: Arc>>, done: Arc>, } impl DfRayProcessorHandler { pub fn new(name: String, addr: String) -> Self { - let plans: Arc>> = Arc::new(RwLock::new(HashMap::new())); + let stages: Arc>> = + Arc::new(RwLock::new(HashMap::new())); let done = Arc::new(Mutex::new(false)); // start a plan janitor ask to clean up old plans that were not collected for // any reason - let c_plans = plans.clone(); + let c_stages = stages.clone(); let c_done = done.clone(); let c_name = name.clone(); std::thread::spawn(move || { @@ -97,43 +112,42 @@ impl DfRayProcessorHandler { // wait for 10 seconds std::thread::sleep(Duration::from_secs(10)); if *c_done.lock() { - info!("{} plan janitor done", c_name); + info!("{} janitor done", c_name); break; } - trace!("{} plan janitor waking up", c_name); + trace!("{} janitor waking up", c_name); let now = SystemTime::now(); let mut to_remove = vec![]; { - let _guard = c_plans.read(); - for (key, plan_vec) in _guard.iter() { - if plan_vec.is_empty() { - error!("unexpectedly found empty plan vec. removing"); + let _guard = c_stages.read(); + for (key, stage_tasks) in _guard.iter() { + if stage_tasks.tasks.is_empty() { + error!("unexpectedly found empty stage tasks. removing"); to_remove.push(key.clone()); } else { // check if any plan in this vec is older than 1 minute - for (insert_time, _, _) in plan_vec.iter() { - if now - .duration_since(*insert_time) - .map(|d| d.as_secs() > 60) - .inspect_err(|e| { - error!("CANNOT COMPUTE DURATION OR REMOVE PLANS: {e:?}"); - }) - .unwrap_or(false) - { - to_remove.push(key.clone()); - break; - } + if now + .duration_since(stage_tasks.insert_time) + .map(|d| d.as_secs() > 60) + .inspect_err(|e| { + error!("CANNOT COMPUTE DURATION OR REMOVE STAGES: {e:?}"); + // maybe just panic here? + }) + .unwrap_or(false) + { + to_remove.push(key.clone()); + break; } } } } if !to_remove.is_empty() { - let mut _guard = c_plans.write(); + let mut _guard = c_stages.write(); for key in to_remove.iter() { _guard.remove(key); - debug!("{} removed old plan key {:?}", c_name, key); + debug!("{} removed old stage key {:?}", c_name, key); } } } @@ -142,7 +156,7 @@ impl DfRayProcessorHandler { Self { name, addr, - plans, + stages, done, } } @@ -153,7 +167,7 @@ impl DfRayProcessorHandler { *self.done.lock() = true; } - pub async fn add_plan( + pub async fn add_task( &self, query_id: String, stage_id: u64, @@ -185,9 +199,10 @@ impl DfRayProcessorHandler { }; trace!( - "{} adding plan for stage {} partitions: {:?} stage_addrs: {:?} plan:\n{}", + "{} adding task for stage {} partition group: {:?} partitions: {:?} stage_addrs: {:?} plan:\n{}", self.name, stage_id, + partition_group, partitions, stage_addrs, display_plan_with_partition_counts(&plan) @@ -197,24 +212,26 @@ impl DfRayProcessorHandler { let now = SystemTime::now(); - for partition in partitions.iter() { - let key = PlanKey { - query_id: query_id.clone(), - stage_id, - partition: *partition, - }; - { - let mut _guard = self.plans.write(); - if let Some(plan_vec) = _guard.get_mut(&key) { - plan_vec.push((now, ctx.clone(), plan.clone())); - } else { - _guard.insert(key.clone(), vec![(now, ctx.clone(), plan.clone())]); - } - trace!("{} added plan for plan key {:?}", self.name, key); - } + let task = Task { + ctx: ctx.clone(), + plan: plan.clone(), + partitions: HashSet::from_iter(partitions.clone()), + }; + + let key = StageKey { + query_id: query_id.clone(), + stage_id, + }; + { + let mut _guard = self.stages.write(); + let stage_tasks = _guard.entry(key.clone()).or_insert_with(|| StageTasks { + tasks: vec![], + insert_time: now, + }); + stage_tasks.tasks.push(task.clone()); + trace!("{} added task for stage key {:?}", self.name, key); } - debug!("{} plans held {:?}", self.name, self.plans.read().len()); Ok(()) } @@ -226,49 +243,97 @@ impl DfRayProcessorHandler { partition_group: Vec, ) -> Result { let mut ctx = get_ctx()?; + let host = Host { + addr: self.addr.clone(), + name: self.name.clone(), + }; add_ctx_extentions( &mut ctx, - &format!("{} stage:{} pg:{:?}", self.name, stage_id, partition_group), + &host, &query_id, + stage_id, stage_addrs.clone(), - Some(partition_group), + partition_group, )?; Ok(ctx) } + /// Retrieve the requested ctx and plan to execute. Also return a bool + /// indicating if this is the last partition for this plan fn get_ctx_and_plan( &self, query_id: &str, stage_id: u64, partition: u64, - ) -> Result<(SessionContext, Arc)> { - let key = PlanKey { + ) -> Result<(SessionContext, Arc, bool)> { + let stage_key = StageKey { query_id: query_id.to_string(), stage_id, - partition, }; - let (ctx, plan) = { - let mut _guard = self.plans.write(); - let (plan_key, mut plan_vec) = _guard.remove_entry(&key).context(format!( - "{}, No plan found for plan key {:?}", - self.name, key, + let (ctx, plan, last) = { + let mut _guard = self.stages.write(); + let stage_tasks = _guard.get_mut(&stage_key).context(format!( + "{}, No plan found for stage key{:?}", + self.name, stage_key, ))?; trace!( - "{} found {} plans for plan key {:?}", + "{} found {} tasks for stage key {:?}", self.name, - plan_vec.len(), - plan_key + stage_tasks.tasks.len(), + stage_key ); - let (_insert_time, ctx, plan) = plan_vec.pop().expect("plan_vec should not be empty"); - if !plan_vec.is_empty() { - _guard.insert(plan_key, plan_vec); + + // of the tasks for this stage, find one that has this partition not yet consumed + let task = stage_tasks + .tasks + .iter_mut() + .find(|t| t.partitions.contains(&partition)) + .context(format!( + "{}, No task found for stage key {:?} partition {}", + self.name, stage_key, partition + ))?; + + // remove this partition from the list of partitions yet to be consumed + if !task.partitions.remove(&partition) { + // it should be in there because we just filtered for it + return Err(anyhow!("UNEXPECTED: partition {partition} not in plan parts").into()); } - (ctx, plan) + // finally, we return the ctx and plan to execute this task, as well as a + // bool indicating if this is the last partition for this task + ( + task.ctx.clone(), + task.plan.clone(), + task.partitions.is_empty(), + ) }; - Ok((ctx, plan)) + { + // some house keeping, if there are no more partitions left for this task, + // remove it from the stage tasks. + let mut _guard = self.stages.write(); + + let remove_it = _guard.get_mut(&stage_key).map(|stage_tasks| { + stage_tasks.tasks.retain(|t| !t.partitions.is_empty()); + trace!( + "remaining tasks: for stage {:?}: {:?}", + stage_key, + stage_tasks + ); + + // furthermore, if there are no more tasks left for this stage, + // remove the stage from the map + stage_tasks.tasks.is_empty() + }); + + if remove_it.unwrap_or(false) { + _guard.remove(&stage_key); + debug!("{} removed stage key {:?}", self.name, stage_key); + } + } + + Ok((ctx, plan, last)) } /// we want to send any additional FlightDataMetadata that we discover while @@ -285,13 +350,13 @@ impl DfRayProcessorHandler { /// The reason we have to do it at the end is that the metadata may include an annodated /// plan with metrics which will only be available after the stream has been /// fully consumed. - fn make_stream( &self, ctx: SessionContext, plan: Arc, stage_id: u64, partition: u64, + last_partition: bool, ) -> Result { let task_ctx = ctx.task_ctx(); @@ -304,7 +369,7 @@ impl DfRayProcessorHandler { }) .map_err(|e| FlightError::from_external_error(Box::new(e))); - info!("{} plans held {}", self.name, self.plans.read().len()); + info!("{} tasks held {}", self.name, self.stages.read().len()); fn find_analyze(plan: &dyn ExecutionPlan) -> Option<&DistributedAnalyzeExec> { if let Some(target) = plan.as_any().downcast_ref::() { @@ -321,15 +386,23 @@ impl DfRayProcessorHandler { let mut flight_data_stream = FlightDataEncoderBuilder::new().build(stream); let name = self.name.clone(); + let host = Host { + addr: self.addr.clone(), + name: self.name.clone(), + }; #[allow(unused_assignments)] // clippy can't understand our assignment to done in the macro let out_stream = async_stream::stream! { let mut done = false; while !done { - match (done, flight_data_stream.next().await) { - (false, None) => { - // no more data so now we yield our additional FlightDataMetadata if required + match (done, last_partition, flight_data_stream.next().await) { + (false, false, None) => { + // we finished a partition, but still have more to do, do nothing + done = true; + } + (false, true, None) => { + // no more data in the last partition, so now we yield our additional FlightDataMetadata debug!("stream exhausted, yielding FlightDataMetadata"); let task_outputs = ctx.state().config() .get_extension::() @@ -337,6 +410,11 @@ impl DfRayProcessorHandler { .0 .clone(); + let partition_group = ctx.state().config() + .get_extension::() + .expect("CtxPartitionGroup to be set") + .0.clone(); + if let Some(analyze) = find_analyze(plan.as_ref()) { let annotated_plan = analyze.annotated_plan(); @@ -344,9 +422,9 @@ impl DfRayProcessorHandler { let output = AnnotatedTaskOutput { plan: annotated_plan, - host: None, + host: Some(host.clone()), stage_id, - partition_group: vec![partition], + partition_group, }; task_outputs.lock().push(output); } @@ -381,20 +459,23 @@ impl DfRayProcessorHandler { done = true; }, - (false, Some(Err(e))) => { + (false, _, Some(Err(e))) => { yield Err(Status::internal(format!( "Unexpected error getting flight data stream: {e:?}", ))); done = true; }, - (false, Some(Ok(flight_data))) => { + (false, _, Some(Ok(flight_data))) => { // we have a flight data, so we yield it trace!("received normal flight data, yielding"); yield Ok(flight_data); }, - (true, _) => { - // we are done, so we don't yield anything - error!("{} we should not arrive at this block!. stage {} partition {}", name, stage_id, partition); + (true, false, _ ) => { + // we've finished a partition, do nothing + done = true; + }, + (true, true, _ ) => { + yield Err(Status::internal(format!("{name} reached expected unreachable block"))); }, } } @@ -422,14 +503,14 @@ impl DfRayProcessorHandler { &self, action: Action, ) -> Result, Status> { - let stage_data = StageData::decode(action.body.as_ref()).map_err(|e| { + let task_data = DDTask::decode(action.body.as_ref()).map_err(|e| { Status::internal(format!( "{}, Unexpected error decoding StageData: {e:?}", self.name )) })?; - let addrs = stage_data + let addrs = task_data .stage_addrs .as_ref() .context("stage addrs not present") @@ -437,13 +518,13 @@ impl DfRayProcessorHandler { .and_then(get_addrs) .map_err(|e| Status::internal(format!("{}, {e}", self.name)))?; - self.add_plan( - stage_data.query_id, - stage_data.stage_id, + self.add_task( + task_data.query_id, + task_data.stage_id, addrs, - stage_data.partition_group, - stage_data.full_partitions, - &stage_data.plan_bytes, + task_data.partition_group, + task_data.full_partitions, + &task_data.plan_bytes, ) .await .map_err(|e| Status::internal(format!("{}, Could not add plan: {e:?}", self.name)))?; @@ -476,29 +557,28 @@ impl FlightHandler for DfRayProcessorHandler { )) })?; - let plan_key = PlanKey { + let task_key = StageKey { query_id: ftd.query_id.clone(), stage_id: ftd.stage_id, - partition: ftd.partition, }; debug!( - "{}, request for plan_key:{:?} from: {},{}", - self.name, plan_key, ftd.requestor_name, remote_addr + "{}, request for task_key:{:?} partition: {} from: {},{}", + self.name, task_key, ftd.partition, ftd.requestor_name, remote_addr ); let name = self.name.clone(); - let (ctx, plan) = self + let (ctx, plan, is_last_partition) = self .get_ctx_and_plan(&ftd.query_id, ftd.stage_id, ftd.partition) .map_err(|e| { Status::internal(format!( - "{name} Could not find plan for query_id {} stage {} partition {}: {e:?}", + "{name} Could not find task for query_id {} stage {} partition {}: {e:?}", ftd.query_id, ftd.stage_id, ftd.partition )) })?; let do_get_stream = self - .make_stream(ctx, plan, ftd.stage_id, ftd.partition) + .make_stream(ctx, plan, ftd.stage_id, ftd.partition, is_last_partition) .map_err(|e| { Status::internal(format!( "{name} Could not make stream for query_id {} stage {} partition {}: {e:?}", @@ -531,16 +611,6 @@ impl FlightHandler for DfRayProcessorHandler { } } -pub async fn start_up(port: usize) -> Result { - let my_host_str = format!("0.0.0.0:{}", port); - - let listener = TcpListener::bind(&my_host_str) - .await - .context("Could not bind socket to {my_host_str}")?; - - Ok(listener) -} - /// DFRayProcessorService is a Arrow Flight service that serves streams of /// partitions from a hosted Physical Plan /// @@ -553,7 +623,6 @@ pub struct DFRayProcessorService { addr: String, all_done_tx: Arc>>, all_done_rx: Option>, - port: usize, } impl DFRayProcessorService { @@ -578,7 +647,6 @@ impl DFRayProcessorService { addr, all_done_tx, all_done_rx: Some(all_done_rx), - port, }) } diff --git a/src/proto/generated/protobuf.rs b/src/proto/generated/protobuf.rs index 4045c9a..fefe365 100644 --- a/src/proto/generated/protobuf.rs +++ b/src/proto/generated/protobuf.rs @@ -40,17 +40,12 @@ pub struct DistributedAnalyzeRootExecNode { pub show_statistics: bool, } #[derive(Clone, PartialEq, ::prost::Message)] -pub struct DistributedExplainExecNode { +pub struct RecordBatchExecNode { #[prost(message, optional, tag = "1")] pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, - #[prost(string, tag = "2")] - pub logical_plan: ::prost::alloc::string::String, - #[prost(string, tag = "3")] - pub physical_plan: ::prost::alloc::string::String, - #[prost(string, tag = "4")] - pub distributed_plan: ::prost::alloc::string::String, - #[prost(string, tag = "5")] - pub distributed_stages: ::prost::alloc::string::String, + /// the arrow IPC serialized record batch of the response + #[prost(bytes = "vec", tag = "2")] + pub batch: ::prost::alloc::vec::Vec, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct DfRayExecNode { @@ -124,9 +119,6 @@ pub struct TicketStatementData { /// the schema of the final stage #[prost(message, optional, tag = "4")] pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, - /// For EXPLAIN queries, store the explain plan data directly - #[prost(message, optional, tag = "5")] - pub explain_data: ::core::option::Option, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct Host { @@ -152,7 +144,7 @@ pub struct StageAddrs { pub stage_addrs: ::std::collections::HashMap, } #[derive(Clone, PartialEq, ::prost::Message)] -pub struct StageData { +pub struct DdTask { #[prost(string, tag = "1")] pub query_id: ::prost::alloc::string::String, #[prost(uint64, tag = "2")] diff --git a/src/proxy_service.rs b/src/proxy_service.rs index 8947682..c08e503 100644 --- a/src/proxy_service.rs +++ b/src/proxy_service.rs @@ -17,10 +17,19 @@ use std::sync::Arc; -use anyhow::{anyhow, Context}; +use anyhow::Context; +use arrow::datatypes::Schema; use arrow_flight::{ - flight_service_server::FlightServiceServer, FlightDescriptor, FlightInfo, Ticket, + encode::FlightDataEncoderBuilder, + error::FlightError, + flight_service_server::FlightServiceServer, + sql::{ProstMessageExt, TicketStatementQuery}, + FlightDescriptor, FlightEndpoint, FlightInfo, Ticket, }; +use datafusion::physical_plan::{ + coalesce_partitions::CoalescePartitionsExec, displayable, ExecutionPlan, Partitioning, +}; +use futures::TryStreamExt; use parking_lot::Mutex; use prost::Message; use tokio::{ @@ -30,28 +39,123 @@ use tokio::{ use tonic::{async_trait, transport::Server, Request, Response, Status}; use crate::{ - explain::is_explain_query, flight::{FlightSqlHandler, FlightSqlServ}, - flight_handlers::FlightRequestHandler, logging::{debug, info, trace}, + planning::{add_ctx_extentions, get_ctx}, protobuf::TicketStatementData, - query_planner::QueryPlanner, + query_planner::{QueryPlan, QueryPlanner}, result::Result, + stage_reader::DFRayStageReaderExec, + util::{display_plan_with_partition_counts, get_addrs, start_up}, + vocab::{Addrs, Host}, worker_discovery::get_worker_addresses, }; pub struct DfRayProxyHandler { - pub flight_handler: FlightRequestHandler, + /// our host info, useful for logging + pub host: Host, + + pub planner: QueryPlanner, } impl DfRayProxyHandler { - pub fn new() -> Self { + pub fn new(name: String, addr: String) -> Self { // call this function to bootstrap the worker discovery mechanism get_worker_addresses().expect("Could not get worker addresses upon startup"); + + let host = Host { + name: name.clone(), + addr: addr.clone(), + }; Self { - flight_handler: FlightRequestHandler::new(QueryPlanner::new()), + host: host.clone(), + planner: QueryPlanner::new(), } } + + pub fn create_flight_info_response(&self, query_plan: QueryPlan) -> Result { + let mut flight_info = FlightInfo::new() + .try_with_schema(&query_plan.schema) + .map_err(|e| Status::internal(format!("Could not create flight info {e:?}")))?; + + let ticket_data = TicketStatementData { + query_id: query_plan.query_id, + stage_id: query_plan.final_stage_id, + stage_addrs: Some(query_plan.worker_addresses.into()), + schema: Some( + query_plan + .schema + .try_into() + .map_err(|e| Status::internal(format!("Could not convert schema {e:?}")))?, + ), + }; + + let ticket = Ticket::new( + TicketStatementQuery { + statement_handle: ticket_data.encode_to_vec().into(), + } + .as_any() + .encode_to_vec(), + ); + + let endpoint = FlightEndpoint::new().with_ticket(ticket); + flight_info = flight_info.with_endpoint(endpoint); + + Ok(flight_info) + } + + pub async fn execute_plan_and_build_stream( + &self, + plan: Arc, + query_id: String, + stage_id: u64, + addrs: Addrs, + ) -> Result, Status> { + let mut ctx = + get_ctx().map_err(|e| Status::internal(format!("Could not create context {e:?}")))?; + + add_ctx_extentions(&mut ctx, &self.host, &query_id, stage_id, addrs, vec![]) + .map_err(|e| Status::internal(format!("Could not add context extensions {e:?}")))?; + + // TODO: revisit this to allow for consuming a partitular partition + trace!("calling execute plan"); + let partition = 0; + let stream = plan + .execute(partition, ctx.task_ctx()) + .map_err(|e| { + Status::internal(format!( + "Error executing plan for query_id {} partition {}: {e:?}", + query_id, partition + )) + })? + .map_err(|e| FlightError::ExternalError(Box::new(e))); + + let out_stream = FlightDataEncoderBuilder::new() + .build(stream) + .map_err(move |e| Status::internal(format!("Unexpected error building stream {e:?}"))); + + Ok(Response::new(Box::pin(out_stream))) + } + + pub fn validate_single_stage_addrs( + &self, + addrs: &Addrs, + expected_stage_id: u64, + ) -> Result<(), Status> { + if addrs.len() != 1 { + return Err(Status::internal(format!( + "Expected exactly one stage in addrs, got {}", + addrs.len() + ))); + } + if !addrs.contains_key(&expected_stage_id) { + return Err(Status::internal(format!( + "No addresses found for stage_id {} in addrs", + expected_stage_id + ))); + } + Ok(()) + } } #[async_trait] @@ -61,15 +165,16 @@ impl FlightSqlHandler for DfRayProxyHandler { query: arrow_flight::sql::CommandStatementQuery, _request: Request, ) -> Result, Status> { - let is_explain = is_explain_query(&query.query); + let query_plan = self + .planner + .prepare_query(&query.query) + .await + .map_err(|e| Status::internal(format!("Could not prepare query {e:?}")))?; - if is_explain { - self.flight_handler - .handle_explain_request(&query.query) - .await - } else { - self.flight_handler.handle_query_request(&query.query).await - } + self.create_flight_info_response(query_plan) + .map(|flight_info| Response::new(flight_info)) + .context("Could not create flight info response") + .map_err(|e| Status::internal(format!("Error creating flight info: {e:?}"))) } async fn do_get_statement( @@ -88,15 +193,54 @@ impl FlightSqlHandler for DfRayProxyHandler { debug!("request for ticket: {:?} from {}", tsd, remote_addr); - if tsd.explain_data.is_some() { - self.flight_handler - .handle_explain_statement_execution(tsd, &remote_addr) - .await - } else { - self.flight_handler - .handle_regular_statement_execution(tsd, &remote_addr) - .await - } + let schema: Schema = tsd + .schema + .as_ref() + .ok_or_else(|| Status::internal("No schema in TicketStatementData"))? + .try_into() + .map_err(|e| Status::internal(format!("Cannot convert schema {e}")))?; + + // Create an Addrs from the final stage information in the tsd + let stage_addrs = tsd.stage_addrs.ok_or_else(|| { + Status::internal("No stages_addrs in TicketStatementData, cannot proceed") + })?; + + let addrs: Addrs = get_addrs(&stage_addrs).map_err(|e| { + Status::internal(format!("Cannot get addresses from stage_addrs {e:?}")) + })?; + + trace!("calculated addrs: {:?}", addrs); + + // Validate that addrs contains exactly one stage + self.validate_single_stage_addrs(&addrs, tsd.stage_id)?; + + let stage_partition_addrs = addrs.get(&tsd.stage_id).ok_or_else(|| { + Status::internal(format!( + "No partition addresses found for stage_id {}", + tsd.stage_id + )) + })?; + + let plan = Arc::new( + DFRayStageReaderExec::try_new( + Partitioning::UnknownPartitioning(stage_partition_addrs.len()), + Arc::new(schema), + tsd.stage_id, + ) + // TODO: revisit this to allow for consuming a particular partition + .map(|stg| CoalescePartitionsExec::new(Arc::new(stg))) + .map_err(|e| Status::internal(format!("Unexpected error {e}")))?, + ) as Arc; + + debug!( + "request for query_id {} from {} reader plan:\n{}", + tsd.query_id, + remote_addr, + display_plan_with_partition_counts(&plan) + ); + + self.execute_plan_and_build_stream(plan, tsd.query_id, tsd.stage_id, addrs) + .await } } @@ -105,59 +249,44 @@ impl FlightSqlHandler for DfRayProxyHandler { /// /// It only responds to the DoGet Arrow Flight method pub struct DFRayProxyService { - listener: Option, + name: String, + listener: TcpListener, handler: Arc, - addr: Option, + addr: String, all_done_tx: Arc>>, all_done_rx: Option>, port: usize, } impl DFRayProxyService { - pub fn new(port: usize) -> Self { + pub async fn new(name: String, port: usize) -> Result { debug!("Creating DFRayProxyService!"); - let listener = None; - let addr = None; let (all_done_tx, all_done_rx) = channel(1); let all_done_tx = Arc::new(Mutex::new(all_done_tx)); - let handler = Arc::new(DfRayProxyHandler::new()); + let listener = start_up(port).await?; - Self { + let addr = format!("{}", listener.local_addr().unwrap()); + + info!("DFRayProcessorService bound to {addr}"); + + let handler = Arc::new(DfRayProxyHandler::new(name.clone(), addr.clone())); + + Ok(Self { + name, listener, handler, addr, all_done_tx, all_done_rx: Some(all_done_rx), port, - } - } - - pub async fn start_up(&mut self) -> Result<()> { - let my_host_str = format!("0.0.0.0:{}", self.port); - - self.listener = TcpListener::bind(&my_host_str) - .await - .map(Some) - .context("Could not bind socket to {my_host_str}")?; - - self.addr = Some(format!( - "{}", - self.listener.as_ref().unwrap().local_addr().unwrap() - )); - - info!("DFRayProxyService bound to {}", self.addr.as_ref().unwrap()); - - Ok(()) + }) } /// get the address of the listing socket for this service pub fn addr(&self) -> Result { - let addr = self.addr.clone().ok_or(anyhow!( - "DFRayProxyService not started yet, no address available" - ))?; - Ok(addr) + Ok(self.addr.clone()) } pub async fn all_done(&self) -> Result<()> { @@ -171,7 +300,7 @@ impl DFRayProxyService { } /// start the service - pub async fn serve(&mut self) -> Result<()> { + pub async fn serve(mut self) -> Result<()> { let mut all_done_rx = self.all_done_rx.take().unwrap(); let signal = async move { @@ -194,12 +323,10 @@ impl DFRayProxyService { //let svc = FlightServiceServer::new(service); let svc = FlightServiceServer::with_interceptor(service, intercept); - let listener = self.listener.take().unwrap(); - Server::builder() .add_service(svc) .serve_with_incoming_shutdown( - tokio_stream::wrappers::TcpListenerStream::new(listener), + tokio_stream::wrappers::TcpListenerStream::new(self.listener), signal, ) .await @@ -207,142 +334,3 @@ impl DFRayProxyService { Ok(()) } } - -#[cfg(test)] -mod tests { - use super::*; - // Test-specific imports - use arrow_flight::{ - sql::{CommandStatementQuery, TicketStatementQuery}, - FlightDescriptor, Ticket, - }; - use prost::Message; - use tonic::Request; - - use crate::test_utils::explain_test_helpers::{ - create_explain_ticket_statement_data, create_test_proxy_handler, - verify_explain_stream_results, - }; - - #[tokio::test] - async fn test_get_flight_info_statement_explain() { - let handler = create_test_proxy_handler(); - - // Test EXPLAIN query - let command = CommandStatementQuery { - query: "EXPLAIN SELECT 1 as test_col".to_string(), - transaction_id: None, - }; - - let request = Request::new(FlightDescriptor::new_cmd(vec![])); - let result = handler.get_flight_info_statement(command, request).await; - - assert!(result.is_ok()); - let response = result.unwrap(); - let flight_info = response.into_inner(); - - // Verify FlightInfo structure - assert!(!flight_info.schema.is_empty()); - assert_eq!(flight_info.endpoint.len(), 1); - assert!(flight_info.endpoint[0].ticket.is_some()); - - // Verify that ticket has content (encoded TicketStatementData) - let ticket = flight_info.endpoint[0].ticket.as_ref().unwrap(); - assert!(!ticket.ticket.is_empty()); - - println!( - "✓ FlightInfo created successfully with {} schema bytes and ticket with {} bytes", - flight_info.schema.len(), - ticket.ticket.len() - ); - } - - #[tokio::test] - async fn test_do_get_statement_explain() { - let handler = create_test_proxy_handler(); - - // First prepare an EXPLAIN query to get proper ticket data - let query = "EXPLAIN SELECT 1 as test_col"; - let plans = handler - .flight_handler - .planner - .prepare_explain(query) - .await - .unwrap(); - - let tsd = create_explain_ticket_statement_data(plans); - - // Create the ticket - let ticket_query = TicketStatementQuery { - statement_handle: tsd.encode_to_vec().into(), - }; - - let request = Request::new(Ticket::new(vec![])); - let result = handler.do_get_statement(ticket_query, request).await; - - assert!(result.is_ok()); - let response = result.unwrap(); - let stream = response.into_inner(); - - // Use shared verification function - verify_explain_stream_results(stream).await; - } - - #[tokio::test] - async fn test_compare_explain_flight_info_responses() { - let handler = create_test_proxy_handler(); - let query = "EXPLAIN SELECT 1 as test_col"; - - // Get FlightInfo from handle_explain_request - let result1 = handler - .flight_handler - .handle_explain_request(query) - .await - .unwrap(); - let flight_info1 = result1.into_inner(); - - // Get FlightInfo from get_flight_info_statement - let command = CommandStatementQuery { - query: query.to_string(), - transaction_id: None, - }; - let request = Request::new(FlightDescriptor::new_cmd(vec![])); - let result2 = handler - .get_flight_info_statement(command, request) - .await - .unwrap(); - let flight_info2 = result2.into_inner(); - - // Compare FlightInfo responses (structure should be identical) - assert_eq!(flight_info1.schema.len(), flight_info2.schema.len()); // Same schema size - assert_eq!(flight_info1.endpoint.len(), flight_info2.endpoint.len()); // Same number of endpoints - assert_eq!(flight_info1.endpoint.len(), 1); // Both should have exactly one endpoint - - // Both should have tickets with content - let ticket1 = flight_info1.endpoint[0].ticket.as_ref().unwrap(); - let ticket2 = flight_info2.endpoint[0].ticket.as_ref().unwrap(); - assert!(!ticket1.ticket.is_empty()); - assert!(!ticket2.ticket.is_empty()); - - println!("✓ Both tests produce FlightInfo with identical structure:"); - println!( - " - Schema bytes: {} vs {}", - flight_info1.schema.len(), - flight_info2.schema.len() - ); - println!( - " - Endpoints: {} vs {}", - flight_info1.endpoint.len(), - flight_info2.endpoint.len() - ); - println!( - " - Ticket bytes: {} vs {}", - ticket1.ticket.len(), - ticket2.ticket.len() - ); - } - - // TODO: Add tests for regular (non-explain) queries - // We might need to create integration or end-to-end test infrastructure for this because - // they need workers -} diff --git a/src/query_planner.rs b/src/query_planner.rs index 668b1a4..1119709 100644 --- a/src/query_planner.rs +++ b/src/query_planner.rs @@ -3,63 +3,70 @@ use std::sync::Arc; use anyhow::anyhow; use arrow::datatypes::SchemaRef; use datafusion::{ - logical_expr::LogicalPlan, - physical_plan::ExecutionPlan, - prelude::SessionContext, + logical_expr::LogicalPlan, physical_plan::ExecutionPlan, prelude::SessionContext, }; use crate::{ - explain::{DistributedExplainExec, is_explain_query}, - worker_discovery::get_worker_addresses, logging::debug, planning::{ - distribute_stages, - execution_planning, - get_ctx, - logical_planning, - physical_planning, - DFRayStage, + distribute_stages, execution_planning, get_ctx, logical_planning, physical_planning, + DDStage, }, result::Result, - vocab::Addrs, + vocab::{Addrs, DDTask}, + worker_discovery::get_worker_addresses, }; -/// Result of base query preparation containing all planning artifacts for both query and its EXPLAIN -pub struct QueryPlanBase { +/// Result of query preparation for execution of both query and its EXPLAIN +pub struct QueryPlan { pub query_id: String, pub session_context: SessionContext, + pub worker_addresses: Addrs, + pub final_stage_id: u64, + pub schema: SchemaRef, pub logical_plan: LogicalPlan, pub physical_plan: Arc, pub distributed_plan: Arc, - pub distributed_stages: Vec, + pub distributed_tasks: Vec, +} + +impl QueryPlan { + pub fn is_explain(&self) -> bool { + match self.logical_plan { + LogicalPlan::Explain { .. } => true, + _ => false, + } + } } -impl std::fmt::Debug for QueryPlanBase { +impl std::fmt::Debug for QueryPlan { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("QueryPlanBase") + f.debug_struct("QueryPlan") .field("query_id", &self.query_id) .field("session_context", &"") .field("logical_plan", &self.logical_plan) .field("physical_plan", &format!("<{}>", self.physical_plan.name())) - .field("distributed_plan", &format!("<{}>", self.distributed_plan.name())) - .field("stages", &format!("<{} stages>", self.distributed_stages.len())) + .field( + "distributed_plan", + &format!("<{}>", self.distributed_plan.name()), + ) + .field( + "stages", + &format!("<{} stages>", self.distributed_tasks.len()), + ) .finish() } } -/// Result of query preparation for execution of both query and its EXPLAIN -#[derive(Debug)] -pub struct QueryPlan { - pub query_id: String, - pub worker_addresses: Addrs, - pub final_stage_id: u64, - pub schema: SchemaRef, - pub explain_data: Option, -} - /// Query planner responsible for preparing SQL queries for distributed execution pub struct QueryPlanner; +impl Default for QueryPlanner { + fn default() -> Self { + Self::new() + } +} + impl QueryPlanner { pub fn new() -> Self { Self @@ -69,264 +76,42 @@ impl QueryPlanner { /// /// Prepare a query by parsing the SQL, planning it, and distributing the /// physical plan into stages that can be executed by workers. - pub async fn prepare_query_base(&self, sql: &str, query_type: &str) -> Result { - debug!("prepare_query_base: {} SQL = {}", query_type, sql); - + pub async fn prepare_query(&self, sql: &str) -> Result { let query_id = uuid::Uuid::new_v4().to_string(); let ctx = get_ctx().map_err(|e| anyhow!("Could not create context: {e}"))?; let logical_plan = logical_planning(sql, &ctx).await?; let physical_plan = physical_planning(&logical_plan, &ctx).await?; - // divide the physical plan into chunks (stages) that we can distribute to workers - let (distributed_plan, distributed_stages) = execution_planning(physical_plan.clone(), 8192, Some(2)).await?; - - Ok(QueryPlanBase { - query_id, - session_context: ctx, - logical_plan, - physical_plan, - distributed_plan, - distributed_stages, - }) - } - - /// Prepare a distributed query - pub async fn prepare_query(&self, sql: &str) -> Result { - let base_result = self.prepare_query_base(sql, "REGULAR").await?; - - if base_result.distributed_stages.is_empty() { - return Err(anyhow!("No stages generated for query").into()); - } + // divide the physical plan into chunks (tasks) that we can distribute to workers + let (distributed_plan, distributed_stages) = + execution_planning(physical_plan.clone(), 8192, Some(2)).await?; let worker_addrs = get_worker_addresses()?; // gather some information we need to send back such that // we can send a ticket to the client - let final_stage = &base_result.distributed_stages[base_result.distributed_stages.len() - 1]; + let final_stage = &distributed_stages[distributed_stages.len() - 1]; let schema = Arc::clone(&final_stage.plan.schema()); let final_stage_id = final_stage.stage_id; // distribute the stages to workers, further dividing them up // into chunks of partitions (partition_groups) - let final_workers = distribute_stages(&base_result.query_id, base_result.distributed_stages, worker_addrs).await?; + let (final_workers, tasks) = + distribute_stages(&query_id, distributed_stages, worker_addrs).await?; - Ok(QueryPlan { - query_id: base_result.query_id, + let qp = QueryPlan { + query_id, + session_context: ctx, worker_addresses: final_workers, final_stage_id, schema, - explain_data: None, - }) - } - - /// Prepare an EXPLAIN query - /// This method only handles EXPLAIN queries (plan only). EXPLAIN ANALYZE queries are handled as regular queries because they need to be executed. - pub async fn prepare_explain(&self, sql: &str) -> Result { - // Validate that this is actually an EXPLAIN query (not EXPLAIN ANALYZE) - if !is_explain_query(sql) { - return Err(anyhow!("prepare_explain called with non-EXPLAIN query or EXPLAIN ANALYZE query: {}", sql).into()); - } - - // Extract the underlying query from the EXPLAIN statement - let underlying_query = sql.trim() - .strip_prefix("EXPLAIN") - .or_else(|| sql.trim().to_uppercase().strip_prefix("EXPLAIN").map(|_| &sql.trim()[7..])) - .unwrap_or(sql) - .trim(); - - let base_result = self.prepare_query_base(underlying_query, "EXPLAIN").await?; - - // generate the plan strings - let logical_plan_string = format!("{}", base_result.logical_plan.display_indent()); - let physical_plan_string = format!("{}", datafusion::physical_plan::displayable(base_result.physical_plan.as_ref()).indent(true)); - let distributed_plan_string = format!("{}", datafusion::physical_plan::displayable(base_result.distributed_plan.as_ref()).indent(true)); - let distributed_stages_string = DistributedExplainExec::format_distributed_stages(base_result.distributed_stages.as_slice()); - - // create the schema for EXPLAIN results - use arrow::datatypes::{DataType, Field, Schema}; - let schema = Arc::new(Schema::new(vec![ - Field::new("plan_type", DataType::Utf8, false), - Field::new("plan", DataType::Utf8, false), - ])); - - // Create explain data - let explain_data = DistributedExplainExec::new( - Arc::clone(&schema), - logical_plan_string, - physical_plan_string, - distributed_plan_string, - distributed_stages_string, - ); - - // Create dummy addresses for EXPLAIN (no real workers needed) - let mut dummy_addrs = std::collections::HashMap::new(); - let mut partition_addrs = std::collections::HashMap::new(); - partition_addrs.insert(0u64, vec![]); - dummy_addrs.insert(0u64, partition_addrs); + logical_plan, + physical_plan, + distributed_plan, + distributed_tasks: tasks, + }; - Ok(QueryPlan { - query_id: base_result.query_id, - worker_addresses: dummy_addrs, - final_stage_id: 0, - schema, - explain_data: Some(explain_data), - }) + Ok(qp) } } - -#[cfg(test)] -mod tests { - use super::*; - - // ////////////////////////////////////////////////////////////// - // Test helper functions - // ////////////////////////////////////////////////////////////// - - /// Set up mock worker environment for testing - fn setup_mock_worker_env() { - let mock_addrs = vec![ - ("mock_worker_1".to_string(), "localhost:9001".to_string()), - ("mock_worker_2".to_string(), "localhost:9002".to_string()), - ]; - let mock_env_value = mock_addrs.iter() - .map(|(name, addr)| format!("{}/{}", name, addr)) - .collect::>() - .join(","); - std::env::set_var("DFRAY_WORKER_ADDRESSES", &mock_env_value); - } - - // ////////////////////////////////////////////////////////////// - // Core function tests - // ////////////////////////////////////////////////////////////// - - #[tokio::test] - async fn test_prepare_query_base() { - let planner = QueryPlanner::new(); - - // Test with a simple SELECT query without the need to read any table - let sql = "SELECT 1 as test_col"; - let result = planner.prepare_query_base(sql, "TEST").await; - - if result.is_ok() { - let query_plan_base = result.unwrap(); - // verify all fields have values - assert!(!query_plan_base.query_id.is_empty()); - assert!(!query_plan_base.distributed_stages.is_empty()); - assert!(!query_plan_base.physical_plan.schema().fields().is_empty()); - // logical plan of select 1 on empty relation - assert_eq!(query_plan_base.logical_plan.to_string(), "Projection: Int64(1) AS test_col\n EmptyRelation"); - // physical plan of select 1 on empty releation is ProjectionExec - assert_eq!(query_plan_base.physical_plan.name(), "ProjectionExec"); - } else { - // If worker discovery fails, we expect a specific error - let error_msg = format!("{:?}", result.unwrap_err()); - assert!(error_msg.contains("worker") || error_msg.contains("address")); - } - } - - #[tokio::test] - async fn test_prepare_explain() { - let planner = QueryPlanner::new(); - - // Test with a simple EXPLAIN query - let sql = "EXPLAIN SELECT 1 as test_col"; - let query_plan = planner.prepare_explain(sql).await.unwrap(); - - // EXPLAIN queries should work even without worker discovery since they use dummy addresses - assert!(!query_plan.query_id.is_empty()); - assert_eq!(query_plan.final_stage_id, 0); - assert!(query_plan.explain_data.is_some()); - - // Verify content of the explain data - let explain_data = query_plan.explain_data.unwrap(); - assert_eq!(explain_data.logical_plan(), "Projection: Int64(1) AS test_col\n EmptyRelation"); - assert_eq!(explain_data.physical_plan(), "ProjectionExec: expr=[1 as test_col]\n PlaceholderRowExec\n"); - assert_eq!(explain_data.distributed_plan(), - "RayStageExec[0] (output_partitioning=UnknownPartitioning(1))\n ProjectionExec: expr=[1 as test_col]\n PlaceholderRowExec\n"); - assert_eq!(explain_data.distributed_stages(), - "Stage 0:\n Partition Groups: [[0]]\n Full Partitions: false\n Plan:\n MaxRowsExec[max_rows=8192]\n CoalesceBatchesExec: target_batch_size=8192\n ProjectionExec: expr=[1 as test_col]\n PlaceholderRowExec\n"); - - // Should have explain schema (plan_type, plan columns) - assert_eq!(query_plan.schema.fields().len(), 2); - assert_eq!(query_plan.schema.field(0).name(), "plan_type"); - assert_eq!(query_plan.schema.field(1).name(), "plan"); - println!("✓ prepare_explain_query succeeded with proper structure"); - - } - - #[tokio::test] - async fn test_prepare_explain_invalid_input() { - let planner = QueryPlanner::new(); - - // Test with EXPLAIN ANALYZE (should fail) - let sql = "EXPLAIN ANALYZE SELECT 1"; - let result = planner.prepare_explain(sql).await; - assert!(result.is_err()); - let error_msg = format!("{:?}", result.unwrap_err()); - assert!(error_msg.contains("prepare_explain called with non-EXPLAIN query")); - - // Test with non-EXPLAIN query (should fail) - let sql = "SELECT 1"; - let result = planner.prepare_explain(sql).await; - assert!(result.is_err()); - let error_msg = format!("{:?}", result.unwrap_err()); - assert!(error_msg.contains("prepare_explain called with non-EXPLAIN query")); - } - - // NOTE: This test is ignored because prepare_query() requires actual worker communication. - // - // 🔍 Root Cause Analysis: - // The issue is NOT with mock worker setup - that works perfectly. The problem is in the - // distribute_stages() retry logic: - // - // 1. ✅ Mock workers are set up correctly: ["mock_worker_1/localhost:9001", "mock_worker_2/localhost:9002"] - // 2. ✅ get_worker_addresses() successfully returns: [("mock_worker_1", "localhost:9001"), ("mock_worker_2", "localhost:9002")] - // 3. ✅ distribute_stages() receives workers and creates HashMap: {"mock_worker_1": "localhost:9001", "mock_worker_2": "localhost:9002"} - // 4. ❌ try_distribute_stages() attempts to create Flight client connections to mock workers (which don't exist) - // 5. ❌ Each connection fails, returning WorkerCommunicationError("mock_worker_X") - // 6. ❌ Retry logic removes "failed" workers: first removes mock_worker_2, then mock_worker_1 - // 7. ❌ After 3 retries, workers HashMap is empty: {} - // 8. ❌ assign_to_workers() panics when trying to access worker_addrs[0] on empty list - // - // 💡 Solutions for proper testing: - // - Mock the Flight client layer (complex, requires significant refactoring) - // - Create a test-only version of distribute_stages() that skips communication - // - Refactor the architecture to use dependency injection for better testability - // - Use integration tests with actual worker processes instead of unit tests - // - // For now, we focus on testing the individual components that don't require worker communication. - #[tokio::test] - #[ignore] - async fn test_prepare_query() { - setup_mock_worker_env(); - let planner = QueryPlanner::new(); - - // Test with a simple SELECT query - let sql = "SELECT 1 as test_col, 'hello' as text_col"; - let result = planner.prepare_query(sql).await; - - match result { - Ok(query_plan) => { - assert!(query_plan.explain_data.is_none()); - assert!(!query_plan.query_id.is_empty()); - assert!(!query_plan.worker_addresses.is_empty()); - assert_eq!(query_plan.schema.fields().len(), 2); - assert_eq!(query_plan.schema.field(0).name(), "test_col"); - assert_eq!(query_plan.schema.field(1).name(), "text_col"); - println!("✓ prepare_query succeeded with proper structure"); - } - Err(e) => { - let error_msg = format!("{:?}", e); - assert!( - error_msg.contains("worker") || - error_msg.contains("address") || - error_msg.contains("DFRAY_WORKER") || - error_msg.contains("index out of bounds"), - "Unexpected error type: {}", error_msg - ); - println!("✓ prepare_query failed with expected worker discovery error: {}", error_msg); - } - } - } -} diff --git a/src/stage_reader.rs b/src/stage_reader.rs index 7214670..3f1343a 100644 --- a/src/stage_reader.rs +++ b/src/stage_reader.rs @@ -1,10 +1,6 @@ use std::{fmt::Formatter, sync::Arc}; -use arrow_flight::{ - decode::{DecodedPayload, FlightRecordBatchStream}, - error::FlightError, - Ticket, -}; +use arrow_flight::{decode::FlightRecordBatchStream, error::FlightError, Ticket}; use datafusion::{ arrow::datatypes::SchemaRef, common::{internal_datafusion_err, internal_err}, @@ -16,7 +12,7 @@ use datafusion::{ stream::RecordBatchStreamAdapter, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, }, - prelude::{SessionConfig, SessionContext}, + prelude::SessionConfig, }; use futures::{stream::TryStreamExt, StreamExt}; use prost::Message; @@ -25,7 +21,7 @@ use crate::{ logging::{error, trace}, protobuf::{FlightDataMetadata, FlightTicketData}, util::{get_client, CombinedRecordBatchStream}, - vocab::{CtxAnnotatedOutputs, CtxName, CtxStageAddrs}, + vocab::{CtxAnnotatedOutputs, CtxHost, CtxStageAddrs}, }; pub(crate) struct QueryId(pub String); @@ -129,9 +125,9 @@ impl ExecutionPlan for DFRayStageReaderExec { let ctx_name = &context .session_config() - .get_extension::() - .unwrap_or(Arc::new(CtxName("unknown ctx".to_string()))) - .0; + .get_extension::() + .map(|ctx_host| ctx_host.0.to_string()) + .unwrap_or("unknown_context_host!".to_string()); trace!(" trying to get clients for {:?}", stage_addrs); let clients = stage_addrs @@ -152,7 +148,7 @@ impl ExecutionPlan for DFRayStageReaderExec { )) })?? .iter() - .map(|host| get_client(host)) + .map(get_client) .collect::>>()?; trace!("got clients. {name} num clients: {}", clients.len()); @@ -270,9 +266,6 @@ fn make_flight_metadata_saver_stream( yield Err(e); } (false, None) => { - // we should not reach this block, because we decide we have reach the end when - // we receive a valid message with trailing data - error!("{name} Unexpected arrival in block we do not expect!"); done = true; } (true,_) => { diff --git a/src/test_utils.rs b/src/test_utils.rs deleted file mode 100644 index c7a6c43..0000000 --- a/src/test_utils.rs +++ /dev/null @@ -1,113 +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. - -//! Shared test utilities for DataFusion Distributed tests -//! -//! This module contains common test helper functions that are used across -//! multiple test modules to avoid code duplication. - -#[cfg(test)] -pub mod explain_test_helpers { - use arrow_flight::decode::FlightRecordBatchStream; - use datafusion::physical_plan::ExecutionPlan; - use futures::StreamExt; - - use crate::{ - flight_handlers::FlightRequestHandler, - protobuf::{DistributedExplainExecNode, TicketStatementData}, - proxy_service::DfRayProxyHandler, - query_planner::{QueryPlan, QueryPlanner}, - }; - - /// Create a test FlightRequestHandler for testing - pub fn create_test_flight_handler() -> FlightRequestHandler { - FlightRequestHandler::new(QueryPlanner::new()) - } - - /// Create a test DfRayProxyHandler for testing - bypasses worker discovery initialization - pub fn create_test_proxy_handler() -> DfRayProxyHandler { - // Create the handler directly without calling new() to avoid worker discovery - // during test initialization. - DfRayProxyHandler { - flight_handler: FlightRequestHandler::new(QueryPlanner::new()), - } - } - - /// Create TicketStatementData for EXPLAIN testing from QueryPlan - pub fn create_explain_ticket_statement_data(plans: QueryPlan) -> TicketStatementData { - let explain_data = plans.explain_data.map(|data| { - DistributedExplainExecNode { - schema: data.schema().as_ref().try_into().ok(), - logical_plan: data.logical_plan().to_string(), - physical_plan: data.physical_plan().to_string(), - distributed_plan: data.distributed_plan().to_string(), - distributed_stages: data.distributed_stages().to_string(), - } - }); - - TicketStatementData { - query_id: plans.query_id, - stage_id: plans.final_stage_id, - stage_addrs: Some(plans.worker_addresses.into()), - schema: Some(plans.schema.as_ref().try_into().unwrap()), - explain_data, - } - } - - /// Consume a DoGetStream and verify it contains expected EXPLAIN results - pub async fn verify_explain_stream_results(stream: crate::flight::DoGetStream) { - - // Convert the stream to a FlightRecordBatchStream and consume it - // Map Status errors to FlightError to match the expected stream type - let mapped_stream = stream.map(|result| { - result.map_err(|status| arrow_flight::error::FlightError::from(status)) - }); - let mut flight_stream = FlightRecordBatchStream::new_from_flight_data(mapped_stream); - - let mut batches = Vec::new(); - while let Some(batch_result) = flight_stream.next().await { - let batch = batch_result.expect("Failed to get batch from stream"); - batches.push(batch); - } - - // Verify we got exactly one batch with EXPLAIN results - assert_eq!(batches.len(), 1); - let batch = &batches[0]; - - // Verify schema: should have 2 columns (plan_type, plan) - assert_eq!(batch.num_columns(), 2); - assert_eq!(batch.schema().field(0).name(), "plan_type"); - assert_eq!(batch.schema().field(1).name(), "plan"); - - // Verify we have 4 rows (logical_plan, physical_plan, distributed_plan, distributed_stages) - assert_eq!(batch.num_rows(), 4); - - // Verify the plan_type column contains the expected values - let plan_type_column = batch.column(0).as_any().downcast_ref::().unwrap(); - assert_eq!(plan_type_column.value(0), "logical_plan"); - assert_eq!(plan_type_column.value(1), "physical_plan"); - assert_eq!(plan_type_column.value(2), "distributed_plan"); - assert_eq!(plan_type_column.value(3), "distributed_stages"); - - // Verify the plan column contains actual plan content - let plan_column = batch.column(1).as_any().downcast_ref::().unwrap(); - assert!(plan_column.value(0).contains("Projection: Int64(1) AS test_col")); - assert!(plan_column.value(1).contains("ProjectionExec")); - assert!(plan_column.value(2).contains("RayStageExec")); - assert!(plan_column.value(3).contains("Stage 0:")); - } -} \ No newline at end of file diff --git a/src/util.rs b/src/util.rs index b9907d2..d5e9e8c 100644 --- a/src/util.rs +++ b/src/util.rs @@ -40,6 +40,7 @@ use parking_lot::RwLock; use prost::Message; use tokio::{ macros::support::thread_rng_n, + net::TcpListener, runtime::{Handle, Runtime}, }; use tonic::transport::Channel; @@ -613,6 +614,16 @@ pub fn maybe_register_object_store(ctx: &SessionContext, url: &Url) -> Result<() Ok(()) } +pub async fn start_up(port: usize) -> Result { + let my_host_str = format!("0.0.0.0:{}", port); + + let listener = TcpListener::bind(&my_host_str) + .await + .context("Could not bind socket to {my_host_str}")?; + + Ok(listener) +} + #[cfg(test)] mod test { use std::{sync::Arc, vec}; diff --git a/src/vocab.rs b/src/vocab.rs index 8af334f..54fd60f 100644 --- a/src/vocab.rs +++ b/src/vocab.rs @@ -3,11 +3,11 @@ use std::{collections::HashMap, fmt::Display, sync::Arc}; use parking_lot::Mutex; pub use crate::protobuf::AnnotatedTaskOutput; +pub use crate::protobuf::DdTask as DDTask; pub use crate::protobuf::Host; pub use crate::protobuf::Hosts; pub use crate::protobuf::PartitionAddrs; pub use crate::protobuf::StageAddrs; -pub use crate::protobuf::StageData; /// a map of stage_id, partition to a list (name,endpoint address) that can /// serve this (stage_id, and partition). It is assumed that to consume a @@ -20,12 +20,15 @@ pub type Addrs = HashMap>>; /// used to hold an Addrs as an extenstion for datafusion SessionContext pub(crate) struct CtxStageAddrs(pub Addrs); -/// used to hold a worker name as an extension for datafusion SessionContext -pub(crate) struct CtxName(pub String); +/// used to hold information about how is executing +pub(crate) struct CtxHost(pub Host); /// used to hold a partition group as an extension for datafusion SessionContext pub(crate) struct CtxPartitionGroup(pub Vec); +/// used to hold a stage id as an extension for datafusion SessionContext +pub(crate) struct CtxStageId(pub u64); + #[derive(Default)] pub(crate) struct CtxAnnotatedOutputs(pub Arc>>); From 18bff98322eb0592163bf97f54cb0fdf52211f1a Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 10 Jul 2025 12:41:17 -0400 Subject: [PATCH 4/6] explain refactor and explain analyze and cleanup --- src/analyze.rs | 30 +-- src/codec.rs | 19 +- src/explain.rs | 279 +++++------------------- src/lib.rs | 3 +- src/physical.rs | 14 +- src/planning.rs | 10 +- src/proto/distributed_datafusion.proto | 132 +++++++++++ src/proto/generated/protobuf.rs | 9 +- src/proxy_service.rs | 12 +- src/query_planner.rs | 70 ++++-- src/record_batch_exec.rs | 81 +++++++ src/util.rs | 33 ++- src/worker_discovery.rs | 291 +++++++++++++++++++++++++ 13 files changed, 704 insertions(+), 279 deletions(-) create mode 100644 src/proto/distributed_datafusion.proto create mode 100644 src/record_batch_exec.rs create mode 100644 src/worker_discovery.rs diff --git a/src/analyze.rs b/src/analyze.rs index 20ba5c5..5828d17 100644 --- a/src/analyze.rs +++ b/src/analyze.rs @@ -9,6 +9,7 @@ use datafusion::{ execution::SendableRecordBatchStream, physical_expr::EquivalenceProperties, physical_plan::{ + coalesce_partitions::CoalescePartitionsExec, display::DisplayableExecutionPlan, execution_plan::{Boundedness, EmissionType}, stream::RecordBatchStreamAdapter, @@ -178,16 +179,6 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { partition: usize, context: std::sync::Arc, ) -> Result { - let input_capture = self.input.clone(); - let show_statistics_capture = self.show_statistics; - let verbose_capture = self.verbose; - let fmt_plan = move || -> String { - DisplayableExecutionPlan::with_metrics(input_capture.as_ref()) - .set_show_statistics(show_statistics_capture) - .indent(verbose_capture) - .to_string() - }; - let task_outputs = context .session_config() .get_extension::() @@ -234,9 +225,22 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { .0 .clone(); - let mut input_stream = self.input.execute(partition, context)?; + // we want to gather all partitions + let coalesce = CoalescePartitionsExec::new(self.input.clone()); + + let mut input_stream = coalesce.execute(partition, context)?; - let schema_clone = self.schema().clone(); + let schema_capture = self.schema().clone(); + let input_capture = self.input.clone(); + let show_statistics_capture = self.show_statistics; + let verbose_capture = self.verbose; + + let fmt_plan = move || -> String { + DisplayableExecutionPlan::with_metrics(input_capture.as_ref()) + .set_show_statistics(show_statistics_capture) + .indent(verbose_capture) + .to_string() + }; let output = async move { // consume input, and we do not have to send it downstream as we are the @@ -288,7 +292,7 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { } RecordBatch::try_new( - schema_clone, + schema_capture, vec![ Arc::new(task_builder.finish()), Arc::new(plan_builder.finish()), diff --git a/src/codec.rs b/src/codec.rs index 390baf4..346b768 100644 --- a/src/codec.rs +++ b/src/codec.rs @@ -23,9 +23,11 @@ use crate::{ protobuf::{ df_ray_exec_node::Payload, DfRayExecNode, DfRayStageReaderExecNode, DistributedAnalyzeExecNode, DistributedAnalyzeRootExecNode, MaxRowsExecNode, - PartitionIsolatorExecNode, + PartitionIsolatorExecNode, RecordBatchExecNode, }, + record_batch_exec::RecordBatchExec, stage_reader::DFRayStageReaderExec, + util::{batch_to_ipc, ipc_to_batch}, }; #[derive(Debug)] @@ -117,6 +119,14 @@ impl PhysicalExtensionCodec for DFRayCodec { ))) } } + Payload::RecordBatchExec(rb_exec) => { + // deserialize the record batch stored in the opaque bytes field + let batch = ipc_to_batch(&rb_exec.batch).map_err(|e| { + internal_datafusion_err!("Failed to decode RecordBatch: {:#?}", e) + })?; + + Ok(Arc::new(RecordBatchExec::new(batch))) + } } } else { internal_err!("cannot decode proto extension in dfray codec") @@ -166,6 +176,13 @@ impl PhysicalExtensionCodec for DFRayCodec { show_statistics: exec.show_statistics, }; Payload::DistributedAnalyzeRootExec(pb) + } else if let Some(exec) = node.as_any().downcast_ref::() { + let pb = RecordBatchExecNode { + batch: batch_to_ipc(&exec.batch).map_err(|e| { + internal_datafusion_err!("Failed to encode RecordBatch: {:#?}", e) + })?, + }; + Payload::RecordBatchExec(pb) } else { return internal_err!("Not supported node to encode to proto"); }; diff --git a/src/explain.rs b/src/explain.rs index 1c7a406..5bf6968 100644 --- a/src/explain.rs +++ b/src/explain.rs @@ -1,234 +1,73 @@ -// 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::{any::Any, fmt::Formatter, sync::Arc}; +use std::sync::Arc; use anyhow::Context; -use arrow::{array::StringArray, datatypes::SchemaRef, record_batch::RecordBatch}; +use arrow::{ + array::StringArray, + datatypes::{DataType, Field, Schema}, + record_batch::RecordBatch, +}; use datafusion::{ - execution::TaskContext, logical_expr::LogicalPlan, - physical_expr::EquivalenceProperties, - physical_plan::{ - displayable, - execution_plan::{Boundedness, EmissionType}, - memory::MemoryStream, - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, - }, + physical_plan::{displayable, ExecutionPlan}, prelude::SessionContext, }; -use crate::{ - planning::{get_ctx, logical_planning}, - result::Result, - util::bytes_to_physical_plan, - vocab::DDTask, -}; - -/// Custom distributed EXPLAIN execution plan that also returns distributed plan and stages -#[derive(Debug)] -pub struct DistributedExplainExec { - schema: SchemaRef, - logical_plan: String, - physical_plan: String, - distributed_plan: String, - distributed_tasks: String, - properties: PlanProperties, -} - -impl DistributedExplainExec { - pub fn new( - schema: SchemaRef, - logical_plan: String, - physical_plan: String, - distributed_plan: String, - distributed_tasks: String, - ) -> Self { - // properties required by the ExecutionPlan trait - let properties = PlanProperties::new( - EquivalenceProperties::new(schema.clone()), - Partitioning::UnknownPartitioning(1), - EmissionType::Incremental, - Boundedness::Bounded, - ); - - Self { - schema, - logical_plan, - physical_plan, - distributed_plan, - distributed_tasks, - properties, +use crate::{result::Result, util::bytes_to_physical_plan, vocab::DDTask}; + +pub fn format_distributed_tasks(tasks: &[DDTask]) -> Result { + let mut result = String::new(); + for (i, task) in tasks.iter().enumerate() { + let plan = bytes_to_physical_plan(&SessionContext::new(), &task.plan_bytes) + .context("unable to decode task plan for formatted output")?; + + result.push_str(&format!( + "Task: Stage {}, Partitions {:?}\n", + task.stage_id, task.partition_group + )); + result.push_str(&format!(" Full Partitions: {}\n", task.full_partitions)); + result.push_str(" Plan:\n"); + let plan_display = format!("{}", displayable(plan.as_ref()).indent(true)); + for line in plan_display.lines() { + result.push_str(&format!(" {}\n", line)); } - } - - pub fn logical_plan(&self) -> &str { - &self.logical_plan - } - - pub fn physical_plan(&self) -> &str { - &self.physical_plan - } - - pub fn distributed_plan(&self) -> &str { - &self.distributed_plan - } - - pub fn distributed_tasks(&self) -> &str { - &self.distributed_tasks - } - - /// Format distributed tasks for display - pub fn format_distributed_tasks(tasks: &[DDTask]) -> Result { - let mut result = String::new(); - for (i, task) in tasks.iter().enumerate() { - let plan = bytes_to_physical_plan(&SessionContext::new(), &task.plan_bytes) - .context(format!("unable to decode task plan for formatted output"))?; - - result.push_str(&format!("Stage {}:\n", task.stage_id)); - result.push_str(&format!(" Partition Group: {:?}\n", task.partition_group)); - result.push_str(&format!(" Full Partitions: {}\n", task.full_partitions)); - result.push_str(" Plan:\n"); - let plan_display = format!("{}", displayable(plan.as_ref()).indent(true)); - for line in plan_display.lines() { - result.push_str(&format!(" {}\n", line)); - } - if i < tasks.len() - 1 { - result.push('\n'); - } - } - if result.is_empty() { - result.push_str("No distributed tasks generated"); + if i < tasks.len() - 1 { + result.push('\n'); } - Ok(result) } -} - -impl DisplayAs for DistributedExplainExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "DistributedExplainExec") + if result.is_empty() { + result.push_str("No distributed tasks generated"); } + Ok(result) } -impl ExecutionPlan for DistributedExplainExec { - fn name(&self) -> &str { - "DistributedExplainExec" - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.properties - } - - fn children(&self) -> Vec<&Arc> { - vec![] - } - - fn with_new_children( - self: Arc, - _children: Vec>, - ) -> datafusion::error::Result> { - Ok(self) - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> datafusion::error::Result { - let schema = self.schema.clone(); - - // Create the result data with our 4 plan types - let plan_types = StringArray::from(vec![ - "logical_plan", - "physical_plan", - "distributed_plan", - "distributed_tasks", - ]); - let plans = StringArray::from(vec![ - self.logical_plan.as_str(), - self.physical_plan.as_str(), - self.distributed_plan.as_str(), - self.distributed_tasks.as_str(), - ]); - - let batch = - RecordBatch::try_new(schema.clone(), vec![Arc::new(plan_types), Arc::new(plans)]) - .map_err(|e| datafusion::error::DataFusionError::ArrowError(e, None))?; - - // Use MemoryStream which is designed for DataFusion execution plans - let stream = MemoryStream::try_new(vec![batch], schema, None)?; - - Ok(Box::pin(stream)) - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - -/// Check if this is an EXPLAIN query (but not EXPLAIN ANALYZE) -/// -/// This function distinguishes between: -/// - EXPLAIN queries (returns true) - show plan information only -/// - EXPLAIN ANALYZE queries (returns false) - execute and show runtime stats -/// - Regular queries (returns false) - normal query execution -pub async fn is_explain_query(sql: &str) -> Result { - let ctx = get_ctx().map_err(|e| anyhow!("Could not create context: {e}"))?; - let logical_plan = logical_planning(sql, &ctx).await?; - - match logical_plan { - LogicalPlan::Explain(_) => Ok(true), - _ => Ok(false), // Not an EXPLAIN plan - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_is_explain_query() { - // Test EXPLAIN queries (should return true) - assert!(is_explain_query("EXPLAIN SELECT * FROM table")); - assert!(is_explain_query("explain select * from table")); - assert!(is_explain_query(" EXPLAIN SELECT 1")); - assert!(is_explain_query("EXPLAIN\nSELECT * FROM test")); - - // Test EXPLAIN ANALYZE queries (should return false) - assert!(!is_explain_query("EXPLAIN ANALYZE SELECT * FROM table")); - assert!(!is_explain_query("explain analyze SELECT * FROM table")); - assert!(!is_explain_query(" EXPLAIN ANALYZE SELECT 1")); - - // Test regular queries (should return false) - assert!(!is_explain_query("SELECT * FROM table")); - assert!(!is_explain_query("INSERT INTO table VALUES (1)")); - assert!(!is_explain_query("UPDATE table SET col = 1")); - assert!(!is_explain_query("DELETE FROM table")); - assert!(!is_explain_query("CREATE TABLE test (id INT)")); - - // Test edge cases - assert!(!is_explain_query("")); - assert!(!is_explain_query(" ")); - assert!(!is_explain_query("EXPLAINSELECT")); // No space - assert!(is_explain_query("EXPLAIN")); // Just EXPLAIN - } +pub fn build_explain_batch( + logical_plan: &LogicalPlan, + physical_plan: &Arc, + distributed_plan: &Arc, + distributed_tasks: &[DDTask], +) -> Result { + let schema = Arc::new(Schema::new(vec![ + Field::new("plan_type", DataType::Utf8, false), + Field::new("plan", DataType::Utf8, false), + ])); + + // Create the result data with our 4 plan types + let plan_types = StringArray::from(vec![ + "logical_plan", + "physical_plan", + "distributed_plan", + "distributed_tasks", + ]); + let plans = StringArray::from(vec![ + logical_plan.display_indent().to_string(), + displayable(physical_plan.as_ref()).indent(true).to_string(), + displayable(distributed_plan.as_ref()) + .indent(true) + .to_string(), + format_distributed_tasks(distributed_tasks)?, + ]); + + let batch = RecordBatch::try_new(schema, vec![Arc::new(plan_types), Arc::new(plans)])?; + + Ok(batch) } diff --git a/src/lib.rs b/src/lib.rs index a68372f..e6198db 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -24,7 +24,7 @@ pub use proto::generated::protobuf; pub mod analyze; pub mod codec; -//pub mod explain; +pub mod explain; pub mod flight; pub mod friendly; pub mod isolator; @@ -35,6 +35,7 @@ pub mod planning; pub mod processor_service; pub mod proxy_service; pub mod query_planner; +pub mod record_batch_exec; pub mod result; pub mod stage; pub mod stage_reader; diff --git a/src/physical.rs b/src/physical.rs index 78844cf..ac94e9c 100644 --- a/src/physical.rs +++ b/src/physical.rs @@ -22,18 +22,12 @@ use datafusion::{ error::Result, physical_optimizer::PhysicalOptimizerRule, physical_plan::{ - analyze::AnalyzeExec, coalesce_partitions::CoalescePartitionsExec, joins::NestedLoopJoinExec, repartition::RepartitionExec, sorts::sort::SortExec, ExecutionPlan, }, }; -use crate::{ - analyze::{DistributedAnalyzeExec, DistributedAnalyzeRootExec}, - logging::info, - stage::DFRayStageExec, - util::display_plan_with_partition_counts, -}; +use crate::{logging::info, stage::DFRayStageExec, util::display_plan_with_partition_counts}; /// This optimizer rule walks up the physical plan tree /// and inserts RayStageExec nodes where appropriate to denote where we will @@ -108,6 +102,8 @@ impl PhysicalOptimizerRule for DFRayStageOptimizerRule { #[cfg(test)] mod tests { + use crate::analyze::DistributedAnalyzeRootExec; + use super::*; use datafusion::arrow::array::{Int32Array, StringArray}; use datafusion::arrow::datatypes::{DataType, Field, Schema}; @@ -163,9 +159,7 @@ mod tests { .await .unwrap(); - let coalesce = Arc::new(CoalescePartitionsExec::new(data_source)); - - let analyze = Arc::new(DistributedAnalyzeRootExec::new(coalesce, false, false)); + let analyze = Arc::new(DistributedAnalyzeRootExec::new(data_source, false, false)); let target_plan = DFRayStageExec::new( analyze, 0, // stage counter diff --git a/src/planning.rs b/src/planning.rs index efb6a1d..07923a7 100644 --- a/src/planning.rs +++ b/src/planning.rs @@ -18,10 +18,9 @@ use datafusion::{ logical_expr::LogicalPlan, physical_optimizer::PhysicalOptimizerRule, physical_plan::{ - analyze::AnalyzeExec, coalesce_batches::CoalesceBatchesExec, - coalesce_partitions::CoalescePartitionsExec, displayable, joins::NestedLoopJoinExec, - repartition::RepartitionExec, sorts::sort::SortExec, ExecutionPlan, - ExecutionPlanProperties, + analyze::AnalyzeExec, coalesce_batches::CoalesceBatchesExec, displayable, + joins::NestedLoopJoinExec, repartition::RepartitionExec, sorts::sort::SortExec, + ExecutionPlan, ExecutionPlanProperties, }, prelude::{SQLOptions, SessionConfig, SessionContext}, }; @@ -417,8 +416,7 @@ pub fn add_distributed_analyze( displayable(plan_without_analyze.as_ref()).indent(false) ); stage.plan = Arc::new(DistributedAnalyzeRootExec::new( - Arc::new(CoalescePartitionsExec::new(plan_without_analyze)) - as Arc, + plan_without_analyze, verbose, show_statistics, )) as Arc; diff --git a/src/proto/distributed_datafusion.proto b/src/proto/distributed_datafusion.proto new file mode 100644 index 0000000..e6d4a8c --- /dev/null +++ b/src/proto/distributed_datafusion.proto @@ -0,0 +1,132 @@ +syntax = "proto3"; + +package distributed_datafusion.protobuf; + +import "datafusion.proto"; +import "datafusion_common.proto"; + +option java_multiple_files = true; +option java_outer_classname = "DistributedDataFusionProto"; +option java_package = "distributed_datafusion.protobuf"; + +message DFRayStageReaderExecNode { + // schema of the stage we will consume + datafusion_common.Schema schema = 1; + // properties of the stage we will consume + datafusion.Partitioning partitioning = 2; + // stage to read from + uint64 stage_id = 3; +} + +message MaxRowsExecNode { + uint64 max_rows = 1; +} + +message PartitionIsolatorExecNode { + uint64 partition_count = 2; +} + +message DistributedAnalyzeExecNode { + // how much data to show + bool verbose = 1; + // if statistics should be displayed + bool show_statistics = 2; +} + +message DistributedAnalyzeRootExecNode { + // how much data to show + bool verbose = 1; + // if statistics should be displayed + bool show_statistics = 2; +} + +message RecordBatchExecNode { + bytes batch = 1; +} + +message DFRayExecNode { + oneof payload { + DFRayStageReaderExecNode stage_reader_exec = 1; + MaxRowsExecNode max_rows_exec = 2; + PartitionIsolatorExecNode isolator_exec = 3; + DistributedAnalyzeExecNode distributed_analyze_exec = 4; + DistributedAnalyzeRootExecNode distributed_analyze_root_exec = 5; + RecordBatchExecNode record_batch_exec = 6; + } +} + +message AnnotatedTaskOutput { + // the output of the explain analyze + string plan = 1; + // the host who executed this stage + Host host = 2; + // the stage id that was executed + uint64 stage_id = 3; + // the partitions that were executed by this stage + repeated uint64 partition_group = 4; +} + +message AnnotatedTaskOutputs { + repeated AnnotatedTaskOutput outputs = 1; +} + +message FlightDataMetadata { + AnnotatedTaskOutputs annotated_task_outputs = 1; +} + +message FlightTicketData { + string query_id = 1; + uint64 stage_id = 2; + uint64 partition = 3; + // name of the requestor, useful in logs for debugging + string requestor_name = 4; +} + +message TicketStatementData { + // identity of the query we want to consume + string query_id = 1; + // the stage id of the final stage + uint64 stage_id = 2; + // host holding the final stage, the root of the query plan + StageAddrs stage_addrs = 3; + // the schema of the final stage + datafusion_common.Schema schema = 4; +} + +message Host { + string addr = 1; + string name = 2; +} + +message Hosts { + repeated Host hosts = 1; +} + +message PartitionAddrs { + map partition_addrs = 1; +} + +message StageAddrs { + // map of stage id to stage address + map stage_addrs = 1; +} + +message DDTask { + string query_id = 1; + + uint64 stage_id = 2; + + repeated uint64 partition_group = 3; + + bytes plan_bytes = 4; + + repeated uint64 child_stage_ids = 5; + + StageAddrs stage_addrs = 6; + + uint64 num_output_partitions = 7; + + bool full_partitions = 8; + + Host assigned_host = 9; +} diff --git a/src/proto/generated/protobuf.rs b/src/proto/generated/protobuf.rs index fefe365..401aa10 100644 --- a/src/proto/generated/protobuf.rs +++ b/src/proto/generated/protobuf.rs @@ -41,15 +41,12 @@ pub struct DistributedAnalyzeRootExecNode { } #[derive(Clone, PartialEq, ::prost::Message)] pub struct RecordBatchExecNode { - #[prost(message, optional, tag = "1")] - pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, - /// the arrow IPC serialized record batch of the response - #[prost(bytes = "vec", tag = "2")] + #[prost(bytes = "vec", tag = "1")] pub batch: ::prost::alloc::vec::Vec, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct DfRayExecNode { - #[prost(oneof = "df_ray_exec_node::Payload", tags = "1, 2, 3, 4, 5")] + #[prost(oneof = "df_ray_exec_node::Payload", tags = "1, 2, 3, 4, 5, 6")] pub payload: ::core::option::Option, } /// Nested message and enum types in `DFRayExecNode`. @@ -66,6 +63,8 @@ pub mod df_ray_exec_node { DistributedAnalyzeExec(super::DistributedAnalyzeExecNode), #[prost(message, tag = "5")] DistributedAnalyzeRootExec(super::DistributedAnalyzeRootExecNode), + #[prost(message, tag = "6")] + RecordBatchExec(super::RecordBatchExecNode), } } #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/src/proxy_service.rs b/src/proxy_service.rs index c08e503..08a50fa 100644 --- a/src/proxy_service.rs +++ b/src/proxy_service.rs @@ -27,7 +27,7 @@ use arrow_flight::{ FlightDescriptor, FlightEndpoint, FlightInfo, Ticket, }; use datafusion::physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, displayable, ExecutionPlan, Partitioning, + coalesce_partitions::CoalescePartitionsExec, ExecutionPlan, Partitioning, }; use futures::TryStreamExt; use parking_lot::Mutex; @@ -167,12 +167,12 @@ impl FlightSqlHandler for DfRayProxyHandler { ) -> Result, Status> { let query_plan = self .planner - .prepare_query(&query.query) + .prepare(&query.query) .await .map_err(|e| Status::internal(format!("Could not prepare query {e:?}")))?; self.create_flight_info_response(query_plan) - .map(|flight_info| Response::new(flight_info)) + .map(Response::new) .context("Could not create flight info response") .map_err(|e| Status::internal(format!("Error creating flight info: {e:?}"))) } @@ -249,13 +249,11 @@ impl FlightSqlHandler for DfRayProxyHandler { /// /// It only responds to the DoGet Arrow Flight method pub struct DFRayProxyService { - name: String, listener: TcpListener, handler: Arc, addr: String, all_done_tx: Arc>>, all_done_rx: Option>, - port: usize, } impl DFRayProxyService { @@ -271,16 +269,14 @@ impl DFRayProxyService { info!("DFRayProcessorService bound to {addr}"); - let handler = Arc::new(DfRayProxyHandler::new(name.clone(), addr.clone())); + let handler = Arc::new(DfRayProxyHandler::new(name, addr.clone())); Ok(Self { - name, listener, handler, addr, all_done_tx, all_done_rx: Some(all_done_rx), - port, }) } diff --git a/src/query_planner.rs b/src/query_planner.rs index 1119709..5e7b029 100644 --- a/src/query_planner.rs +++ b/src/query_planner.rs @@ -7,11 +7,11 @@ use datafusion::{ }; use crate::{ - logging::debug, + explain::build_explain_batch, planning::{ distribute_stages, execution_planning, get_ctx, logical_planning, physical_planning, - DDStage, }, + record_batch_exec::RecordBatchExec, result::Result, vocab::{Addrs, DDTask}, worker_discovery::get_worker_addresses, @@ -30,15 +30,6 @@ pub struct QueryPlan { pub distributed_tasks: Vec, } -impl QueryPlan { - pub fn is_explain(&self) -> bool { - match self.logical_plan { - LogicalPlan::Explain { .. } => true, - _ => false, - } - } -} - impl std::fmt::Debug for QueryPlan { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("QueryPlan") @@ -76,13 +67,36 @@ impl QueryPlanner { /// /// Prepare a query by parsing the SQL, planning it, and distributing the /// physical plan into stages that can be executed by workers. - pub async fn prepare_query(&self, sql: &str) -> Result { - let query_id = uuid::Uuid::new_v4().to_string(); + pub async fn prepare(&self, sql: &str) -> Result { let ctx = get_ctx().map_err(|e| anyhow!("Could not create context: {e}"))?; let logical_plan = logical_planning(sql, &ctx).await?; + + match logical_plan { + p @ LogicalPlan::Explain(_) => self.prepare_explain(p, ctx).await, + // add other logical plans for local execution here following the pattern for explain + p => self.prepare_query(p, ctx).await, + } + } + + async fn prepare_query( + &self, + logical_plan: LogicalPlan, + ctx: SessionContext, + ) -> Result { let physical_plan = physical_planning(&logical_plan, &ctx).await?; + self.send_it(logical_plan, physical_plan, ctx).await + } + + async fn send_it( + &self, + logical_plan: LogicalPlan, + physical_plan: Arc, + ctx: SessionContext, + ) -> Result { + let query_id = uuid::Uuid::new_v4().to_string(); + // divide the physical plan into chunks (tasks) that we can distribute to workers let (distributed_plan, distributed_stages) = execution_planning(physical_plan.clone(), 8192, Some(2)).await?; @@ -114,4 +128,34 @@ impl QueryPlanner { Ok(qp) } + + async fn prepare_explain( + &self, + explain_plan: LogicalPlan, + ctx: SessionContext, + ) -> Result { + let child_plan = explain_plan.inputs(); + if child_plan.len() != 1 { + return Err(anyhow!("EXPLAIN plan must have exactly one child").into()); + } + + let logical_plan = child_plan[0]; + + let query_plan = self.prepare_query(logical_plan.clone(), ctx).await?; + + let batch = build_explain_batch( + &query_plan.logical_plan, + &query_plan.physical_plan, + &query_plan.distributed_plan, + &query_plan.distributed_tasks, + )?; + let physical_plan = Arc::new(RecordBatchExec::new(batch)); + + self.send_it( + query_plan.logical_plan, + physical_plan, + query_plan.session_context, + ) + .await + } } diff --git a/src/record_batch_exec.rs b/src/record_batch_exec.rs new file mode 100644 index 0000000..dce2793 --- /dev/null +++ b/src/record_batch_exec.rs @@ -0,0 +1,81 @@ +use std::{fmt::Formatter, sync::Arc}; + +use arrow::array::RecordBatch; +use datafusion::{ + error::Result, + execution::SendableRecordBatchStream, + physical_expr::EquivalenceProperties, + physical_plan::{ + execution_plan::{Boundedness, EmissionType}, + stream::RecordBatchStreamAdapter, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, + }, +}; + +#[derive(Debug)] +pub struct RecordBatchExec { + pub batch: RecordBatch, + pub properties: PlanProperties, +} + +impl RecordBatchExec { + pub fn new(batch: RecordBatch) -> Self { + let properties = PlanProperties::new( + EquivalenceProperties::new(batch.schema()), + Partitioning::UnknownPartitioning(1), + EmissionType::Final, + Boundedness::Bounded, + ); + Self { batch, properties } + } +} + +impl DisplayAs for RecordBatchExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!(f, "RecordBatchExec") + } +} + +impl ExecutionPlan for RecordBatchExec { + fn name(&self) -> &str { + "RecordBatchExec" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn children(&self) -> Vec<&std::sync::Arc> { + vec![] + } + + fn with_new_children( + self: std::sync::Arc, + children: Vec>, + ) -> Result> { + // TODO: generalize this + assert_eq!(children.len(), 1); + Ok(Arc::new(Self::new(self.batch.clone()))) + } + + fn execute( + &self, + _partition: usize, + _context: std::sync::Arc, + ) -> Result { + let batch = self.batch.clone(); + + let output = async move { Ok(batch) }; + + let stream = Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(output), + )) as SendableRecordBatchStream; + + Ok(stream) + } +} diff --git a/src/util.rs b/src/util.rs index d5e9e8c..f652ac3 100644 --- a/src/util.rs +++ b/src/util.rs @@ -2,18 +2,25 @@ use std::{ collections::HashMap, fmt::Display, future::Future, + io::Cursor, pin::Pin, sync::{Arc, OnceLock}, task::{Context, Poll}, time::Duration, }; -use anyhow::Context as anyhowctx; +use anyhow::{anyhow, Context as anyhowctx}; use arrow::{ array::RecordBatch, datatypes::SchemaRef, error::ArrowError, - ipc::{convert::fb_to_schema, root_as_message}, + ipc::{ + convert::fb_to_schema, + reader::StreamReader, + root_as_message, + writer::{IpcWriteOptions, StreamWriter}, + MetadataVersion, + }, }; use arrow_flight::{decode::FlightRecordBatchStream, FlightClient, FlightData, Ticket}; use async_stream::stream; @@ -175,6 +182,28 @@ pub fn flight_data_to_schema(flight_data: &FlightData) -> anyhow::Result Result> { + let schema = batch.schema(); + let buffer: Vec = Vec::new(); + let options = IpcWriteOptions::try_new(8, false, MetadataVersion::V5) + .map_err(|e| internal_datafusion_err!("Cannot create ipcwriteoptions {e}"))?; + + let mut stream_writer = StreamWriter::try_new_with_options(buffer, &schema, options)?; + stream_writer.write(batch)?; + let bytes = stream_writer.into_inner()?; + Ok(bytes) +} + +pub fn ipc_to_batch(bytes: &[u8]) -> Result { + let mut stream_reader = StreamReader::try_new_buffered(Cursor::new(bytes), None)?; + + match stream_reader.next() { + Some(Ok(batch_res)) => Ok(batch_res), + Some(Err(e)) => Err(e.into()), + None => Err(anyhow!("Expected a valid batch").into()), + } +} + /// produce a new SendableRecordBatchStream that will respect the rows /// limit in the batches that it produces. /// diff --git a/src/worker_discovery.rs b/src/worker_discovery.rs new file mode 100644 index 0000000..d4a78a9 --- /dev/null +++ b/src/worker_discovery.rs @@ -0,0 +1,291 @@ +use std::{ + collections::HashMap, + sync::{Arc, OnceLock}, + time::Duration, +}; + +use anyhow::{anyhow, Context}; +use arrow_flight::{Action, FlightClient}; +use futures::{StreamExt, TryStreamExt}; +use k8s_openapi::api::{apps::v1::Deployment, core::v1::Pod}; +use kube::{ + api::{Api, ResourceExt, WatchEvent, WatchParams}, + Client, +}; +use parking_lot::RwLock; +use prost::Message; +use tonic::transport::Channel; + +use crate::{ + logging::{debug, error, trace}, + result::Result, + vocab::Host, +}; + +static WORKER_DISCOVERY: OnceLock> = OnceLock::new(); + +pub fn get_worker_addresses() -> Result> { + match WORKER_DISCOVERY.get_or_init(WorkerDiscovery::new) { + Ok(wd) => { + let worker_addrs = wd.get_addresses(); + debug!( + "Worker addresses found:\n{}", + worker_addrs + .iter() + .map(|host| format!("{host}")) + .collect::>() + .join("\n") + ); + Ok(worker_addrs) + } + Err(e) => Err(anyhow!("Failed to initialize WorkerDiscovery: {}", e).into()), + } +} + +struct WorkerDiscovery { + addresses: Arc>>, +} + +impl WorkerDiscovery { + pub fn new() -> Result { + let wd = WorkerDiscovery { + addresses: Arc::new(RwLock::new(HashMap::new())), + }; + wd.start()?; + Ok(wd) + } + + fn get_addresses(&self) -> Vec { + let guard = self.addresses.read(); + guard.iter().map(|(_ip, host)| host.clone()).collect() + } + + fn start(&self) -> Result<()> { + let worker_addrs_env = std::env::var("DFRAY_WORKER_ADDRESSES"); + let worker_deployment_env = std::env::var("DFRAY_WORKER_DEPLOYMENT"); + let worker_deployment_namespace_env = std::env::var("DFRAY_WORKER_DEPLOYMENT_NAMESPACE"); + + if worker_addrs_env.is_ok() { + let addresses = self.addresses.clone(); + tokio::spawn(async move { + // if the env var is set, use it + set_worker_addresses_from_env(addresses, worker_addrs_env.unwrap().as_str()) + .await + .expect("Could not set worker addresses from env"); + }); + } else if worker_deployment_namespace_env.is_ok() && worker_deployment_env.is_ok() { + let addresses = self.addresses.clone(); + let deployment = worker_deployment_env.unwrap(); + let namespace = worker_deployment_namespace_env.unwrap(); + tokio::spawn(async move { + match watch_deployment_hosts_continuous(addresses, &deployment, &namespace).await { + Ok(_) => {} + Err(e) => error!("Error starting worker watcher: {:?}", e), + } + }); + } else { + // if neither env var is set, return an error + return Err(anyhow!( + "Either DFRAY_WORKER_ADDRESSES or both DFRAY_WORKER_DEPLOYMENT and \ + DFRAY_WORKER_DEPLOYMENT_NAMESPACE must be set" + ) + .into()); + } + Ok(()) + } +} + +async fn set_worker_addresses_from_env( + addresses: Arc>>, + env_str: &str, +) -> Result<()> { + // get addresss from an env var where addresses are split by comans + // and in the form of name/address,name/address + + for addr in env_str.split(',') { + let host = get_worker_host(addr.to_string()) + .await + .context(format!("Failed to get worker host for address: {}", addr))?; + addresses.write().insert(addr.to_owned(), host); + } + Ok(()) +} + +/// Continuously watch for changes to pods in a Kubernetes deployment and call a +/// handler function whenever the list of hosts changes. +/// +/// # Arguments +/// * `deployment_name` - Name of the deployment +/// * `namespace` - Kubernetes namespace where the deployment is located +/// * `handler` - A function to call when the host list changes +/// +/// # Returns +/// This function runs indefinitely until an error occurs +/// +/// # Errors +/// Returns an error if there's an issue connecting to the Kubernetes API +/// or if the deployment or its pods cannot be found +async fn watch_deployment_hosts_continuous( + addresses: Arc>>, + deployment_name: &str, + namespace: &str, +) -> Result<()> { + debug!( + "Starting to watch deployment {} in namespace {}", + deployment_name, namespace + ); + // Initialize the Kubernetes client + let client = Client::try_default() + .await + .context("Failed to create Kubernetes client")?; + + // Access the Deployments API + let deployments: Api = Api::namespaced(client.clone(), namespace); + + // Get the specific deployment + let deployment = deployments + .get(deployment_name) + .await + .context(format!("Failed to get deployment {}", deployment_name))?; + + // Extract the selector labels from the deployment + let selector = deployment + .spec + .as_ref() + .and_then(|spec| spec.selector.match_labels.as_ref()) + .context("Deployment has no selector labels")?; + + // Convert selector to a string format for the label selector + let label_selector = selector + .iter() + .map(|(k, v)| format!("{}={}", k, v)) + .collect::>() + .join(","); + + // Access the Pods API + let pods: Api = Api::namespaced(client, namespace); + + debug!( + "Watching deployment {} in namespace {} with label selector: {}", + deployment_name, namespace, label_selector + ); + + let wp = WatchParams::default().labels(&label_selector); + + // Start watching for pod changes + let mut watcher = pods + .watch(&wp, "0") + .await + .context("could not build watcher")? + .boxed(); + + while let Some(event_result) = watcher + .try_next() + .await + .context("could not get next event from watcher")? + { + match &event_result { + WatchEvent::Added(pod) | WatchEvent::Modified(pod) => { + trace!( + "Pod event: {:?}, added or modified: {:#?}", + event_result, + pod + ); + if let Some(Some(_ip)) = pod.status.as_ref().map(|s| s.pod_ip.as_ref()) { + let (pod_ip, host) = get_worker_info_from_pod(pod).await?; + debug!( + "Pod {} has IP address {}, host {}", + pod.name_any(), + pod_ip, + host + ); + addresses.write().insert(pod_ip, host); + } else { + trace!("Pod {} has no IP address, skipping", pod.name_any()); + } + } + WatchEvent::Deleted(pod) => { + debug!("Pod deleted: {}", pod.name_any()); + if let Some(status) = &pod.status { + if let Some(pod_ip) = &status.pod_ip { + if !pod_ip.is_empty() { + debug!("Removing pod IP: {}", pod_ip); + addresses.write().remove(pod_ip); + } + } + } + } + WatchEvent::Bookmark(_) => {} + WatchEvent::Error(e) => { + eprintln!("Watch error: {}", e); + } + } + } + + Ok(()) +} + +async fn get_worker_host(addr: String) -> Result { + let mut client = Channel::from_shared(format!("http://{addr}")) + .context("Failed to create channel")? + .connect_timeout(Duration::from_secs(2)) + .connect() + .await + .map(FlightClient::new) + .context("Failed to connect to worker")?; + + let action = Action { + r#type: "get_host".to_string(), + body: vec![].into(), + }; + + let mut response = client + .do_action(action) + .await + .context("Failed to send action to worker")?; + + Ok(response + .try_next() + .await + .transpose() + .context("error consuming do_action response")? + .map(Host::decode)? + .context("Failed to decode Host from worker response")?) +} + +async fn get_worker_info_from_pod(pod: &Pod) -> Result<(String, Host)> { + let status = pod.status.as_ref().context("Pod has no status")?; + let pod_ip = status.pod_ip.as_ref().context("Pod has no IP address")?; + + // filter on container name + let port = pod + .spec + .as_ref() + .and_then(|spec| { + spec.containers + .iter() + .find(|c| c.name == "dfray-worker") + .and_then(|c| { + c.ports + .as_ref() + .and_then(|ports| ports.iter().next().map(|p| p.container_port)) + }) + }) + .ok_or_else(|| { + anyhow::anyhow!( + "No could not find container port for container named dfray-worker found in pod {}", + pod.name_any() + ) + })?; + + if pod_ip.is_empty() { + Err(anyhow::anyhow!("Pod {} has no IP address", pod.name_any()).into()) + } else { + let host_str = format!("{}:{}", pod_ip, port); + let host = get_worker_host(host_str.clone()).await.context(format!( + "Failed to get worker host for pod {}", + pod.name_any() + ))?; + Ok((pod_ip.to_owned(), host)) + } +} From 7a2091b94613672e8125fabe1e9ce2efd50e39ef Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 10 Jul 2025 12:42:48 -0400 Subject: [PATCH 5/6] add cargo lock --- Cargo.lock | 5323 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 5323 insertions(+) create mode 100644 Cargo.lock diff --git a/Cargo.lock b/Cargo.lock new file mode 100644 index 0000000..ae0737b --- /dev/null +++ b/Cargo.lock @@ -0,0 +1,5323 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.24.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler2" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" + +[[package]] +name = "ahash" +version = "0.8.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" +dependencies = [ + "cfg-if", + "const-random", + "getrandom 0.3.3", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[package]] +name = "allocator-api2" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" + +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anstream" +version = "0.6.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "301af1932e46185686725e0fad2f8f2aa7da69dd70bf6ecc44d6b703844a3933" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "is_terminal_polyfill", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" + +[[package]] +name = "anstyle-parse" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e7644824f0aa2c7b9384579234ef10eb7efb6a0deb83f9630a49594dd9c15c2" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c8bdeb6047d8983be085bab0ba1472e6dc604e7041dbf6fcd5e71523014fae9" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "403f75924867bb1033c59fbf0797484329750cfbe3c4325cd33127941fabc882" +dependencies = [ + "anstyle", + "once_cell_polyfill", + "windows-sys 0.59.0", +] + +[[package]] +name = "anyhow" +version = "1.0.98" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e16d2d3311acee920a9eb8d33b8cbc1787ce4a264e85f964c2404b969bdcd487" + +[[package]] +name = "arrayref" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76a2e8124351fda1ef8aaaa3bbd7ebbcb486bbcd4225aca0aa0d84bb2db8fecb" + +[[package]] +name = "arrayvec" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" + +[[package]] +name = "arrow" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3f15b4c6b148206ff3a2b35002e08929c2462467b62b9c02036d9c34f9ef994" +dependencies = [ + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-csv", + "arrow-data", + "arrow-ipc", + "arrow-json", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", +] + +[[package]] +name = "arrow-arith" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30feb679425110209ae35c3fbf82404a39a4c0436bb3ec36164d8bffed2a4ce4" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "num", +] + +[[package]] +name = "arrow-array" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70732f04d285d49054a48b72c54f791bb3424abae92d27aafdf776c98af161c8" +dependencies = [ + "ahash", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "chrono-tz", + "half", + "hashbrown 0.15.4", + "num", +] + +[[package]] +name = "arrow-buffer" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "169b1d5d6cb390dd92ce582b06b23815c7953e9dfaaea75556e89d890d19993d" +dependencies = [ + "bytes", + "half", + "num", +] + +[[package]] +name = "arrow-cast" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4f12eccc3e1c05a766cafb31f6a60a46c2f8efec9b74c6e0648766d30686af8" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "atoi", + "base64", + "chrono", + "comfy-table", + "half", + "lexical-core", + "num", + "ryu", +] + +[[package]] +name = "arrow-csv" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "012c9fef3f4a11573b2c74aec53712ff9fdae4a95f4ce452d1bbf088ee00f06b" +dependencies = [ + "arrow-array", + "arrow-cast", + "arrow-schema", + "chrono", + "csv", + "csv-core", + "regex", +] + +[[package]] +name = "arrow-data" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de1ce212d803199684b658fc4ba55fb2d7e87b213de5af415308d2fee3619c2" +dependencies = [ + "arrow-buffer", + "arrow-schema", + "half", + "num", +] + +[[package]] +name = "arrow-flight" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cb3e1d2b441e6d1d5988e3f7c4523c9466b18ef77d7c525d92d36d4cad49fbe" +dependencies = [ + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", + "base64", + "bytes", + "futures", + "once_cell", + "paste", + "prost", + "prost-types", + "tonic", +] + +[[package]] +name = "arrow-ipc" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d9ea5967e8b2af39aff5d9de2197df16e305f47f404781d3230b2dc672da5d92" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "flatbuffers", + "lz4_flex", +] + +[[package]] +name = "arrow-json" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5709d974c4ea5be96d900c01576c7c0b99705f4a3eec343648cb1ca863988a9c" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "half", + "indexmap 2.10.0", + "lexical-core", + "memchr", + "num", + "serde", + "serde_json", + "simdutf8", +] + +[[package]] +name = "arrow-ord" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6506e3a059e3be23023f587f79c82ef0bcf6d293587e3272d20f2d30b969b5a7" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", +] + +[[package]] +name = "arrow-row" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52bf7393166beaf79b4bed9bfdf19e97472af32ce5b6b48169d321518a08cae2" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "half", +] + +[[package]] +name = "arrow-schema" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af7686986a3bf2254c9fb130c623cdcb2f8e1f15763e7c71c310f0834da3d292" +dependencies = [ + "serde", + "serde_json", +] + +[[package]] +name = "arrow-select" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd2b45757d6a2373faa3352d02ff5b54b098f5e21dccebc45a21806bc34501e5" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "num", +] + +[[package]] +name = "arrow-string" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0377d532850babb4d927a06294314b316e23311503ed580ec6ce6a0158f49d40" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "memchr", + "num", + "regex", + "regex-syntax 0.8.5", +] + +[[package]] +name = "async-broadcast" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "435a87a52755b8f27fcf321ac4f04b2802e337c8c4872923137471ec39c37532" +dependencies = [ + "event-listener", + "event-listener-strategy", + "futures-core", + "pin-project-lite", +] + +[[package]] +name = "async-compression" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" +dependencies = [ + "bzip2", + "flate2", + "futures-core", + "memchr", + "pin-project-lite", + "tokio", + "xz2", + "zstd", + "zstd-safe", +] + +[[package]] +name = "async-stream" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b5a71a6f37880a80d1d7f19efd781e4b5de42c88f0722cc13bcb6cc2cfe8476" +dependencies = [ + "async-stream-impl", + "futures-core", + "pin-project-lite", +] + +[[package]] +name = "async-stream-impl" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "async-trait" +version = "0.1.88" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e539d3fca749fcee5236ab05e93a52867dd549cc157c8cb7f99595f3cedffdb5" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "atoi" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" +dependencies = [ + "num-traits", +] + +[[package]] +name = "atomic-waker" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" + +[[package]] +name = "autocfg" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" + +[[package]] +name = "aws-lc-rs" +version = "1.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fcc8f365936c834db5514fc45aee5b1202d677e6b40e48468aaaa8183ca8c7" +dependencies = [ + "aws-lc-sys", + "zeroize", +] + +[[package]] +name = "aws-lc-sys" +version = "0.29.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "61b1d86e7705efe1be1b569bab41d4fa1e14e220b60a160f78de2db687add079" +dependencies = [ + "bindgen", + "cc", + "cmake", + "dunce", + "fs_extra", +] + +[[package]] +name = "axum" +version = "0.7.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "edca88bc138befd0323b20752846e6587272d3b03b0343c8ea28a6f819e6e71f" +dependencies = [ + "async-trait", + "axum-core", + "bytes", + "futures-util", + "http", + "http-body", + "http-body-util", + "itoa", + "matchit", + "memchr", + "mime", + "percent-encoding", + "pin-project-lite", + "rustversion", + "serde", + "sync_wrapper", + "tower 0.5.2", + "tower-layer", + "tower-service", +] + +[[package]] +name = "axum-core" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09f2bd6146b97ae3359fa0cc6d6b376d9539582c7b4220f041a33ec24c226199" +dependencies = [ + "async-trait", + "bytes", + "futures-util", + "http", + "http-body", + "http-body-util", + "mime", + "pin-project-lite", + "rustversion", + "sync_wrapper", + "tower-layer", + "tower-service", +] + +[[package]] +name = "backon" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "302eaff5357a264a2c42f127ecb8bac761cf99749fc3dc95677e2743991f99e7" +dependencies = [ + "fastrand", + "gloo-timers", + "tokio", +] + +[[package]] +name = "backtrace" +version = "0.3.75" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" +dependencies = [ + "addr2line", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", + "windows-targets 0.52.6", +] + +[[package]] +name = "base64" +version = "0.22.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" + +[[package]] +name = "bigdecimal" +version = "0.4.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a22f228ab7a1b23027ccc6c350b72868017af7ea8356fbdf19f8d991c690013" +dependencies = [ + "autocfg", + "libm", + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "bindgen" +version = "0.69.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" +dependencies = [ + "bitflags", + "cexpr", + "clang-sys", + "itertools 0.12.1", + "lazy_static", + "lazycell", + "log", + "prettyplease", + "proc-macro2", + "quote", + "regex", + "rustc-hash 1.1.0", + "shlex", + "syn", + "which", +] + +[[package]] +name = "bitflags" +version = "2.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b8e56985ec62d17e9c1001dc89c88ecd7dc08e47eba5ec7c29c7b5eeecde967" + +[[package]] +name = "blake2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" +dependencies = [ + "digest", +] + +[[package]] +name = "blake3" +version = "1.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3888aaa89e4b2a40fca9848e400f6a658a5a3978de7be858e209cafa8be9a4a0" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "brotli" +version = "8.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9991eea70ea4f293524138648e41ee89b0b2b12ddef3b255effa43c8056e0e0d" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + +[[package]] +name = "brotli-decompressor" +version = "5.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "874bb8112abecc98cbd6d81ea4fa7e94fb9449648c93cc89aa40c81c24d7de03" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46c5e41b57b8bba42a04676d81cb89e9ee8e859a1a66f80a5a72e1cb76b34d43" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a" + +[[package]] +name = "bzip2" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49ecfb22d906f800d4fe833b6282cf4dc1c298f5057ca0b5445e5c209735ca47" +dependencies = [ + "bzip2-sys", +] + +[[package]] +name = "bzip2-sys" +version = "0.1.13+1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225bff33b2141874fe80d71e07d6eec4f85c5c216453dd96388240f96e1acc14" +dependencies = [ + "cc", + "pkg-config", +] + +[[package]] +name = "cc" +version = "1.2.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d487aa071b5f64da6f19a3e848e3578944b726ee5a4854b82172f02aa876bfdc" +dependencies = [ + "jobserver", + "libc", + "shlex", +] + +[[package]] +name = "cexpr" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fac387a98bb7c37292057cffc56d62ecb629900026402633ae9160df93a8766" +dependencies = [ + "nom", +] + +[[package]] +name = "cfg-if" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9555578bc9e57714c812a1f84e4fc5b4d21fcb063490c624de019f7464c91268" + +[[package]] +name = "cfg_aliases" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" + +[[package]] +name = "chrono" +version = "0.4.41" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c469d952047f47f91b68d1cba3f10d63c11d73e4636f24f08daf0278abf01c4d" +dependencies = [ + "android-tzdata", + "iana-time-zone", + "num-traits", + "serde", + "windows-link", +] + +[[package]] +name = "chrono-tz" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "efdce149c370f133a071ca8ef6ea340b7b88748ab0810097a9e2976eaa34b4f3" +dependencies = [ + "chrono", + "chrono-tz-build", + "phf", +] + +[[package]] +name = "chrono-tz-build" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f10f8c9340e31fc120ff885fcdb54a0b48e474bbd77cab557f0c30a3e569402" +dependencies = [ + "parse-zoneinfo", + "phf_codegen", +] + +[[package]] +name = "clang-sys" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" +dependencies = [ + "glob", + "libc", + "libloading", +] + +[[package]] +name = "clap" +version = "4.5.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40b6887a1d8685cebccf115538db5c0efe625ccac9696ad45c409d96566e910f" +dependencies = [ + "clap_builder", + "clap_derive", +] + +[[package]] +name = "clap_builder" +version = "4.5.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0c66c08ce9f0c698cbce5c0279d0bb6ac936d8674174fe48f736533b964f59e" +dependencies = [ + "anstream", + "anstyle", + "clap_lex", + "strsim", +] + +[[package]] +name = "clap_derive" +version = "4.5.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2c7947ae4cc3d851207c1adb5b5e260ff0cca11446b1d6d1423788e442257ce" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "clap_lex" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b94f61472cee1439c0b966b47e3aca9ae07e45d070759512cd390ea2bebc6675" + +[[package]] +name = "cmake" +version = "0.1.54" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7caa3f9de89ddbe2c607f4101924c5abec803763ae9534e4f4d7d8f84aa81f0" +dependencies = [ + "cc", +] + +[[package]] +name = "colorchoice" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b05b61dc5112cbb17e4b6cd61790d9845d13888356391624cbe7e41efeac1e75" + +[[package]] +name = "comfy-table" +version = "7.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a65ebfec4fb190b6f90e944a817d60499ee0744e582530e2c9900a22e591d9a" +dependencies = [ + "unicode-segmentation", + "unicode-width", +] + +[[package]] +name = "concurrent-queue" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ca0197aee26d1ae37445ee532fefce43251d24cc7c166799f4d46817f1d3973" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "const-random" +version = "0.1.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom 0.2.16", + "once_cell", + "tiny-keccak", +] + +[[package]] +name = "constant_time_eq" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" + +[[package]] +name = "core-foundation" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91e195e091a93c46f7102ec7818a2aa394e1e1771c3ab4825963fa03e45afb8f" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "core-foundation" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2a6cd9ae233e7f62ba4e9353e81a88df7fc8a5987b8d445b4d90c879bd156f6" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "core-foundation-sys" +version = "0.8.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" + +[[package]] +name = "cpufeatures" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59ed5838eebb26a2bb2e58f6d5b5316989ae9d08bab10e0e6d103e656d1b0280" +dependencies = [ + "libc", +] + +[[package]] +name = "crc32fast" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" + +[[package]] +name = "crunchy" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "460fbee9c2c2f33933d720630a6a0bac33ba7053db5344fac858d4b8952d77d5" + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d02f3b0da4c6504f86e9cd789d8dbafab48c2321be74e9987593de5a894d93d" +dependencies = [ + "memchr", +] + +[[package]] +name = "darling" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc7f46116c46ff9ab3eb1597a45688b6715c6e628b5c133e288e709a29bcb4ee" +dependencies = [ + "darling_core", + "darling_macro", +] + +[[package]] +name = "darling_core" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d00b9596d185e565c2207a0b01f8bd1a135483d02d9b7b0a54b11da8d53412e" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn", +] + +[[package]] +name = "darling_macro" +version = "0.20.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" +dependencies = [ + "darling_core", + "quote", + "syn", +] + +[[package]] +name = "dashmap" +version = "6.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5041cc499144891f3790297212f32a74fb938e5136a14943f338ef9e0ae276cf" +dependencies = [ + "cfg-if", + "crossbeam-utils", + "hashbrown 0.14.5", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "datafusion" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc6cb8c2c81eada072059983657d6c9caf3fddefc43b4a65551d243253254a96" +dependencies = [ + "arrow", + "arrow-ipc", + "arrow-schema", + "async-trait", + "bytes", + "bzip2", + "chrono", + "datafusion-catalog", + "datafusion-catalog-listing", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-datasource-csv", + "datafusion-datasource-json", + "datafusion-datasource-parquet", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-nested", + "datafusion-functions-table", + "datafusion-functions-window", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-optimizer", + "datafusion-physical-plan", + "datafusion-session", + "datafusion-sql", + "flate2", + "futures", + "itertools 0.14.0", + "log", + "object_store", + "parking_lot", + "parquet", + "rand 0.9.1", + "regex", + "sqlparser", + "tempfile", + "tokio", + "url", + "uuid", + "xz2", + "zstd", +] + +[[package]] +name = "datafusion-catalog" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7be8d1b627843af62e447396db08fe1372d882c0eb8d0ea655fd1fbc33120ee" +dependencies = [ + "arrow", + "async-trait", + "dashmap", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-session", + "datafusion-sql", + "futures", + "itertools 0.14.0", + "log", + "object_store", + "parking_lot", + "tokio", +] + +[[package]] +name = "datafusion-catalog-listing" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38ab16c5ae43f65ee525fc493ceffbc41f40dee38b01f643dfcfc12959e92038" +dependencies = [ + "arrow", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "log", + "object_store", + "tokio", +] + +[[package]] +name = "datafusion-common" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3d56b2ac9f476b93ca82e4ef5fb00769c8a3f248d12b4965af7e27635fa7e12" +dependencies = [ + "ahash", + "arrow", + "arrow-ipc", + "base64", + "half", + "hashbrown 0.14.5", + "indexmap 2.10.0", + "libc", + "log", + "object_store", + "parquet", + "paste", + "recursive", + "sqlparser", + "tokio", + "web-time", +] + +[[package]] +name = "datafusion-common-runtime" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16015071202d6133bc84d72756176467e3e46029f3ce9ad2cb788f9b1ff139b2" +dependencies = [ + "futures", + "log", + "tokio", +] + +[[package]] +name = "datafusion-datasource" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b77523c95c89d2a7eb99df14ed31390e04ab29b43ff793e562bdc1716b07e17b" +dependencies = [ + "arrow", + "async-compression", + "async-trait", + "bytes", + "bzip2", + "chrono", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "flate2", + "futures", + "glob", + "itertools 0.14.0", + "log", + "object_store", + "parquet", + "rand 0.9.1", + "tempfile", + "tokio", + "tokio-util", + "url", + "xz2", + "zstd", +] + +[[package]] +name = "datafusion-datasource-csv" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40d25c5e2c0ebe8434beeea997b8e88d55b3ccc0d19344293f2373f65bc524fc" +dependencies = [ + "arrow", + "async-trait", + "bytes", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "object_store", + "regex", + "tokio", +] + +[[package]] +name = "datafusion-datasource-json" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3dc6959e1155741ab35369e1dc7673ba30fc45ed568fad34c01b7cb1daeb4d4c" +dependencies = [ + "arrow", + "async-trait", + "bytes", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "object_store", + "serde_json", + "tokio", +] + +[[package]] +name = "datafusion-datasource-parquet" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7a6afdfe358d70f4237f60eaef26ae5a1ce7cb2c469d02d5fc6c7fd5d84e58b" +dependencies = [ + "arrow", + "async-trait", + "bytes", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-optimizer", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "itertools 0.14.0", + "log", + "object_store", + "parking_lot", + "parquet", + "rand 0.9.1", + "tokio", +] + +[[package]] +name = "datafusion-doc" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9bcd8a3e3e3d02ea642541be23d44376b5d5c37c2938cce39b3873cdf7186eea" + +[[package]] +name = "datafusion-execution" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "670da1d45d045eee4c2319b8c7ea57b26cf48ab77b630aaa50b779e406da476a" +dependencies = [ + "arrow", + "dashmap", + "datafusion-common", + "datafusion-expr", + "futures", + "log", + "object_store", + "parking_lot", + "rand 0.9.1", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b3a577f64bdb7e2cc4043cd97f8901d8c504711fde2dbcb0887645b00d7c660b" +dependencies = [ + "arrow", + "chrono", + "datafusion-common", + "datafusion-doc", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", + "datafusion-physical-expr-common", + "indexmap 2.10.0", + "paste", + "recursive", + "serde_json", + "sqlparser", +] + +[[package]] +name = "datafusion-expr-common" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51b7916806ace3e9f41884f230f7f38ebf0e955dfbd88266da1826f29a0b9a6a" +dependencies = [ + "arrow", + "datafusion-common", + "indexmap 2.10.0", + "itertools 0.14.0", + "paste", +] + +[[package]] +name = "datafusion-functions" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fb31c9dc73d3e0c365063f91139dc273308f8a8e124adda9898db8085d68357" +dependencies = [ + "arrow", + "arrow-buffer", + "base64", + "blake2", + "blake3", + "chrono", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", + "hex", + "itertools 0.14.0", + "log", + "md-5", + "rand 0.9.1", + "regex", + "sha2", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-functions-aggregate" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebb72c6940697eaaba9bd1f746a697a07819de952b817e3fb841fb75331ad5d4" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "half", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-aggregate-common" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7fdc54656659e5ecd49bf341061f4156ab230052611f4f3609612a0da259696" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr-common", + "datafusion-physical-expr-common", +] + +[[package]] +name = "datafusion-functions-nested" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fad94598e3374938ca43bca6b675febe557e7a14eb627d617db427d70d65118b" +dependencies = [ + "arrow", + "arrow-ord", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-macros", + "datafusion-physical-expr-common", + "itertools 0.14.0", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-table" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de2fc6c2946da5cab8364fb28b5cac3115f0f3a87960b235ed031c3f7e2e639b" +dependencies = [ + "arrow", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-plan", + "parking_lot", + "paste", +] + +[[package]] +name = "datafusion-functions-window" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3e5746548a8544870a119f556543adcd88fe0ba6b93723fe78ad0439e0fbb8b4" +dependencies = [ + "arrow", + "datafusion-common", + "datafusion-doc", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-window-common" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dcbe9404382cda257c434f22e13577bee7047031dfdb6216dd5e841b9465e6fe" +dependencies = [ + "datafusion-common", + "datafusion-physical-expr-common", +] + +[[package]] +name = "datafusion-macros" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8dce50e3b637dab0d25d04d2fe79dfdca2b257eabd76790bffd22c7f90d700c8" +dependencies = [ + "datafusion-expr", + "quote", + "syn", +] + +[[package]] +name = "datafusion-optimizer" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03cfaacf06445dc3bbc1e901242d2a44f2cae99a744f49f3fefddcee46240058" +dependencies = [ + "arrow", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", + "indexmap 2.10.0", + "itertools 0.14.0", + "log", + "recursive", + "regex", + "regex-syntax 0.8.5", +] + +[[package]] +name = "datafusion-physical-expr" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1908034a89d7b2630898e06863583ae4c00a0dd310c1589ca284195ee3f7f8a6" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", + "half", + "hashbrown 0.14.5", + "indexmap 2.10.0", + "itertools 0.14.0", + "log", + "paste", + "petgraph 0.8.2", +] + +[[package]] +name = "datafusion-physical-expr-common" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "47b7a12dd59ea07614b67dbb01d85254fbd93df45bcffa63495e11d3bdf847df" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr-common", + "hashbrown 0.14.5", + "itertools 0.14.0", +] + +[[package]] +name = "datafusion-physical-optimizer" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4371cc4ad33978cc2a8be93bd54a232d3f2857b50401a14631c0705f3f910aae" +dependencies = [ + "arrow", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "itertools 0.14.0", + "log", + "recursive", +] + +[[package]] +name = "datafusion-physical-plan" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc47bc33025757a5c11f2cd094c5b6b5ed87f46fa33c023e6fdfa25fcbfade23" +dependencies = [ + "ahash", + "arrow", + "arrow-ord", + "arrow-schema", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "futures", + "half", + "hashbrown 0.14.5", + "indexmap 2.10.0", + "itertools 0.14.0", + "log", + "parking_lot", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "datafusion-proto" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8f5d9acd7d96e3bf2a7bb04818373cab6e51de0356e3694b94905fee7b4e8b6" +dependencies = [ + "arrow", + "chrono", + "datafusion", + "datafusion-common", + "datafusion-expr", + "datafusion-proto-common", + "object_store", + "prost", +] + +[[package]] +name = "datafusion-proto-common" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09ecb5ec152c4353b60f7a5635489834391f7a291d2b39a4820cd469e318b78e" +dependencies = [ + "arrow", + "datafusion-common", + "prost", +] + +[[package]] +name = "datafusion-session" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7485da32283985d6b45bd7d13a65169dcbe8c869e25d01b2cfbc425254b4b49" +dependencies = [ + "arrow", + "async-trait", + "dashmap", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-sql", + "futures", + "itertools 0.14.0", + "log", + "object_store", + "parking_lot", + "tokio", +] + +[[package]] +name = "datafusion-sql" +version = "48.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a466b15632befddfeac68c125f0260f569ff315c6831538cbb40db754134e0df" +dependencies = [ + "arrow", + "bigdecimal", + "datafusion-common", + "datafusion-expr", + "indexmap 2.10.0", + "log", + "recursive", + "regex", + "sqlparser", +] + +[[package]] +name = "derive_more" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "093242cf7570c207c83073cf82f79706fe7b8317e98620a47d5be7c3d8497678" +dependencies = [ + "derive_more-impl", +] + +[[package]] +name = "derive_more-impl" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bda628edc44c4bb645fbe0f758797143e4e07926f7ebf4e9bdfbd3d2ce621df3" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + +[[package]] +name = "displaydoc" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "distributed-datafusion" +version = "0.1.0" +dependencies = [ + "anyhow", + "arrow", + "arrow-flight", + "async-stream", + "bytes", + "clap", + "datafusion", + "datafusion-proto", + "env_logger", + "futures", + "itertools 0.14.0", + "k8s-openapi", + "kube", + "log", + "object_store", + "parking_lot", + "prost", + "prost-types", + "rand 0.8.5", + "rustc_version", + "rustls", + "tempfile", + "test-log", + "thiserror 1.0.69", + "tikv-jemalloc-ctl", + "tikv-jemalloc-sys", + "tikv-jemallocator", + "tokio", + "tokio-stream", + "tonic", + "tonic-build", + "url", + "uuid", +] + +[[package]] +name = "dunce" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" + +[[package]] +name = "dyn-clone" +version = "1.0.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c7a8fb8a9fbf66c1f703fe16184d10ca0ee9d23be5b4436400408ba54a95005" + +[[package]] +name = "educe" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d7bc049e1bd8cdeb31b68bbd586a9464ecf9f3944af3958a7a9d0f8b9799417" +dependencies = [ + "enum-ordinalize", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "either" +version = "1.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48c757948c5ede0e46177b7add2e67155f70e33c07fea8284df6576da70b3719" + +[[package]] +name = "enum-ordinalize" +version = "4.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fea0dcfa4e54eeb516fe454635a95753ddd39acda650ce703031c6973e315dd5" +dependencies = [ + "enum-ordinalize-derive", +] + +[[package]] +name = "enum-ordinalize-derive" +version = "4.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "env_filter" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "186e05a59d4c50738528153b83b0b0194d3a29507dfec16eccd4b342903397d0" +dependencies = [ + "log", + "regex", +] + +[[package]] +name = "env_logger" +version = "0.11.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13c863f0904021b108aa8b2f55046443e6b1ebde8fd4a15c399893aae4fa069f" +dependencies = [ + "anstream", + "anstyle", + "env_filter", + "jiff", + "log", +] + +[[package]] +name = "equivalent" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" + +[[package]] +name = "errno" +version = "0.3.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "778e2ac28f6c47af28e4907f13ffd1e1ddbd400980a9abd7c8df189bf578a5ad" +dependencies = [ + "libc", + "windows-sys 0.60.2", +] + +[[package]] +name = "event-listener" +version = "5.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3492acde4c3fc54c845eaab3eed8bd00c7a7d881f78bfc801e43a93dec1331ae" +dependencies = [ + "concurrent-queue", + "parking", + "pin-project-lite", +] + +[[package]] +name = "event-listener-strategy" +version = "0.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8be9f3dfaaffdae2972880079a491a1a8bb7cbed0b8dd7a347f668b4150a3b93" +dependencies = [ + "event-listener", + "pin-project-lite", +] + +[[package]] +name = "fastrand" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" + +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + +[[package]] +name = "fixedbitset" +version = "0.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" + +[[package]] +name = "flatbuffers" +version = "25.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" +dependencies = [ + "bitflags", + "rustc_version", +] + +[[package]] +name = "flate2" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a3d7db9596fecd151c5f638c0ee5d5bd487b6e0ea232e5dc96d5250f6f94b1d" +dependencies = [ + "crc32fast", + "libz-rs-sys", + "miniz_oxide", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "foldhash" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" + +[[package]] +name = "form_urlencoded" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "fs_extra" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" + +[[package]] +name = "futures" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" + +[[package]] +name = "futures-executor" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" + +[[package]] +name = "futures-macro" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "futures-sink" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" + +[[package]] +name = "futures-task" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" + +[[package]] +name = "futures-util" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + +[[package]] +name = "getrandom" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "335ff9f135e4384c8150d6f27c6daed433577f86b4750418338c01a1a2528592" +dependencies = [ + "cfg-if", + "js-sys", + "libc", + "wasi 0.11.1+wasi-snapshot-preview1", + "wasm-bindgen", +] + +[[package]] +name = "getrandom" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26145e563e54f2cadc477553f1ec5ee650b00862f0a58bcd12cbdc5f0ea2d2f4" +dependencies = [ + "cfg-if", + "js-sys", + "libc", + "r-efi", + "wasi 0.14.2+wasi-0.2.4", + "wasm-bindgen", +] + +[[package]] +name = "gimli" +version = "0.31.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" + +[[package]] +name = "glob" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" + +[[package]] +name = "gloo-timers" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbb143cf96099802033e0d4f4963b19fd2e0b728bcf076cd9cf7f6634f092994" +dependencies = [ + "futures-channel", + "futures-core", + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "h2" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a9421a676d1b147b16b82c9225157dc629087ef8ec4d5e2960f9437a90dac0a5" +dependencies = [ + "atomic-waker", + "bytes", + "fnv", + "futures-core", + "futures-sink", + "http", + "indexmap 2.10.0", + "slab", + "tokio", + "tokio-util", + "tracing", +] + +[[package]] +name = "half" +version = "2.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "459196ed295495a68f7d7fe1d84f6c4b7ff0e21fe3017b2f283c6fac3ad803c9" +dependencies = [ + "cfg-if", + "crunchy", + "num-traits", +] + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" + +[[package]] +name = "hashbrown" +version = "0.14.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" +dependencies = [ + "ahash", + "allocator-api2", +] + +[[package]] +name = "hashbrown" +version = "0.15.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5971ac85611da7067dbfcabef3c70ebb5606018acd9e2a3903a0da507521e0d5" +dependencies = [ + "allocator-api2", + "equivalent", + "foldhash", +] + +[[package]] +name = "headers" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b3314d5adb5d94bcdf56771f2e50dbbc80bb4bdf88967526706205ac9eff24eb" +dependencies = [ + "base64", + "bytes", + "headers-core", + "http", + "httpdate", + "mime", + "sha1", +] + +[[package]] +name = "headers-core" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54b4a22553d4242c49fddb9ba998a99962b5cc6f22cb5a3482bec22522403ce4" +dependencies = [ + "http", +] + +[[package]] +name = "heck" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" + +[[package]] +name = "home" +version = "0.5.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589533453244b0995c858700322199b2becb13b627df2851f64a2775d024abcf" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "hostname" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a56f203cd1c76362b69e3863fd987520ac36cf70a8c92627449b2f64a8cf7d65" +dependencies = [ + "cfg-if", + "libc", + "windows-link", +] + +[[package]] +name = "http" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4a85d31aea989eead29a3aaf9e1115a180df8282431156e533de47660892565" +dependencies = [ + "bytes", + "fnv", + "itoa", +] + +[[package]] +name = "http-body" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1efedce1fb8e6913f23e0c92de8e62cd5b772a67e7b3946df930a62566c93184" +dependencies = [ + "bytes", + "http", +] + +[[package]] +name = "http-body-util" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b021d93e26becf5dc7e1b75b1bed1fd93124b374ceb73f43d4d4eafec896a64a" +dependencies = [ + "bytes", + "futures-core", + "http", + "http-body", + "pin-project-lite", +] + +[[package]] +name = "httparse" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6dbf3de79e51f3d586ab4cb9d5c3e2c14aa28ed23d180cf89b4df0454a69cc87" + +[[package]] +name = "httpdate" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" + +[[package]] +name = "humantime" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b112acc8b3adf4b107a8ec20977da0273a8c386765a3ec0229bd500a1443f9f" + +[[package]] +name = "hyper" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc2b571658e38e0c01b1fdca3bbbe93c00d3d71693ff2770043f8c29bc7d6f80" +dependencies = [ + "bytes", + "futures-channel", + "futures-util", + "h2", + "http", + "http-body", + "httparse", + "httpdate", + "itoa", + "pin-project-lite", + "smallvec", + "tokio", + "want", +] + +[[package]] +name = "hyper-http-proxy" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ad4b0a1e37510028bc4ba81d0e38d239c39671b0f0ce9e02dfa93a8133f7c08" +dependencies = [ + "bytes", + "futures-util", + "headers", + "http", + "hyper", + "hyper-rustls", + "hyper-util", + "pin-project-lite", + "rustls-native-certs 0.7.3", + "tokio", + "tokio-rustls", + "tower-service", +] + +[[package]] +name = "hyper-rustls" +version = "0.27.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3c93eb611681b207e1fe55d5a71ecf91572ec8a6705cdb6857f7d8d5242cf58" +dependencies = [ + "http", + "hyper", + "hyper-util", + "log", + "rustls", + "rustls-native-certs 0.8.1", + "rustls-pki-types", + "tokio", + "tokio-rustls", + "tower-service", +] + +[[package]] +name = "hyper-timeout" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b90d566bffbce6a75bd8b09a05aa8c2cb1fabb6cb348f8840c9e4c90a0d83b0" +dependencies = [ + "hyper", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", +] + +[[package]] +name = "hyper-util" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc2fdfdbff08affe55bb779f33b053aa1fe5dd5b54c257343c17edfa55711bdb" +dependencies = [ + "base64", + "bytes", + "futures-channel", + "futures-core", + "futures-util", + "http", + "http-body", + "hyper", + "ipnet", + "libc", + "percent-encoding", + "pin-project-lite", + "socket2", + "tokio", + "tower-service", + "tracing", +] + +[[package]] +name = "iana-time-zone" +version = "0.1.63" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0c919e5debc312ad217002b8048a17b7d83f80703865bbfcfebb0458b0b27d8" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "log", + "wasm-bindgen", + "windows-core", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[package]] +name = "icu_collections" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" +dependencies = [ + "displaydoc", + "potential_utf", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locale_core" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +dependencies = [ + "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_normalizer" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_normalizer_data", + "icu_properties", + "icu_provider", + "smallvec", + "zerovec", +] + +[[package]] +name = "icu_normalizer_data" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" + +[[package]] +name = "icu_properties" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_locale_core", + "icu_properties_data", + "icu_provider", + "potential_utf", + "zerotrie", + "zerovec", +] + +[[package]] +name = "icu_properties_data" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" + +[[package]] +name = "icu_provider" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +dependencies = [ + "displaydoc", + "icu_locale_core", + "stable_deref_trait", + "tinystr", + "writeable", + "yoke", + "zerofrom", + "zerotrie", + "zerovec", +] + +[[package]] +name = "ident_case" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" + +[[package]] +name = "idna" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "686f825264d630750a544639377bae737628043f20d38bbc029e8f29ea968a7e" +dependencies = [ + "idna_adapter", + "smallvec", + "utf8_iter", +] + +[[package]] +name = "idna_adapter" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3acae9609540aa318d1bc588455225fb2085b9ed0c4f6bd0d9d5bcd86f1a0344" +dependencies = [ + "icu_normalizer", + "icu_properties", +] + +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", +] + +[[package]] +name = "indexmap" +version = "2.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe4cd85333e22411419a0bcae1297d25e58c9443848b11dc6a86fefe8c78a661" +dependencies = [ + "equivalent", + "hashbrown 0.15.4", +] + +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "ipnet" +version = "2.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" + +[[package]] +name = "iri-string" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbc5ebe9c3a1a7a5127f920a418f7585e9e758e911d0466ed004f393b0e380b2" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "is_terminal_polyfill" +version = "1.70.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" + +[[package]] +name = "itertools" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b192c782037fadd9cfa75548310488aabdbf3d2da73885b31bd0abd03351285" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a5f13b858c8d314ee3e8f639011f7ccefe71f97f96e50151fb991f267928e2c" + +[[package]] +name = "jiff" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be1f93b8b1eb69c77f24bbb0afdf66f54b632ee39af40ca21c4365a1d7347e49" +dependencies = [ + "jiff-static", + "log", + "portable-atomic", + "portable-atomic-util", + "serde", +] + +[[package]] +name = "jiff-static" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03343451ff899767262ec32146f6d559dd759fdadf42ff0e227c7c48f72594b4" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "jobserver" +version = "0.1.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38f262f097c174adebe41eb73d66ae9c06b2844fb0da69969647bbddd9b0538a" +dependencies = [ + "getrandom 0.3.3", + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.77" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1cfaf33c695fc6e08064efbc1f72ec937429614f25eef83af942d0e227c3a28f" +dependencies = [ + "once_cell", + "wasm-bindgen", +] + +[[package]] +name = "json-patch" +version = "4.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "159294d661a039f7644cea7e4d844e6b25aaf71c1ffe9d73a96d768c24b0faf4" +dependencies = [ + "jsonptr", + "serde", + "serde_json", + "thiserror 1.0.69", +] + +[[package]] +name = "jsonpath-rust" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c00ae348f9f8fd2d09f82a98ca381c60df9e0820d8d79fce43e649b4dc3128b" +dependencies = [ + "pest", + "pest_derive", + "regex", + "serde_json", + "thiserror 2.0.12", +] + +[[package]] +name = "jsonptr" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5a3cc660ba5d72bce0b3bb295bf20847ccbb40fd423f3f05b61273672e561fe" +dependencies = [ + "serde", + "serde_json", +] + +[[package]] +name = "k8s-openapi" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa60a41b57ae1a0a071af77dbcf89fc9819cfe66edaf2beeb204c34459dcf0b2" +dependencies = [ + "base64", + "chrono", + "serde", + "serde_json", +] + +[[package]] +name = "kube" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "778f98664beaf4c3c11372721e14310d1ae00f5e2d9aabcf8906c881aa4e9f51" +dependencies = [ + "k8s-openapi", + "kube-client", + "kube-core", + "kube-derive", + "kube-runtime", +] + +[[package]] +name = "kube-client" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7cb276b85b6e94ded00ac8ea2c68fcf4697ea0553cb25fddc35d4a0ab718db8d" +dependencies = [ + "base64", + "bytes", + "chrono", + "either", + "futures", + "home", + "http", + "http-body", + "http-body-util", + "hyper", + "hyper-http-proxy", + "hyper-rustls", + "hyper-timeout", + "hyper-util", + "jsonpath-rust", + "k8s-openapi", + "kube-core", + "pem", + "rustls", + "secrecy", + "serde", + "serde_json", + "serde_yaml", + "thiserror 2.0.12", + "tokio", + "tokio-util", + "tower 0.5.2", + "tower-http", + "tracing", +] + +[[package]] +name = "kube-core" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3c56ff45deb0031f2a476017eed60c06872251f271b8387ad8020b8fef60960" +dependencies = [ + "chrono", + "derive_more", + "form_urlencoded", + "http", + "json-patch", + "k8s-openapi", + "schemars", + "serde", + "serde-value", + "serde_json", + "thiserror 2.0.12", +] + +[[package]] +name = "kube-derive" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "079fc8c1c397538628309cfdee20696ebdcc26745f9fb17f89b78782205bd995" +dependencies = [ + "darling", + "proc-macro2", + "quote", + "serde", + "serde_json", + "syn", +] + +[[package]] +name = "kube-runtime" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2f1326e946fadf6248febdf8a1c001809c3899ccf48cb9768cbc536b741040dc" +dependencies = [ + "ahash", + "async-broadcast", + "async-stream", + "backon", + "educe", + "futures", + "hashbrown 0.15.4", + "hostname", + "json-patch", + "k8s-openapi", + "kube-client", + "parking_lot", + "pin-project", + "serde", + "serde_json", + "thiserror 2.0.12", + "tokio", + "tokio-util", + "tracing", +] + +[[package]] +name = "lazy_static" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" + +[[package]] +name = "lazycell" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" + +[[package]] +name = "lexical-core" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b765c31809609075565a70b4b71402281283aeda7ecaf4818ac14a7b2ade8958" +dependencies = [ + "lexical-parse-float", + "lexical-parse-integer", + "lexical-util", + "lexical-write-float", + "lexical-write-integer", +] + +[[package]] +name = "lexical-parse-float" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de6f9cb01fb0b08060209a057c048fcbab8717b4c1ecd2eac66ebfe39a65b0f2" +dependencies = [ + "lexical-parse-integer", + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-parse-integer" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72207aae22fc0a121ba7b6d479e42cbfea549af1479c3f3a4f12c70dd66df12e" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-util" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a82e24bf537fd24c177ffbbdc6ebcc8d54732c35b50a3f28cc3f4e4c949a0b3" +dependencies = [ + "static_assertions", +] + +[[package]] +name = "lexical-write-float" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c5afc668a27f460fb45a81a757b6bf2f43c2d7e30cb5a2dcd3abf294c78d62bd" +dependencies = [ + "lexical-util", + "lexical-write-integer", + "static_assertions", +] + +[[package]] +name = "lexical-write-integer" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "629ddff1a914a836fb245616a7888b62903aae58fa771e1d83943035efa0f978" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "libc" +version = "0.2.174" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1171693293099992e19cddea4e8b849964e9846f4acee11b3948bcc337be8776" + +[[package]] +name = "libloading" +version = "0.8.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" +dependencies = [ + "cfg-if", + "windows-targets 0.53.2", +] + +[[package]] +name = "libm" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9fbbcab51052fe104eb5e5d351cf728d30a5be1fe14d9be8a3b097481fb97de" + +[[package]] +name = "libz-rs-sys" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "172a788537a2221661b480fee8dc5f96c580eb34fa88764d3205dc356c7e4221" +dependencies = [ + "zlib-rs", +] + +[[package]] +name = "linux-raw-sys" +version = "0.4.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" + +[[package]] +name = "linux-raw-sys" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd945864f07fe9f5371a27ad7b52a172b4b499999f1d97574c9fa68373937e12" + +[[package]] +name = "litemap" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" + +[[package]] +name = "lock_api" +version = "0.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96936507f153605bddfcda068dd804796c84324ed2510809e5b2a624c81da765" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13dc2df351e3202783a1fe0d44375f7295ffb4049267b0f3018346dc122a1d94" + +[[package]] +name = "lru-slab" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" + +[[package]] +name = "lz4_flex" +version = "0.11.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08ab2867e3eeeca90e844d1940eab391c9dc5228783db2ed999acbc0a9ed375a" +dependencies = [ + "twox-hash", +] + +[[package]] +name = "lzma-sys" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fda04ab3764e6cde78b9974eec4f779acaba7c4e84b36eca3cf77c581b85d27" +dependencies = [ + "cc", + "libc", + "pkg-config", +] + +[[package]] +name = "matchers" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" +dependencies = [ + "regex-automata 0.1.10", +] + +[[package]] +name = "matchit" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" + +[[package]] +name = "md-5" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" +dependencies = [ + "cfg-if", + "digest", +] + +[[package]] +name = "memchr" +version = "2.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" + +[[package]] +name = "mime" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" + +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.8.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" +dependencies = [ + "adler2", +] + +[[package]] +name = "mio" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78bed444cc8a2160f01cbcf811ef18cac863ad68ae8ca62092e8db51d51c761c" +dependencies = [ + "libc", + "wasi 0.11.1+wasi-snapshot-preview1", + "windows-sys 0.59.0", +] + +[[package]] +name = "multimap" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" + +[[package]] +name = "nom" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "nu-ansi-term" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +dependencies = [ + "overload", + "winapi", +] + +[[package]] +name = "num" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + +[[package]] +name = "num-bigint" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" +dependencies = [ + "num-integer", + "num-traits", +] + +[[package]] +name = "num-complex" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-integer" +version = "0.1.46" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" +dependencies = [ + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "object" +version = "0.36.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" +dependencies = [ + "memchr", +] + +[[package]] +name = "object_store" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7781f96d79ed0f961a7021424ab01840efbda64ae7a505aaea195efc91eaaec4" +dependencies = [ + "async-trait", + "base64", + "bytes", + "chrono", + "form_urlencoded", + "futures", + "http", + "http-body-util", + "httparse", + "humantime", + "hyper", + "itertools 0.14.0", + "md-5", + "parking_lot", + "percent-encoding", + "quick-xml", + "rand 0.9.1", + "reqwest", + "ring", + "rustls-pemfile", + "serde", + "serde_json", + "serde_urlencoded", + "thiserror 2.0.12", + "tokio", + "tracing", + "url", + "walkdir", + "wasm-bindgen-futures", + "web-time", +] + +[[package]] +name = "once_cell" +version = "1.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" + +[[package]] +name = "once_cell_polyfill" +version = "1.70.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4895175b425cb1f87721b59f0f286c2092bd4af812243672510e1ac53e2e0ad" + +[[package]] +name = "openssl-probe" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d05e27ee213611ffe7d6348b942e8f942b37114c00cc03cec254295a4a17852e" + +[[package]] +name = "ordered-float" +version = "2.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" +dependencies = [ + "num-traits", +] + +[[package]] +name = "overload" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" + +[[package]] +name = "parking" +version = "2.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" + +[[package]] +name = "parking_lot" +version = "0.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70d58bf43669b5795d1576d0641cfb6fbb2057bf629506267a92807158584a13" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc838d2a56b5b1a6c25f55575dfc605fabb63bb2365f6c2353ef9159aa69e4a5" +dependencies = [ + "backtrace", + "cfg-if", + "libc", + "petgraph 0.6.5", + "redox_syscall", + "smallvec", + "thread-id", + "windows-targets 0.52.6", +] + +[[package]] +name = "parquet" +version = "55.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b17da4150748086bd43352bc77372efa9b6e3dbd06a04831d2a98c041c225cfa" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-schema", + "arrow-select", + "base64", + "brotli", + "bytes", + "chrono", + "flate2", + "futures", + "half", + "hashbrown 0.15.4", + "lz4_flex", + "num", + "num-bigint", + "object_store", + "paste", + "seq-macro", + "simdutf8", + "snap", + "thrift", + "tokio", + "twox-hash", + "zstd", +] + +[[package]] +name = "parse-zoneinfo" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f2a05b18d44e2957b88f96ba460715e295bc1d7510468a2f3d3b44535d26c24" +dependencies = [ + "regex", +] + +[[package]] +name = "paste" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" + +[[package]] +name = "pem" +version = "3.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38af38e8470ac9dee3ce1bae1af9c1671fffc44ddfd8bd1d0a3445bf349a8ef3" +dependencies = [ + "base64", + "serde", +] + +[[package]] +name = "percent-encoding" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" + +[[package]] +name = "pest" +version = "2.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1db05f56d34358a8b1066f67cbb203ee3e7ed2ba674a6263a1d5ec6db2204323" +dependencies = [ + "memchr", + "thiserror 2.0.12", + "ucd-trie", +] + +[[package]] +name = "pest_derive" +version = "2.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb056d9e8ea77922845ec74a1c4e8fb17e7c218cc4fc11a15c5d25e189aa40bc" +dependencies = [ + "pest", + "pest_generator", +] + +[[package]] +name = "pest_generator" +version = "2.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87e404e638f781eb3202dc82db6760c8ae8a1eeef7fb3fa8264b2ef280504966" +dependencies = [ + "pest", + "pest_meta", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "pest_meta" +version = "2.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "edd1101f170f5903fde0914f899bb503d9ff5271d7ba76bbb70bea63690cc0d5" +dependencies = [ + "pest", + "sha2", +] + +[[package]] +name = "petgraph" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" +dependencies = [ + "fixedbitset 0.4.2", + "indexmap 2.10.0", +] + +[[package]] +name = "petgraph" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" +dependencies = [ + "fixedbitset 0.5.7", + "indexmap 2.10.0", +] + +[[package]] +name = "petgraph" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54acf3a685220b533e437e264e4d932cfbdc4cc7ec0cd232ed73c08d03b8a7ca" +dependencies = [ + "fixedbitset 0.5.7", + "hashbrown 0.15.4", + "indexmap 2.10.0", + "serde", +] + +[[package]] +name = "phf" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd6780a80ae0c52cc120a26a1a42c1ae51b247a253e4e06113d23d2c2edd078" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_codegen" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aef8048c789fa5e851558d709946d6d79a8ff88c0440c587967f8e94bfb1216a" +dependencies = [ + "phf_generator", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c80231409c20246a13fddb31776fb942c38553c51e871f8cbd687a4cfb5843d" +dependencies = [ + "phf_shared", + "rand 0.8.5", +] + +[[package]] +name = "phf_shared" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67eabc2ef2a60eb7faa00097bd1ffdb5bd28e62bf39990626a582201b7a754e5" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project" +version = "1.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "677f1add503faace112b9f1373e43e9e054bfdd22ff1a63c1bc485eaec6a6a8a" +dependencies = [ + "pin-project-internal", +] + +[[package]] +name = "pin-project-internal" +version = "1.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b3cff922bd51709b605d9ead9aa71031d81447142d828eb4a6eba76fe619f9b" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "pkg-config" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" + +[[package]] +name = "portable-atomic" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" + +[[package]] +name = "portable-atomic-util" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8a2f0d8d040d7848a709caf78912debcc3f33ee4b3cac47d73d1e1069e83507" +dependencies = [ + "portable-atomic", +] + +[[package]] +name = "potential_utf" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5a7c30837279ca13e7c867e9e40053bc68740f988cb07f7ca6df43cc734b585" +dependencies = [ + "zerovec", +] + +[[package]] +name = "ppv-lite86" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" +dependencies = [ + "zerocopy", +] + +[[package]] +name = "prettyplease" +version = "0.2.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "061c1221631e079b26479d25bbf2275bfe5917ae8419cd7e34f13bfc2aa7539a" +dependencies = [ + "proc-macro2", + "syn", +] + +[[package]] +name = "proc-macro2" +version = "1.0.95" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "02b3e5e68a3a1a02aad3ec490a98007cbc13c37cbe84a3cd7b8e406d76e7f778" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.13.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2796faa41db3ec313a31f7624d9286acf277b52de526150b7e69f3debf891ee5" +dependencies = [ + "bytes", + "prost-derive", +] + +[[package]] +name = "prost-build" +version = "0.13.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be769465445e8c1474e9c5dac2018218498557af32d9ed057325ec9a41ae81bf" +dependencies = [ + "heck", + "itertools 0.14.0", + "log", + "multimap", + "once_cell", + "petgraph 0.7.1", + "prettyplease", + "prost", + "prost-types", + "regex", + "syn", + "tempfile", +] + +[[package]] +name = "prost-derive" +version = "0.13.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a56d757972c98b346a9b766e3f02746cde6dd1cd1d1d563472929fdd74bec4d" +dependencies = [ + "anyhow", + "itertools 0.14.0", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "prost-types" +version = "0.13.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52c2c1bf36ddb1a1c396b3601a3cec27c2462e45f07c386894ec3ccf5332bd16" +dependencies = [ + "prost", +] + +[[package]] +name = "psm" +version = "0.1.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" +dependencies = [ + "cc", +] + +[[package]] +name = "quick-xml" +version = "0.37.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quinn" +version = "0.11.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "626214629cda6781b6dc1d316ba307189c85ba657213ce642d9c77670f8202c8" +dependencies = [ + "bytes", + "cfg_aliases", + "pin-project-lite", + "quinn-proto", + "quinn-udp", + "rustc-hash 2.1.1", + "rustls", + "socket2", + "thiserror 2.0.12", + "tokio", + "tracing", + "web-time", +] + +[[package]] +name = "quinn-proto" +version = "0.11.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49df843a9161c85bb8aae55f101bc0bac8bcafd637a620d9122fd7e0b2f7422e" +dependencies = [ + "bytes", + "getrandom 0.3.3", + "lru-slab", + "rand 0.9.1", + "ring", + "rustc-hash 2.1.1", + "rustls", + "rustls-pki-types", + "slab", + "thiserror 2.0.12", + "tinyvec", + "tracing", + "web-time", +] + +[[package]] +name = "quinn-udp" +version = "0.5.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcebb1209ee276352ef14ff8732e24cc2b02bbac986cd74a4c81bcb2f9881970" +dependencies = [ + "cfg_aliases", + "libc", + "once_cell", + "socket2", + "tracing", + "windows-sys 0.59.0", +] + +[[package]] +name = "quote" +version = "1.0.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "r-efi" +version = "5.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha 0.3.1", + "rand_core 0.6.4", +] + +[[package]] +name = "rand" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fbfd9d094a40bf3ae768db9361049ace4c0e04a4fd6b359518bd7b73a73dd97" +dependencies = [ + "rand_chacha 0.9.0", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core 0.6.4", +] + +[[package]] +name = "rand_chacha" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" +dependencies = [ + "ppv-lite86", + "rand_core 0.9.3", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom 0.2.16", +] + +[[package]] +name = "rand_core" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "99d9a13982dcf210057a8a78572b2217b667c3beacbf3a0d8b454f6f82837d38" +dependencies = [ + "getrandom 0.3.3", +] + +[[package]] +name = "recursive" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0786a43debb760f491b1bc0269fe5e84155353c67482b9e60d0cfb596054b43e" +dependencies = [ + "recursive-proc-macro-impl", + "stacker", +] + +[[package]] +name = "recursive-proc-macro-impl" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" +dependencies = [ + "quote", + "syn", +] + +[[package]] +name = "redox_syscall" +version = "0.5.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d04b7d0ee6b4a0207a0a7adb104d23ecb0b47d6beae7152d0fa34b692b29fd6" +dependencies = [ + "bitflags", +] + +[[package]] +name = "regex" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata 0.4.9", + "regex-syntax 0.8.5", +] + +[[package]] +name = "regex-automata" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax 0.8.5", +] + +[[package]] +name = "regex-syntax" +version = "0.6.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" + +[[package]] +name = "regex-syntax" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" + +[[package]] +name = "reqwest" +version = "0.12.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eabf4c97d9130e2bf606614eb937e86edac8292eaa6f422f995d7e8de1eb1813" +dependencies = [ + "base64", + "bytes", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "http-body-util", + "hyper", + "hyper-rustls", + "hyper-util", + "js-sys", + "log", + "percent-encoding", + "pin-project-lite", + "quinn", + "rustls", + "rustls-native-certs 0.8.1", + "rustls-pki-types", + "serde", + "serde_json", + "serde_urlencoded", + "sync_wrapper", + "tokio", + "tokio-rustls", + "tokio-util", + "tower 0.5.2", + "tower-http", + "tower-service", + "url", + "wasm-bindgen", + "wasm-bindgen-futures", + "wasm-streams", + "web-sys", +] + +[[package]] +name = "ring" +version = "0.17.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4689e6c2294d81e88dc6261c768b63bc4fcdb852be6d1352498b114f61383b7" +dependencies = [ + "cc", + "cfg-if", + "getrandom 0.2.16", + "libc", + "untrusted", + "windows-sys 0.52.0", +] + +[[package]] +name = "rustc-demangle" +version = "0.1.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "989e6739f80c4ad5b13e0fd7fe89531180375b18520cc8c82080e4dc4035b84f" + +[[package]] +name = "rustc-hash" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" + +[[package]] +name = "rustc-hash" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "357703d41365b4b27c590e3ed91eabb1b663f07c4c084095e60cbed4362dff0d" + +[[package]] +name = "rustc_version" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" +dependencies = [ + "semver", +] + +[[package]] +name = "rustix" +version = "0.38.44" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" +dependencies = [ + "bitflags", + "errno", + "libc", + "linux-raw-sys 0.4.15", + "windows-sys 0.59.0", +] + +[[package]] +name = "rustix" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c71e83d6afe7ff64890ec6b71d6a69bb8a610ab78ce364b3352876bb4c801266" +dependencies = [ + "bitflags", + "errno", + "libc", + "linux-raw-sys 0.9.4", + "windows-sys 0.59.0", +] + +[[package]] +name = "rustls" +version = "0.23.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7160e3e10bf4535308537f3c4e1641468cd0e485175d6163087c0393c7d46643" +dependencies = [ + "aws-lc-rs", + "log", + "once_cell", + "ring", + "rustls-pki-types", + "rustls-webpki", + "subtle", + "zeroize", +] + +[[package]] +name = "rustls-native-certs" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" +dependencies = [ + "openssl-probe", + "rustls-pemfile", + "rustls-pki-types", + "schannel", + "security-framework 2.11.1", +] + +[[package]] +name = "rustls-native-certs" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fcff2dd52b58a8d98a70243663a0d234c4e2b79235637849d15913394a247d3" +dependencies = [ + "openssl-probe", + "rustls-pki-types", + "schannel", + "security-framework 3.2.0", +] + +[[package]] +name = "rustls-pemfile" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dce314e5fee3f39953d46bb63bb8a46d40c2f8fb7cc5a3b6cab2bde9721d6e50" +dependencies = [ + "rustls-pki-types", +] + +[[package]] +name = "rustls-pki-types" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +dependencies = [ + "web-time", + "zeroize", +] + +[[package]] +name = "rustls-webpki" +version = "0.103.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4a72fe2bcf7a6ac6fd7d0b9e5cb68aeb7d4c0a0271730218b3e92d43b4eb435" +dependencies = [ + "aws-lc-rs", + "ring", + "rustls-pki-types", + "untrusted", +] + +[[package]] +name = "rustversion" +version = "1.0.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a0d197bd2c9dc6e53b84da9556a69ba4cdfab8619eb41a8bd1cc2027a0f6b1d" + +[[package]] +name = "ryu" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "28d3b2b1366ec20994f1fd18c3c594f05c5dd4bc44d8bb0c1c632c8d6829481f" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "schannel" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f29ebaa345f945cec9fbbc532eb307f0fdad8161f281b6369539c8d84876b3d" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "schemars" +version = "0.8.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fbf2ae1b8bc8e02df939598064d22402220cd5bbcca1c76f7d6a310974d5615" +dependencies = [ + "dyn-clone", + "schemars_derive", + "serde", + "serde_json", +] + +[[package]] +name = "schemars_derive" +version = "0.8.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32e265784ad618884abaea0600a9adf15393368d840e0222d101a072f3f7534d" +dependencies = [ + "proc-macro2", + "quote", + "serde_derive_internals", + "syn", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "secrecy" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e891af845473308773346dc847b2c23ee78fe442e0472ac50e22a18a93d3ae5a" +dependencies = [ + "zeroize", +] + +[[package]] +name = "security-framework" +version = "2.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" +dependencies = [ + "bitflags", + "core-foundation 0.9.4", + "core-foundation-sys", + "libc", + "security-framework-sys", +] + +[[package]] +name = "security-framework" +version = "3.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "271720403f46ca04f7ba6f55d438f8bd878d6b8ca0a1046e8228c4145bcbb316" +dependencies = [ + "bitflags", + "core-foundation 0.10.1", + "core-foundation-sys", + "libc", + "security-framework-sys", +] + +[[package]] +name = "security-framework-sys" +version = "2.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49db231d56a190491cb4aeda9527f1ad45345af50b0851622a7adb8c03b01c32" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "semver" +version = "1.0.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56e6fa9c48d24d85fb3de5ad847117517440f6beceb7798af16b4a87d616b8d0" + +[[package]] +name = "seq-macro" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" + +[[package]] +name = "serde" +version = "1.0.219" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f0e2c6ed6606019b4e29e69dbaba95b11854410e5347d525002456dbbb786b6" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde-value" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +dependencies = [ + "ordered-float", + "serde", +] + +[[package]] +name = "serde_derive" +version = "1.0.219" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b0276cf7f2c73365f7157c8123c21cd9a50fbbd844757af28ca1f5925fc2a00" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "serde_derive_internals" +version = "0.29.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "serde_json" +version = "1.0.140" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20068b6e96dc6c9bd23e01df8827e6c7e1f2fddd43c21810382803c136b99373" +dependencies = [ + "itoa", + "memchr", + "ryu", + "serde", +] + +[[package]] +name = "serde_urlencoded" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3491c14715ca2294c4d6a88f15e84739788c1d030eed8c110436aafdaa2f3fd" +dependencies = [ + "form_urlencoded", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.34+deprecated" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" +dependencies = [ + "indexmap 2.10.0", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] + +[[package]] +name = "sha1" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3bf829a2d51ab4a5ddf1352d8470c140cadc8301b2ae1789db023f01cedd6ba" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + +[[package]] +name = "sha2" +version = "0.10.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + +[[package]] +name = "sharded-slab" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f40ca3c46823713e0d4209592e8d6e826aa57e928f09752619fc696c499637f6" +dependencies = [ + "lazy_static", +] + +[[package]] +name = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + +[[package]] +name = "signal-hook-registry" +version = "1.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9203b8055f63a2a00e2f593bb0510367fe707d7ff1e5c872de2f537b339e5410" +dependencies = [ + "libc", +] + +[[package]] +name = "simdutf8" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3a9fe34e3e7a50316060351f37187a3f546bce95496156754b601a5fa71b76e" + +[[package]] +name = "siphasher" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56199f7ddabf13fe5074ce809e7d3f42b42ae711800501b5b16ea82ad029c39d" + +[[package]] +name = "slab" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04dc19736151f35336d325007ac991178d504a119863a2fcb3758cdb5e52c50d" + +[[package]] +name = "smallvec" +version = "1.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67b1b7a3b5fe4f1376887184045fcf45c69e92af734b7aaddc05fb777b6fbd03" + +[[package]] +name = "snap" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" + +[[package]] +name = "socket2" +version = "0.5.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e22376abed350d73dd1cd119b57ffccad95b4e585a7cda43e286245ce23c0678" +dependencies = [ + "libc", + "windows-sys 0.52.0", +] + +[[package]] +name = "sqlparser" +version = "0.55.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4521174166bac1ff04fe16ef4524c70144cd29682a45978978ca3d7f4e0be11" +dependencies = [ + "log", + "recursive", + "sqlparser_derive", +] + +[[package]] +name = "sqlparser_derive" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + +[[package]] +name = "stacker" +version = "0.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "windows-sys 0.59.0", +] + +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "strsim" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" + +[[package]] +name = "subtle" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" + +[[package]] +name = "syn" +version = "2.0.104" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "17b6f705963418cdb9927482fa304bc562ece2fdd4f616084c50b7023b435a40" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "sync_wrapper" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bf256ce5efdfa370213c1dabab5935a12e49f2c58d15e9eac2870d3b4f27263" +dependencies = [ + "futures-core", +] + +[[package]] +name = "synstructure" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tempfile" +version = "3.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8a64e3985349f2441a1a9ef0b853f869006c3855f2cda6862a94d26ebb9d6a1" +dependencies = [ + "fastrand", + "getrandom 0.3.3", + "once_cell", + "rustix 1.0.7", + "windows-sys 0.59.0", +] + +[[package]] +name = "test-log" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7f46083d221181166e5b6f6b1e5f1d499f3a76888826e6cb1d057554157cd0f" +dependencies = [ + "env_logger", + "test-log-macros", + "tracing-subscriber", +] + +[[package]] +name = "test-log-macros" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "888d0c3c6db53c0fdab160d2ed5e12ba745383d3e85813f2ea0f2b1475ab553f" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "thiserror" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" +dependencies = [ + "thiserror-impl 1.0.69", +] + +[[package]] +name = "thiserror" +version = "2.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "567b8a2dae586314f7be2a752ec7474332959c6460e02bde30d702a66d488708" +dependencies = [ + "thiserror-impl 2.0.12", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "thiserror-impl" +version = "2.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f7cf42b4507d8ea322120659672cf1b9dbb93f8f2d4ecfd6e51350ff5b17a1d" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "thread-id" +version = "4.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfe8f25bbdd100db7e1d34acf7fd2dc59c4bf8f7483f505eaa7d4f12f76cc0ea" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "thread_local" +version = "1.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f60246a4944f24f6e018aa17cdeffb7818b76356965d03b07d6a9886e8962185" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding", + "ordered-float", +] + +[[package]] +name = "tikv-jemalloc-ctl" +version = "0.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "619bfed27d807b54f7f776b9430d4f8060e66ee138a28632ca898584d462c31c" +dependencies = [ + "libc", + "paste", + "tikv-jemalloc-sys", +] + +[[package]] +name = "tikv-jemalloc-sys" +version = "0.5.4+5.3.0-patched" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9402443cb8fd499b6f327e40565234ff34dbda27460c5b47db0db77443dd85d1" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "tikv-jemallocator" +version = "0.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "965fe0c26be5c56c94e38ba547249074803efd52adfb66de62107d95aab3eaca" +dependencies = [ + "libc", + "tikv-jemalloc-sys", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[package]] +name = "tinystr" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +dependencies = [ + "displaydoc", + "zerovec", +] + +[[package]] +name = "tinyvec" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09b3661f17e86524eccd4371ab0429194e0d7c008abb45f7a7495b1719463c71" +dependencies = [ + "tinyvec_macros", +] + +[[package]] +name = "tinyvec_macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" + +[[package]] +name = "tokio" +version = "1.45.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75ef51a33ef1da925cea3e4eb122833cb377c61439ca401b770f54902b806779" +dependencies = [ + "backtrace", + "bytes", + "libc", + "mio", + "parking_lot", + "pin-project-lite", + "signal-hook-registry", + "socket2", + "tokio-macros", + "windows-sys 0.52.0", +] + +[[package]] +name = "tokio-macros" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tokio-rustls" +version = "0.26.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b" +dependencies = [ + "rustls", + "tokio", +] + +[[package]] +name = "tokio-stream" +version = "0.1.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eca58d7bba4a75707817a2c44174253f9236b2d5fbd055602e9d5c07c139a047" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", + "tokio-util", +] + +[[package]] +name = "tokio-util" +version = "0.7.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66a539a9ad6d5d281510d5bd368c973d636c02dbf8a67300bfb6b950696ad7df" +dependencies = [ + "bytes", + "futures-core", + "futures-sink", + "pin-project-lite", + "slab", + "tokio", +] + +[[package]] +name = "tonic" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "877c5b330756d856ffcc4553ab34a5684481ade925ecc54bcd1bf02b1d0d4d52" +dependencies = [ + "async-stream", + "async-trait", + "axum", + "base64", + "bytes", + "h2", + "http", + "http-body", + "http-body-util", + "hyper", + "hyper-timeout", + "hyper-util", + "percent-encoding", + "pin-project", + "prost", + "socket2", + "tokio", + "tokio-stream", + "tower 0.4.13", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tonic-build" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9557ce109ea773b399c9b9e5dca39294110b74f1f342cb347a80d1fce8c26a11" +dependencies = [ + "prettyplease", + "proc-macro2", + "prost-build", + "prost-types", + "quote", + "syn", +] + +[[package]] +name = "tower" +version = "0.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" +dependencies = [ + "futures-core", + "futures-util", + "indexmap 1.9.3", + "pin-project", + "pin-project-lite", + "rand 0.8.5", + "slab", + "tokio", + "tokio-util", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d039ad9159c98b70ecfd540b2573b97f7f52c3e8d9f8ad57a24b916a536975f9" +dependencies = [ + "futures-core", + "futures-util", + "pin-project-lite", + "sync_wrapper", + "tokio", + "tokio-util", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower-http" +version = "0.6.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "adc82fd73de2a9722ac5da747f12383d2bfdb93591ee6c58486e0097890f05f2" +dependencies = [ + "base64", + "bitflags", + "bytes", + "futures-util", + "http", + "http-body", + "iri-string", + "mime", + "pin-project-lite", + "tower 0.5.2", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower-layer" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "121c2a6cda46980bb0fcd1647ffaf6cd3fc79a013de288782836f6df9c48780e" + +[[package]] +name = "tower-service" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8df9b6e13f2d32c91b9bd719c00d1958837bc7dec474d94952798cc8e69eeec3" + +[[package]] +name = "tracing" +version = "0.1.41" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "784e0ac535deb450455cbfa28a6f0df145ea1bb7ae51b821cf5e7927fdcfbdd0" +dependencies = [ + "log", + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tracing-core" +version = "0.1.34" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9d12581f227e93f094d3af2ae690a574abb8a2b9b7a96e7cfe9647b2b617678" +dependencies = [ + "once_cell", + "valuable", +] + +[[package]] +name = "tracing-log" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee855f1f400bd0e5c02d150ae5de3840039a3f54b025156404e34c23c03f47c3" +dependencies = [ + "log", + "once_cell", + "tracing-core", +] + +[[package]] +name = "tracing-subscriber" +version = "0.3.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8189decb5ac0fa7bc8b96b7cb9b2701d60d48805aca84a238004d665fcc4008" +dependencies = [ + "matchers", + "nu-ansi-term", + "once_cell", + "regex", + "sharded-slab", + "thread_local", + "tracing", + "tracing-core", + "tracing-log", +] + +[[package]] +name = "try-lock" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" + +[[package]] +name = "twox-hash" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b907da542cbced5261bd3256de1b3a1bf340a3d37f93425a07362a1d687de56" + +[[package]] +name = "typenum" +version = "1.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" + +[[package]] +name = "ucd-trie" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2896d95c02a80c6d6a5d6e953d479f5ddf2dfdb6a244441010e373ac0fb88971" + +[[package]] +name = "unicode-ident" +version = "1.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a5f39404a5da50712a4c1eecf25e90dd62b613502b7e925fd4e4d19b5c96512" + +[[package]] +name = "unicode-segmentation" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" + +[[package]] +name = "unicode-width" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a1a07cc7db3810833284e8d372ccdc6da29741639ecc70c9ec107df0fa6154c" + +[[package]] +name = "unsafe-libyaml" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" + +[[package]] +name = "untrusted" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" + +[[package]] +name = "url" +version = "2.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" +dependencies = [ + "form_urlencoded", + "idna", + "percent-encoding", +] + +[[package]] +name = "utf8_iter" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" + +[[package]] +name = "utf8parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" + +[[package]] +name = "uuid" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3cf4199d1e5d15ddd86a694e4d0dffa9c323ce759fea589f00fef9d81cc1931d" +dependencies = [ + "getrandom 0.3.3", + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "valuable" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba73ea9cf16a25df0c8caa16c51acb937d5712a8429db78a3ee29d5dcacd3a65" + +[[package]] +name = "version_check" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" + +[[package]] +name = "walkdir" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "want" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa7760aed19e106de2c7c0b581b509f2f25d3dacaf737cb82ac61bc6d760b0e" +dependencies = [ + "try-lock", +] + +[[package]] +name = "wasi" +version = "0.11.1+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" + +[[package]] +name = "wasi" +version = "0.14.2+wasi-0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9683f9a5a998d873c0d21fcbe3c083009670149a8fab228644b8bd36b2c48cb3" +dependencies = [ + "wit-bindgen-rt", +] + +[[package]] +name = "wasm-bindgen" +version = "0.2.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1edc8929d7499fc4e8f0be2262a241556cfc54a0bea223790e71446f2aab1ef5" +dependencies = [ + "cfg-if", + "once_cell", + "rustversion", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2f0a0651a5c2bc21487bde11ee802ccaf4c51935d0d3d42a6101f98161700bc6" +dependencies = [ + "bumpalo", + "log", + "proc-macro2", + "quote", + "syn", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-futures" +version = "0.4.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "555d470ec0bc3bb57890405e5d4322cc9ea83cebb085523ced7be4144dac1e61" +dependencies = [ + "cfg-if", + "js-sys", + "once_cell", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fe63fc6d09ed3792bd0897b314f53de8e16568c2b3f7982f468c0bf9bd0b407" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" +dependencies = [ + "proc-macro2", + "quote", + "syn", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a05d73b933a847d6cccdda8f838a22ff101ad9bf93e33684f39c1f5f0eece3d" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "wasm-streams" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "15053d8d85c7eccdbefef60f06769760a563c7f0a9d6902a13d35c7800b0ad65" +dependencies = [ + "futures-util", + "js-sys", + "wasm-bindgen", + "wasm-bindgen-futures", + "web-sys", +] + +[[package]] +name = "web-sys" +version = "0.3.77" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33b6dd2ef9186f1f2072e409e99cd22a975331a6b3591b12c764e0e55c60d5d2" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "web-time" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a6580f308b1fad9207618087a65c04e7a10bc77e02c8e84e9b00dd4b12fa0bb" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "which" +version = "4.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" +dependencies = [ + "either", + "home", + "once_cell", + "rustix 0.38.44", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-util" +version = "0.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-core" +version = "0.61.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0fdd3ddb90610c7638aa2b3a3ab2904fb9e5cdbecc643ddb3647212781c4ae3" +dependencies = [ + "windows-implement", + "windows-interface", + "windows-link", + "windows-result", + "windows-strings", +] + +[[package]] +name = "windows-implement" +version = "0.60.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "windows-interface" +version = "0.59.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "windows-link" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" + +[[package]] +name = "windows-result" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56f42bd332cc6c8eac5af113fc0c1fd6a8fd2aa08a0119358686e5160d0586c6" +dependencies = [ + "windows-link", +] + +[[package]] +name = "windows-strings" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56e6c93f3a0c3b36176cb1327a4958a0353d5d166c2a35cb268ace15e91d3b57" +dependencies = [ + "windows-link", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.60.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" +dependencies = [ + "windows-targets 0.53.2", +] + +[[package]] +name = "windows-targets" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" +dependencies = [ + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", + "windows_i686_gnullvm 0.52.6", + "windows_i686_msvc 0.52.6", + "windows_x86_64_gnu 0.52.6", + "windows_x86_64_gnullvm 0.52.6", + "windows_x86_64_msvc 0.52.6", +] + +[[package]] +name = "windows-targets" +version = "0.53.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c66f69fcc9ce11da9966ddb31a40968cad001c5bedeb5c2b82ede4253ab48aef" +dependencies = [ + "windows_aarch64_gnullvm 0.53.0", + "windows_aarch64_msvc 0.53.0", + "windows_i686_gnu 0.53.0", + "windows_i686_gnullvm 0.53.0", + "windows_i686_msvc 0.53.0", + "windows_x86_64_gnu 0.53.0", + "windows_x86_64_gnullvm 0.53.0", + "windows_x86_64_msvc 0.53.0", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86b8d5f90ddd19cb4a147a5fa63ca848db3df085e25fee3cc10b39b6eebae764" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7651a1f62a11b8cbd5e0d42526e55f2c99886c77e007179efff86c2b137e66c" + +[[package]] +name = "windows_i686_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" + +[[package]] +name = "windows_i686_gnu" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1dc67659d35f387f5f6c479dc4e28f1d4bb90ddd1a5d3da2e5d97b42d6272c3" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ce6ccbdedbf6d6354471319e781c0dfef054c81fbc7cf83f338a4296c0cae11" + +[[package]] +name = "windows_i686_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" + +[[package]] +name = "windows_i686_msvc" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "581fee95406bb13382d2f65cd4a908ca7b1e4c2f1917f143ba16efe98a589b5d" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e55b5ac9ea33f2fc1716d1742db15574fd6fc8dadc51caab1c16a3d3b4190ba" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0a6e035dd0599267ce1ee132e51c27dd29437f63325753051e71dd9e42406c57" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" + +[[package]] +name = "wit-bindgen-rt" +version = "0.39.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f42320e61fe2cfd34354ecb597f86f413484a798ba44a8ca1165c58d42da6c1" +dependencies = [ + "bitflags", +] + +[[package]] +name = "writeable" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" + +[[package]] +name = "xz2" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "388c44dc09d76f1536602ead6d325eb532f5c122f17782bd57fb47baeeb767e2" +dependencies = [ + "lzma-sys", +] + +[[package]] +name = "yoke" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +dependencies = [ + "serde", + "stable_deref_trait", + "yoke-derive", + "zerofrom", +] + +[[package]] +name = "yoke-derive" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +dependencies = [ + "proc-macro2", + "quote", + "syn", + "synstructure", +] + +[[package]] +name = "zerocopy" +version = "0.8.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1039dd0d3c310cf05de012d8a39ff557cb0d23087fd44cad61df08fc31907a2f" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.8.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ecf5b4cc5364572d7f4c329661bcc82724222973f2cab6f050a4e5c22f75181" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "zerofrom" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50cc42e0333e05660c3587f3bf9d0478688e15d870fab3346451ce7f8c9fbea5" +dependencies = [ + "zerofrom-derive", +] + +[[package]] +name = "zerofrom-derive" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" +dependencies = [ + "proc-macro2", + "quote", + "syn", + "synstructure", +] + +[[package]] +name = "zeroize" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" + +[[package]] +name = "zerotrie" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +dependencies = [ + "displaydoc", + "yoke", + "zerofrom", +] + +[[package]] +name = "zerovec" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a05eb080e015ba39cc9e23bbe5e7fb04d5fb040350f99f34e338d5fdd294428" +dependencies = [ + "yoke", + "zerofrom", + "zerovec-derive", +] + +[[package]] +name = "zerovec-derive" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "zlib-rs" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "626bd9fa9734751fc50d6060752170984d7053f5a39061f524cda68023d4db8a" + +[[package]] +name = "zstd" +version = "0.13.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e91ee311a569c327171651566e07972200e76fcfe2242a4fa446149a3881c08a" +dependencies = [ + "zstd-safe", +] + +[[package]] +name = "zstd-safe" +version = "7.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f49c4d5f0abb602a93fb8736af2a4f4dd9512e36f7f570d66e65ff867ed3b9d" +dependencies = [ + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.15+zstd.1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eb81183ddd97d0c74cedf1d50d85c8d08c1b8b68ee863bdee9e706eedba1a237" +dependencies = [ + "cc", + "pkg-config", +] From e0db0e367ccfaf7b6eae7fc1c8007c6ef1b611f4 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 10 Jul 2025 15:40:11 -0400 Subject: [PATCH 6/6] clean up python tpch client a bit --- .../launch_python_arrowflightsql_client.sh | 2 +- src/analyze.rs | 10 +- src/query_planner.rs | 92 ++++++++++++------- tests/common/mod.rs | 16 ++-- 4 files changed, 74 insertions(+), 46 deletions(-) diff --git a/scripts/launch_python_arrowflightsql_client.sh b/scripts/launch_python_arrowflightsql_client.sh index 90d4332..44776e4 100755 --- a/scripts/launch_python_arrowflightsql_client.sh +++ b/scripts/launch_python_arrowflightsql_client.sh @@ -185,7 +185,7 @@ def run_sql(sql_query): rich_table.add_row(*row_data) console = Console() - console.print(rich_table) + console.print(rich_table, markup=False) except Exception as e: print(f"Error executing SQL query: {str(e)}") diff --git a/src/analyze.rs b/src/analyze.rs index 5828d17..fac7467 100644 --- a/src/analyze.rs +++ b/src/analyze.rs @@ -115,7 +115,7 @@ pub struct DistributedAnalyzeRootExec { impl DistributedAnalyzeRootExec { pub fn new(input: Arc, verbose: bool, show_statistics: bool) -> Self { let field_a = Field::new("Task", DataType::Utf8, false); - let field_b = Field::new("Plan", DataType::Utf8, false); + let field_b = Field::new("Plan with Metrics", DataType::Utf8, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); let properties = PlanProperties::new( @@ -226,7 +226,7 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { .clone(); // we want to gather all partitions - let coalesce = CoalescePartitionsExec::new(self.input.clone()); + let coalesce = Arc::new(CoalescePartitionsExec::new(self.input.clone())); let mut input_stream = coalesce.execute(partition, context)?; @@ -272,10 +272,10 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { tasks.sort_by_key(|t| (t.stage_id, t.partition_group.clone())); + trace!("sorted tasks: {:?}", tasks); + let mut task_builder = StringBuilder::with_capacity(1, 1024); let mut plan_builder = StringBuilder::with_capacity(1, 1024); - task_builder.append_value("Task"); - plan_builder.append_value("Plan with Metrics"); for task_output in tasks.iter() { task_builder.append_value(format!( @@ -285,7 +285,7 @@ impl ExecutionPlan for DistributedAnalyzeRootExec { task_output .host .as_ref() - .map(|h| h.to_string()) + .map(|h| format!("{} {}", h.name, h.addr)) .unwrap_or("Unknown".to_string()) )); plan_builder.append_value(&task_output.plan); diff --git a/src/query_planner.rs b/src/query_planner.rs index af9a770..14ecc78 100644 --- a/src/query_planner.rs +++ b/src/query_planner.rs @@ -1,12 +1,15 @@ use std::sync::Arc; use anyhow::anyhow; -use arrow::datatypes::SchemaRef; +use arrow::{compute::concat_batches, datatypes::SchemaRef}; use datafusion::{ - logical_expr::LogicalPlan, physical_plan::ExecutionPlan, prelude::SessionContext, + logical_expr::LogicalPlan, + physical_plan::{coalesce_partitions::CoalescePartitionsExec, ExecutionPlan}, + prelude::SessionContext, }; use datafusion_substrait::{logical_plan::consumer::from_substrait_plan, substrait::proto::Plan}; +use tokio_stream::StreamExt; use crate::{ explain::build_explain_batch, @@ -77,6 +80,7 @@ impl QueryPlanner { match logical_plan { p @ LogicalPlan::Explain(_) => self.prepare_explain(p, ctx).await, // add other logical plans for local execution here following the pattern for explain + p @ LogicalPlan::DescribeTable(_) => self.prepare_local(p, ctx).await, p => self.prepare_query(p, ctx).await, } } @@ -89,6 +93,7 @@ impl QueryPlanner { match logical_plan { p @ LogicalPlan::Explain(_) => self.prepare_explain(p, ctx).await, // add other logical plans for local execution here following the pattern for explain + p @ LogicalPlan::DescribeTable(_) => self.prepare_local(p, ctx).await, p => self.prepare_query(p, ctx).await, } } @@ -103,44 +108,30 @@ impl QueryPlanner { self.send_it(logical_plan, physical_plan, ctx).await } - async fn send_it( + async fn prepare_local( &self, logical_plan: LogicalPlan, - physical_plan: Arc, ctx: SessionContext, ) -> Result { - let query_id = uuid::Uuid::new_v4().to_string(); - - // divide the physical plan into chunks (tasks) that we can distribute to workers - let (distributed_plan, distributed_stages) = - execution_planning(physical_plan.clone(), 8192, Some(2)).await?; + let physical_plan = physical_planning(&logical_plan, &ctx).await?; - let worker_addrs = get_worker_addresses()?; + // execute it locally + let mut stream = + Arc::new(CoalescePartitionsExec::new(physical_plan)).execute(0, ctx.task_ctx())?; + let mut batches = vec![]; - // gather some information we need to send back such that - // we can send a ticket to the client - let final_stage = &distributed_stages[distributed_stages.len() - 1]; - let schema = Arc::clone(&final_stage.plan.schema()); - let final_stage_id = final_stage.stage_id; + while let Some(batch) = stream.next().await { + batches.push(batch?); + } - // distribute the stages to workers, further dividing them up - // into chunks of partitions (partition_groups) - let (final_workers, tasks) = - distribute_stages(&query_id, distributed_stages, worker_addrs).await?; + if batches.is_empty() { + return Err(anyhow!("No data returned from local execution").into()); + } - let qp = QueryPlan { - query_id, - session_context: ctx, - worker_addresses: final_workers, - final_stage_id, - schema, - logical_plan, - physical_plan, - distributed_plan, - distributed_tasks: tasks, - }; + let combined_batch = concat_batches(&batches[0].schema(), &batches)?; + let physical_plan = Arc::new(RecordBatchExec::new(combined_batch)); - Ok(qp) + self.send_it(logical_plan, physical_plan, ctx).await } async fn prepare_explain( @@ -172,4 +163,43 @@ impl QueryPlanner { ) .await } + async fn send_it( + &self, + logical_plan: LogicalPlan, + physical_plan: Arc, + ctx: SessionContext, + ) -> Result { + let query_id = uuid::Uuid::new_v4().to_string(); + + // divide the physical plan into chunks (tasks) that we can distribute to workers + let (distributed_plan, distributed_stages) = + execution_planning(physical_plan.clone(), 8192, Some(2)).await?; + + let worker_addrs = get_worker_addresses()?; + + // gather some information we need to send back such that + // we can send a ticket to the client + let final_stage = &distributed_stages[distributed_stages.len() - 1]; + let schema = Arc::clone(&final_stage.plan.schema()); + let final_stage_id = final_stage.stage_id; + + // distribute the stages to workers, further dividing them up + // into chunks of partitions (partition_groups) + let (final_workers, tasks) = + distribute_stages(&query_id, distributed_stages, worker_addrs).await?; + + let qp = QueryPlan { + query_id, + session_context: ctx, + worker_addresses: final_workers, + final_stage_id, + schema, + logical_plan, + physical_plan, + distributed_plan, + distributed_tasks: tasks, + }; + + Ok(qp) + } } diff --git a/tests/common/mod.rs b/tests/common/mod.rs index 8ab47cc..db821a0 100644 --- a/tests/common/mod.rs +++ b/tests/common/mod.rs @@ -220,7 +220,7 @@ except Exception as e: print(f"Error executing distributed query: {{str(e)}}", file=sys.stderr) sys.exit(1) "#, - self.get_proxy_address().split(':').last().unwrap() + self.get_proxy_address().split(':').next_back().unwrap() ); let script_path = Self::write_temp_file( @@ -271,15 +271,13 @@ except Exception as e: for port in &ports { // Find and kill processes using lsof if let Ok(output) = Command::new("lsof") - .args(&["-ti", &format!(":{}", port)]) + .args(["-ti", &format!(":{}", port)]) .output() { let stdout = String::from_utf8_lossy(&output.stdout); for line in stdout.lines() { if let Ok(pid) = line.trim().parse::() { - let _ = Command::new("kill") - .args(&["-9", &pid.to_string()]) - .output(); + let _ = Command::new("kill").args(["-9", &pid.to_string()]).output(); println!(" 🔥 Killed process {} on port {}", pid, port); } } @@ -362,7 +360,7 @@ except Exception as e: // Check if packages are already installed let check_cmd = Command::new("python3") - .args(&["-c", "import adbc_driver_manager; import adbc_driver_flightsql; import duckdb; import pyarrow; print('OK')"]) + .args(["-c", "import adbc_driver_manager; import adbc_driver_flightsql; import duckdb; import pyarrow; print('OK')"]) .output(); if let Ok(output) = check_cmd { @@ -449,7 +447,7 @@ except Exception as e: let worker = Self::spawn_process( binary_path_str, &["--mode", "worker", "--port", &port.to_string()], - &[("DFRAY_TABLES", &tpch_tables), ("DFRAY_VIEWS", &tpch_views)], + &[("DFRAY_TABLES", &tpch_tables), ("DFRAY_VIEWS", tpch_views)], &format!("start worker {}", i + 1), )?; self.worker_processes.push(worker); @@ -473,7 +471,7 @@ except Exception as e: &[ ("DFRAY_WORKER_ADDRESSES", &worker_addresses), ("DFRAY_TABLES", &tpch_tables), - ("DFRAY_VIEWS", &tpch_views), + ("DFRAY_VIEWS", tpch_views), ], "start proxy", )?; @@ -819,7 +817,7 @@ pub fn batches_to_sorted_strings( let mut in_table = false; let mut header_found = false; - for (_line_idx, line) in lines.iter().enumerate() { + for line in lines.iter() { let trimmed = line.trim(); // Detect table boundaries (+---+)