-
Notifications
You must be signed in to change notification settings - Fork 32
Expand file tree
/
Copy pathcustom_config_extension.rs
More file actions
236 lines (206 loc) · 8.03 KB
/
custom_config_extension.rs
File metadata and controls
236 lines (206 loc) · 8.03 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
#[cfg(all(feature = "integration", test))]
mod tests {
use datafusion::common::tree_node::{Transformed, TreeNode};
use datafusion::common::{extensions_options, internal_datafusion_err, internal_err};
use datafusion::config::ConfigExtension;
use datafusion::error::DataFusionError;
use datafusion::execution::{SendableRecordBatchStream, SessionState, TaskContext};
use datafusion::physical_expr::EquivalenceProperties;
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::{
DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties,
execute_stream,
};
use datafusion_distributed::test_utils::localhost::start_localhost_context;
use datafusion_distributed::test_utils::parquet::register_parquet_tables;
use datafusion_distributed::{DistributedExt, WorkerQueryContext};
use datafusion_proto::physical_plan::PhysicalExtensionCodec;
use futures::TryStreamExt;
use prost::Message;
use std::any::Any;
use std::fmt::Formatter;
use std::sync::Arc;
async fn build_state(ctx: WorkerQueryContext) -> Result<SessionState, DataFusionError> {
Ok(ctx
.builder
.with_distributed_option_extension_from_headers::<CustomExtension>(&ctx.headers)?
.with_distributed_user_codec(CustomConfigExtensionRequiredExecCodec)
.build())
}
#[tokio::test]
async fn custom_config_extension() -> Result<(), Box<dyn std::error::Error>> {
let (mut ctx, _guard, _) = start_localhost_context(3, build_state).await;
ctx.set_distributed_user_codec(CustomConfigExtensionRequiredExecCodec);
ctx.set_distributed_option_extension(CustomExtension {
foo: "foo".to_string(),
bar: 1,
baz: true,
throw_err: false,
});
let query = r#"SELECT "MinTemp" FROM weather WHERE "MinTemp" > 20.0"#;
register_parquet_tables(&ctx).await?;
let df = ctx.sql(query).await?;
let plan = df.create_physical_plan().await?;
// Wrap leaf nodes with CustomConfigExtensionRequiredExec to test config extension propagation
let transformed = plan.transform_up(|plan| {
if plan.children().is_empty() {
return Ok(Transformed::yes(Arc::new(
CustomConfigExtensionRequiredExec::new(plan),
)));
}
Ok(Transformed::no(plan))
})?;
let plan = transformed.data;
let stream = execute_stream(plan, ctx.task_ctx())?;
// It should not fail.
stream.try_collect::<Vec<_>>().await?;
Ok(())
}
#[tokio::test]
async fn custom_config_extension_runtime_change() -> Result<(), Box<dyn std::error::Error>> {
let (mut ctx, _guard, _) = start_localhost_context(3, build_state).await;
ctx.set_distributed_user_codec(CustomConfigExtensionRequiredExecCodec);
ctx.set_distributed_option_extension(CustomExtension {
throw_err: true,
..Default::default()
});
let query = r#"SELECT "MinTemp" FROM weather WHERE "MinTemp" > 20.0"#;
register_parquet_tables(&ctx).await?;
let df = ctx.sql(query).await?;
let plan = df.create_physical_plan().await?;
// Wrap leaf nodes with CustomConfigExtensionRequiredExec to test config extension propagation
let transformed = plan.transform_up(|plan| {
if plan.children().is_empty() {
return Ok(Transformed::yes(Arc::new(
CustomConfigExtensionRequiredExec::new(plan),
)));
}
Ok(Transformed::no(plan))
})?;
let plan = transformed.data;
// If the value is modified after setting it as a distributed option extension, it should
// propagate the correct headers.
ctx.state_ref()
.write()
.config_mut()
.options_mut()
.extensions
.get_mut::<CustomExtension>()
.unwrap()
.throw_err = false;
let stream = execute_stream(plan, ctx.task_ctx())?;
// It should not fail.
stream.try_collect::<Vec<_>>().await?;
Ok(())
}
extensions_options! {
pub struct CustomExtension {
pub foo: String, default = "".to_string()
pub bar: usize, default = 0
pub baz: bool, default = false
pub throw_err: bool, default = true
}
}
impl ConfigExtension for CustomExtension {
const PREFIX: &'static str = "custom";
}
#[derive(Debug)]
pub struct CustomConfigExtensionRequiredExec {
plan_properties: PlanProperties,
child: Arc<dyn ExecutionPlan>,
}
impl CustomConfigExtensionRequiredExec {
fn new(child: Arc<dyn ExecutionPlan>) -> Self {
let plan_properties = PlanProperties::new(
EquivalenceProperties::new(child.schema()),
child.output_partitioning().clone(),
EmissionType::Incremental,
Boundedness::Bounded,
);
Self {
plan_properties,
child,
}
}
}
impl DisplayAs for CustomConfigExtensionRequiredExec {
fn fmt_as(&self, _: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result {
write!(f, "CustomConfigExtensionRequiredExec")
}
}
impl ExecutionPlan for CustomConfigExtensionRequiredExec {
fn name(&self) -> &str {
"CustomConfigExtensionRequiredExec"
}
fn as_any(&self) -> &dyn Any {
self
}
fn properties(&self) -> &PlanProperties {
&self.plan_properties
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![&self.child]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> datafusion::common::Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(CustomConfigExtensionRequiredExec::new(
children[0].clone(),
)))
}
fn execute(
&self,
partition: usize,
ctx: Arc<TaskContext>,
) -> datafusion::common::Result<SendableRecordBatchStream> {
let Some(ext) = ctx
.session_config()
.options()
.extensions
.get::<CustomExtension>()
else {
return internal_err!("CustomExtension not found in context");
};
if ext.throw_err {
return internal_err!("CustomExtension requested an error to be thrown");
}
// Pass through to child
self.child.execute(partition, ctx)
}
}
#[derive(Debug)]
struct CustomConfigExtensionRequiredExecCodec;
#[derive(Clone, PartialEq, ::prost::Message)]
struct CustomConfigExtensionRequiredExecProto {}
impl PhysicalExtensionCodec for CustomConfigExtensionRequiredExecCodec {
fn try_decode(
&self,
buf: &[u8],
inputs: &[Arc<dyn ExecutionPlan>],
_ctx: &TaskContext,
) -> datafusion::common::Result<Arc<dyn ExecutionPlan>> {
let _node = CustomConfigExtensionRequiredExecProto::decode(buf)
.map_err(|err| internal_datafusion_err!("{err}"))?;
if inputs.len() != 1 {
return internal_err!(
"CustomConfigExtensionRequiredExec expects exactly one child, got {}",
inputs.len()
);
}
Ok(Arc::new(CustomConfigExtensionRequiredExec::new(
inputs[0].clone(),
)))
}
fn try_encode(
&self,
_node: Arc<dyn ExecutionPlan>,
buf: &mut Vec<u8>,
) -> datafusion::common::Result<()> {
CustomConfigExtensionRequiredExecProto::default()
.encode(buf)
.unwrap();
Ok(())
}
}
}