-
Notifications
You must be signed in to change notification settings - Fork 40
Expand file tree
/
Copy pathcustom_execution_plan.rs
More file actions
408 lines (358 loc) · 13.8 KB
/
custom_execution_plan.rs
File metadata and controls
408 lines (358 loc) · 13.8 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
//! This example demonstrates how to create a custom execution plan that works with
//! Distributed DataFusion. It implements a `numbers(start, end)` table function that
//! generates a sequence of numbers and can be distributed across multiple workers.
//!
//! This example includes:
//! - Custom TableFunction for accepting the `numbers(start, end)` in SQL
//! - Custom TableProvider for mapping the table function to an execution plan
//! - Custom ExecutionPlan for returning the requested number range
//! - Custom PhysicalExtensionCodec for serialization across the network
//! - Custom TaskEstimator to control parallelism
//!
//! Run this example with:
//! ```bash
//! cargo run --features integration --example custom_execution_plan "SELECT DISTINCT number FROM numbers(0, 10) ORDER BY number" --show-distributed-plan
//! cargo run --features integration --example custom_execution_plan "SELECT DISTINCT number FROM numbers(0, 11) ORDER BY number" --show-distributed-plan
//! cargo run --features integration --example custom_execution_plan "SELECT DISTINCT number FROM numbers(0, 100) ORDER BY number" --show-distributed-plan
//! cargo run --features integration --example custom_execution_plan "SELECT DISTINCT number FROM numbers(0, 100) ORDER BY number" --workers 10 --show-distributed-plan
//! ```
use arrow::array::{ArrayRef, Int64Array, RecordBatch};
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use arrow::record_batch::RecordBatchOptions;
use arrow::util::pretty::pretty_format_batches;
use async_trait::async_trait;
use datafusion::catalog::{Session, TableFunctionImpl};
use datafusion::common::{
DataFusionError, Result, ScalarValue, exec_err, extensions_options, internal_err, plan_err,
};
use datafusion::config::ConfigExtension;
use datafusion::datasource::{TableProvider, TableType};
use datafusion::execution::{SendableRecordBatchStream, SessionStateBuilder, TaskContext};
use datafusion::logical_expr::Expr;
use datafusion::physical_expr::EquivalenceProperties;
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties};
use datafusion::prelude::{SessionConfig, SessionContext};
use datafusion_distributed::test_utils::in_memory_channel_resolver::{
InMemoryChannelResolver, InMemoryWorkerResolver,
};
use datafusion_distributed::{
DistributedExt, DistributedTaskContext, SessionStateBuilderExt, TaskEstimation, TaskEstimator,
WorkerQueryContext, display_plan_ascii,
};
use datafusion_proto::physical_plan::PhysicalExtensionCodec;
use datafusion_proto::protobuf;
use datafusion_proto::protobuf::proto_error;
use futures::{TryStreamExt, stream};
use prost::Message;
use std::any::Any;
use std::fmt::{self, Formatter};
use std::ops::Range;
use std::sync::Arc;
use structopt::StructOpt;
/// Table function that generates a sequence of numbers from start to end.
/// Can be called in SQL as: SELECT * FROM numbers(start, end)
#[derive(Debug)]
struct NumbersTableFunction;
impl TableFunctionImpl for NumbersTableFunction {
fn call(&self, exprs: &[Expr]) -> Result<Arc<dyn TableProvider>> {
if exprs.len() != 2 {
return plan_err!(
"numbers() requires exactly 2 arguments (start, end), got {}",
exprs.len()
);
}
fn get_number(expr: &Expr) -> Result<i64, DataFusionError> {
match &expr {
Expr::Literal(ScalarValue::Int64(Some(v)), _) => Ok(*v),
Expr::Literal(ScalarValue::Int32(Some(v)), _) => Ok(*v as i64),
v => plan_err!("numbers() arguments must be integer literals, got {v:?}"),
}
}
Ok(Arc::new(NumbersTableProvider {
start: get_number(&exprs[0])?,
end: get_number(&exprs[1])?,
}))
}
}
fn numbers_schema() -> SchemaRef {
Arc::new(Schema::new(vec![Field::new(
"number",
DataType::Int64,
false,
)]))
}
/// TableProvider that generates a sequence of numbers from start to end.
#[derive(Debug)]
struct NumbersTableProvider {
start: i64,
end: i64,
}
#[async_trait]
impl TableProvider for NumbersTableProvider {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
numbers_schema()
}
fn table_type(&self) -> TableType {
TableType::Base
}
async fn scan(
&self,
_state: &dyn Session,
projection: Option<&Vec<usize>>,
_filters: &[Expr],
_limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
let schema = match projection {
Some(indices) => Arc::new(self.schema().project(indices)?),
None => self.schema(),
};
#[allow(clippy::single_range_in_vec_init)]
Ok(Arc::new(NumbersExec::new([self.start..self.end], schema)))
}
}
/// Custom execution plan that generates numbers from start to end.
/// When distributed, each task generates a subset of numbers based on its task_index.
#[derive(Debug, Clone)]
struct NumbersExec {
ranges_per_task: Vec<Range<i64>>,
plan_properties: Arc<PlanProperties>,
}
impl NumbersExec {
fn new(ranges_per_task: impl IntoIterator<Item = Range<i64>>, schema: SchemaRef) -> Self {
let plan_properties = Arc::new(PlanProperties::new(
EquivalenceProperties::new(schema.clone()),
datafusion::physical_expr::Partitioning::UnknownPartitioning(1),
EmissionType::Incremental,
Boundedness::Bounded,
));
Self {
ranges_per_task: ranges_per_task.into_iter().collect(),
plan_properties,
}
}
}
impl DisplayAs for NumbersExec {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> fmt::Result {
write!(f, "NumbersExec: ")?;
for (task_i, range) in self.ranges_per_task.iter().enumerate() {
write!(f, "t{task_i}:[{}-{})", range.start, range.end)?;
if task_i < self.ranges_per_task.len() - 1 {
write!(f, ", ")?;
}
}
Ok(())
}
}
impl ExecutionPlan for NumbersExec {
fn name(&self) -> &str {
"NumbersExec"
}
fn as_any(&self) -> &dyn Any {
self
}
fn properties(&self) -> &Arc<PlanProperties> {
&self.plan_properties
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![]
}
fn with_new_children(
self: Arc<Self>,
_: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(self)
}
fn execute(
&self,
_partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
// Get the distributed task context to determine which subset of numbers
// this task should generate
let dist_ctx = DistributedTaskContext::from_ctx(&context);
let Some(range) = self.ranges_per_task.get(dist_ctx.task_index) else {
return exec_err!("Task index out of range");
};
// Calculate which numbers this task should generate
let numbers: Vec<i64> = range.clone().collect();
let row_count = numbers.len();
// Create batch matching the schema (may be empty for COUNT queries)
let batch = if self.schema().fields().is_empty() {
// For COUNT queries, return batch with correct row count but no columns
let mut options = RecordBatchOptions::new();
options.row_count = Some(row_count);
RecordBatch::try_new_with_options(self.schema(), vec![], &options)?
} else {
let array: ArrayRef = Arc::new(Int64Array::from(numbers));
RecordBatch::try_new(self.schema(), vec![array])?
};
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
stream::once(async { Ok(batch) }),
)))
}
}
/// Custom codec for serializing/deserializing NumbersExec across the network. As the NumbersExec
/// plan will be sent over the wire during distributed queries, both the SessionContext that
/// initiates the query and each Worker need to know how to (de)serialize it.
#[derive(Debug)]
struct NumbersExecCodec;
#[derive(Clone, PartialEq, ::prost::Message)]
struct NumbersExecProto {
#[prost(message, optional, tag = "1")]
schema: Option<protobuf::Schema>,
#[prost(repeated, message, tag = "2")]
ranges: Vec<RangeProto>,
}
#[derive(Clone, PartialEq, ::prost::Message)]
struct RangeProto {
#[prost(int64, tag = "1")]
start: i64,
#[prost(int64, tag = "2")]
end: i64,
}
impl PhysicalExtensionCodec for NumbersExecCodec {
fn try_decode(
&self,
buf: &[u8],
inputs: &[Arc<dyn ExecutionPlan>],
_ctx: &TaskContext,
) -> Result<Arc<dyn ExecutionPlan>> {
if !inputs.is_empty() {
return internal_err!("NumbersExec should have no children, got {}", inputs.len());
}
let proto = NumbersExecProto::decode(buf)
.map_err(|e| proto_error(format!("Failed to decode NumbersExec: {e}")))?;
let schema: Schema = proto
.schema
.as_ref()
.map(|s| s.try_into())
.ok_or(proto_error("NetworkShuffleExec is missing schema"))??;
Ok(Arc::new(NumbersExec::new(
proto.ranges.iter().map(|v| v.start..v.end),
Arc::new(schema),
)))
}
fn try_encode(&self, node: Arc<dyn ExecutionPlan>, buf: &mut Vec<u8>) -> Result<()> {
let Some(exec) = node.as_any().downcast_ref::<NumbersExec>() else {
return internal_err!("Expected plan to be NumbersExec, but was {}", node.name());
};
let proto = NumbersExecProto {
schema: Some(node.schema().try_into()?),
ranges: exec
.ranges_per_task
.iter()
.map(|v| RangeProto {
start: v.start,
end: v.end,
})
.collect(),
};
proto
.encode(buf)
.map_err(|e| proto_error(format!("Failed to encode NumbersExec: {e}")))
}
}
extensions_options! {
/// Custom ConfigExtension for configuring NumbersExec distributed task estimation behavior
/// at runtime with SET statements.
struct NumbersConfig {
/// how many numbers each task will produce
numbers_per_task: usize, default = 10
}
}
impl ConfigExtension for NumbersConfig {
const PREFIX: &'static str = "numbers";
}
/// Custom TaskEstimator that tells the planner how to distribute NumbersExec.
#[derive(Debug)]
struct NumbersTaskEstimator;
impl TaskEstimator for NumbersTaskEstimator {
fn task_estimation(
&self,
plan: &Arc<dyn ExecutionPlan>,
cfg: &datafusion::config::ConfigOptions,
) -> Option<TaskEstimation> {
let plan = plan.as_any().downcast_ref::<NumbersExec>()?;
let cfg: &NumbersConfig = cfg.extensions.get()?;
let task_count = (plan.ranges_per_task[0].end - plan.ranges_per_task[0].start) as f64
/ cfg.numbers_per_task as f64;
Some(TaskEstimation::desired(task_count.ceil() as usize))
}
fn scale_up_leaf_node(
&self,
plan: &Arc<dyn ExecutionPlan>,
task_count: usize,
_cfg: &datafusion::config::ConfigOptions,
) -> Option<Arc<dyn ExecutionPlan>> {
let plan = plan.as_any().downcast_ref::<NumbersExec>()?;
let range = &plan.ranges_per_task[0];
let chunk_size = ((range.end - range.start) as f64 / task_count as f64).ceil() as i64;
let ranges_per_task = (0..task_count).map(|i| {
let start = range.start + (i as i64 * chunk_size);
let end = (start + chunk_size).min(range.end);
start..end
});
Some(Arc::new(NumbersExec::new(ranges_per_task, plan.schema())))
}
}
#[derive(StructOpt)]
#[structopt(
name = "custom_execution_plan",
about = "Example demonstrating custom execution plans with Distributed DataFusion"
)]
struct Args {
/// The SQL query to run.
#[structopt()]
query: String,
/// Number of distributed workers to simulate.
#[structopt(long, default_value = "4")]
workers: usize,
/// Whether the distributed plan should be rendered instead of executing the query.
#[structopt(long)]
show_distributed_plan: bool,
}
#[tokio::main]
async fn main() -> Result<(), Box<dyn std::error::Error>> {
let args = Args::from_args();
let worker_resolver = InMemoryWorkerResolver::new(args.workers);
let channel_resolver =
InMemoryChannelResolver::from_session_builder(|ctx: WorkerQueryContext| async move {
Ok(ctx
.builder
.with_distributed_user_codec(NumbersExecCodec)
.build())
});
let config = SessionConfig::new().with_option_extension(NumbersConfig::default());
let state = SessionStateBuilder::new()
.with_default_features()
.with_config(config)
.with_distributed_worker_resolver(worker_resolver)
.with_distributed_channel_resolver(channel_resolver)
.with_distributed_planner()
.with_distributed_user_codec(NumbersExecCodec)
.with_distributed_task_estimator(NumbersTaskEstimator)
.build();
let ctx = SessionContext::from(state);
ctx.register_udtf("numbers", Arc::new(NumbersTableFunction));
let mut df = None;
for query in args.query.split(';') {
df = Some(ctx.sql(query).await?);
}
let df = df.unwrap();
if args.show_distributed_plan {
let plan = df.create_physical_plan().await?;
println!("{}", display_plan_ascii(plan.as_ref(), false));
} else {
let stream = df.execute_stream().await?;
let batches = stream.try_collect::<Vec<_>>().await?;
let formatted = pretty_format_batches(&batches)?;
println!("{formatted}");
}
Ok(())
}