forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathplanner_api.rs
143 lines (129 loc) · 4.97 KB
/
planner_api.rs
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
// 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 datafusion::error::Result;
use datafusion::physical_plan::displayable;
use datafusion::physical_planner::DefaultPhysicalPlanner;
use datafusion::prelude::*;
use datafusion_expr::{LogicalPlan, PlanType};
/// This example demonstrates the process of converting logical plan
/// into physical execution plans using DataFusion.
///
/// Planning phase in DataFusion contains several steps:
/// 1. Analyzing and optimizing logical plan
/// 2. Converting logical plan into physical plan
///
/// The code in this example shows two ways to convert a logical plan into
/// physical plan:
/// - Via the combined `create_physical_plan` API.
/// - Utilizing the analyzer, optimizer, and query planner APIs separately.
#[tokio::main]
async fn main() -> Result<()> {
// Set up a DataFusion context and load a Parquet file
let ctx = SessionContext::new();
let testdata = datafusion::test_util::parquet_test_data();
let df = ctx
.read_parquet(
&format!("{testdata}/alltypes_plain.parquet"),
ParquetReadOptions::default(),
)
.await?;
// Construct the input logical plan using DataFrame API
let df = df
.clone()
.select(vec![
df.parse_sql_expr("int_col")?,
df.parse_sql_expr("double_col")?,
])?
.filter(df.parse_sql_expr("int_col < 5 OR double_col = 8.0")?)?
.aggregate(
vec![df.parse_sql_expr("double_col")?],
vec![df.parse_sql_expr("SUM(int_col) as sum_int_col")?],
)?
.limit(0, Some(1))?;
let logical_plan = df.logical_plan().clone();
to_physical_plan_in_one_api_demo(&logical_plan, &ctx).await?;
to_physical_plan_step_by_step_demo(logical_plan, &ctx).await?;
Ok(())
}
/// Converts a logical plan into a physical plan using the combined
/// `create_physical_plan` API. It will first optimize the logical
/// plan and then convert it into physical plan.
async fn to_physical_plan_in_one_api_demo(
input: &LogicalPlan,
ctx: &SessionContext,
) -> Result<()> {
let physical_plan = ctx.state().create_physical_plan(input).await?;
println!(
"Physical plan direct from logical plan:\n\n{}\n\n",
displayable(physical_plan.as_ref())
.to_stringified(false, PlanType::InitialPhysicalPlan)
.plan
);
Ok(())
}
/// Converts a logical plan into a physical plan by utilizing the analyzer,
/// optimizer, and query planner APIs separately. This flavor gives more
/// control over the planning process.
async fn to_physical_plan_step_by_step_demo(
input: LogicalPlan,
ctx: &SessionContext,
) -> Result<()> {
// First analyze the logical plan
let analyzed_logical_plan = ctx.state().analyzer().execute_and_check(
input,
ctx.state().config_options(),
|_, _| (),
)?;
println!("Analyzed logical plan:\n\n{:?}\n\n", analyzed_logical_plan);
// Optimize the analyzed logical plan
let optimized_logical_plan = ctx.state().optimizer().optimize(
analyzed_logical_plan,
&ctx.state(),
|_, _| (),
)?;
println!(
"Optimized logical plan:\n\n{:?}\n\n",
optimized_logical_plan
);
// Create the physical plan
let physical_plan = ctx
.state()
.query_planner()
.create_physical_plan(&optimized_logical_plan, &ctx.state())
.await?;
println!(
"Final physical plan:\n\n{}\n\n",
displayable(physical_plan.as_ref())
.to_stringified(false, PlanType::InitialPhysicalPlan)
.plan
);
// Call the physical optimizer with an existing physical plan (in this
// case the plan is already optimized, but an unoptimized plan would
// typically be used in this context)
// Note that this is not part of the trait but a public method
// on DefaultPhysicalPlanner. Not all planners will provide this feature.
let planner = DefaultPhysicalPlanner::default();
let physical_plan =
planner.optimize_physical_plan(physical_plan, &ctx.state(), |_, _| {})?;
println!(
"Optimized physical plan:\n\n{}\n\n",
displayable(physical_plan.as_ref())
.to_stringified(false, PlanType::InitialPhysicalPlan)
.plan
);
Ok(())
}