|
| 1 | +use crate::DefaultSessionBuilder; |
| 2 | +use crate::test_utils::localhost::start_localhost_context; |
| 3 | +use crate::test_utils::parquet::register_parquet_tables; |
| 4 | +use async_trait::async_trait; |
| 5 | +use datafusion::arrow::array::RecordBatch; |
| 6 | +use datafusion::arrow::util::display::array_value_to_string; |
| 7 | +use datafusion::common::runtime::JoinSet; |
| 8 | +use datafusion::error::DataFusionError; |
| 9 | +use datafusion::execution::context::SessionContext; |
| 10 | +use sqllogictest::{AsyncDB, DBOutput, DefaultColumnType}; |
| 11 | +use std::sync::Arc; |
| 12 | + |
| 13 | +pub struct DatafusionDistributedDB { |
| 14 | + ctx: SessionContext, |
| 15 | + _guard: JoinSet<()>, |
| 16 | +} |
| 17 | + |
| 18 | +impl DatafusionDistributedDB { |
| 19 | + pub async fn new(num_nodes: usize) -> Self { |
| 20 | + let (ctx, _guard) = start_localhost_context(num_nodes, DefaultSessionBuilder).await; |
| 21 | + register_parquet_tables(&ctx).await.unwrap(); |
| 22 | + Self { ctx, _guard } |
| 23 | + } |
| 24 | + |
| 25 | + fn convert_batches_to_output( |
| 26 | + &self, |
| 27 | + batches: Vec<RecordBatch>, |
| 28 | + ) -> Result<DBOutput<DefaultColumnType>, datafusion::error::DataFusionError> { |
| 29 | + if batches.is_empty() { |
| 30 | + return Ok(DBOutput::Rows { |
| 31 | + types: vec![], |
| 32 | + rows: vec![], |
| 33 | + }); |
| 34 | + } |
| 35 | + |
| 36 | + let num_columns = batches[0].num_columns(); |
| 37 | + let column_types = vec![DefaultColumnType::Text; num_columns]; // Everything as text |
| 38 | + |
| 39 | + let mut rows = Vec::new(); |
| 40 | + for batch in batches { |
| 41 | + for row_idx in 0..batch.num_rows() { |
| 42 | + let mut row = Vec::new(); |
| 43 | + for col_idx in 0..batch.num_columns() { |
| 44 | + let column = batch.column(col_idx); |
| 45 | + let value = array_value_to_string(column, row_idx) |
| 46 | + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?; |
| 47 | + row.push(value); |
| 48 | + } |
| 49 | + rows.push(row); |
| 50 | + } |
| 51 | + } |
| 52 | + |
| 53 | + Ok(DBOutput::Rows { |
| 54 | + types: column_types, |
| 55 | + rows, |
| 56 | + }) |
| 57 | + } |
| 58 | + |
| 59 | + async fn handle_explain_analyze( |
| 60 | + &mut self, |
| 61 | + _sql: &str, |
| 62 | + ) -> Result<DBOutput<DefaultColumnType>, datafusion::error::DataFusionError> { |
| 63 | + unimplemented!(); |
| 64 | + } |
| 65 | + |
| 66 | + async fn handle_explain( |
| 67 | + &mut self, |
| 68 | + sql: &str, |
| 69 | + ) -> Result<DBOutput<DefaultColumnType>, datafusion::error::DataFusionError> { |
| 70 | + let query = sql.trim_start_matches("EXPLAIN").trim(); |
| 71 | + let df = self.ctx.sql(query).await?; |
| 72 | + let physical_plan = df.create_physical_plan().await?; |
| 73 | + |
| 74 | + // Apply distributed optimizer to get the distributed plan |
| 75 | + use crate::DistributedPhysicalOptimizerRule; |
| 76 | + use datafusion::physical_optimizer::PhysicalOptimizerRule; |
| 77 | + use datafusion::physical_plan::displayable; |
| 78 | + |
| 79 | + let physical_distributed = DistributedPhysicalOptimizerRule::default() |
| 80 | + .with_network_shuffle_tasks(2) |
| 81 | + .with_network_coalesce_tasks(2) |
| 82 | + .optimize(physical_plan, &Default::default())?; |
| 83 | + |
| 84 | + let physical_distributed_str = displayable(physical_distributed.as_ref()) |
| 85 | + .indent(true) |
| 86 | + .to_string(); |
| 87 | + |
| 88 | + // Create a RecordBatch with the plan string |
| 89 | + use datafusion::arrow::array::{ArrayRef, StringArray}; |
| 90 | + use datafusion::arrow::datatypes::{DataType, Field, Schema}; |
| 91 | + |
| 92 | + let lines: Vec<String> = physical_distributed_str |
| 93 | + .lines() |
| 94 | + .map(|s| s.to_string()) |
| 95 | + .collect(); |
| 96 | + let schema = Arc::new(Schema::new(vec![Field::new("plan", DataType::Utf8, false)])); |
| 97 | + let batch = |
| 98 | + RecordBatch::try_new(schema, vec![Arc::new(StringArray::from(lines)) as ArrayRef])?; |
| 99 | + |
| 100 | + self.convert_batches_to_output(vec![batch]) |
| 101 | + } |
| 102 | +} |
| 103 | + |
| 104 | +#[async_trait] |
| 105 | +impl AsyncDB for DatafusionDistributedDB { |
| 106 | + type Error = datafusion::error::DataFusionError; |
| 107 | + type ColumnType = DefaultColumnType; |
| 108 | + |
| 109 | + async fn run(&mut self, sql: &str) -> Result<DBOutput<Self::ColumnType>, Self::Error> { |
| 110 | + let sql = sql.trim(); |
| 111 | + |
| 112 | + // Handle different types of SQL statements |
| 113 | + if sql.to_uppercase().starts_with("CREATE") |
| 114 | + || sql.to_uppercase().starts_with("INSERT") |
| 115 | + || sql.to_uppercase().starts_with("DROP") |
| 116 | + { |
| 117 | + // For DDL/DML statements, just return an empty result |
| 118 | + return Ok(DBOutput::StatementComplete(0)); |
| 119 | + } |
| 120 | + |
| 121 | + // Handle EXPLAIN ANALYZE |
| 122 | + if sql.to_uppercase().starts_with("EXPLAIN ANALYZE") { |
| 123 | + return self.handle_explain_analyze(sql).await; |
| 124 | + } |
| 125 | + |
| 126 | + // Handle regular EXPLAIN - use distributed optimizer |
| 127 | + if sql.to_uppercase().starts_with("EXPLAIN") { |
| 128 | + return self.handle_explain(sql).await; |
| 129 | + } |
| 130 | + |
| 131 | + let df = self.ctx.sql(sql).await?; |
| 132 | + let batches = df.collect().await?; |
| 133 | + |
| 134 | + self.convert_batches_to_output(batches) |
| 135 | + } |
| 136 | + |
| 137 | + fn engine_name(&self) -> &str { |
| 138 | + "datafusion-distributed" |
| 139 | + } |
| 140 | +} |
0 commit comments