Enum datafusion_expr::logical_plan::LogicalPlan
source · pub enum LogicalPlan {
Show 27 variants
Projection(Projection),
Filter(Filter),
Window(Window),
Aggregate(Aggregate),
Sort(Sort),
Join(Join),
CrossJoin(CrossJoin),
Repartition(Repartition),
Union(Union),
TableScan(TableScan),
EmptyRelation(EmptyRelation),
Subquery(Subquery),
SubqueryAlias(SubqueryAlias),
Limit(Limit),
Statement(Statement),
Values(Values),
Explain(Explain),
Analyze(Analyze),
Extension(Extension),
Distinct(Distinct),
Prepare(Prepare),
Dml(DmlStatement),
Ddl(DdlStatement),
Copy(CopyTo),
DescribeTable(DescribeTable),
Unnest(Unnest),
RecursiveQuery(RecursiveQuery),
}
Expand description
A LogicalPlan
is a node in a tree of relational operators (such as
Projection or Filter).
Represents transforming an input relation (table) to an output relation (table) with a potentially different schema. Plans form a dataflow tree where data flows from leaves up to the root to produce the query result.
LogicalPlan
s can be created by the SQL query planner, the DataFrame API,
or programmatically (for example custom query languages).
§See also:
Expr
: For the expressions that are evaluated by the planLogicalPlanBuilder
: For buildingLogicalPlan
stree_node
: To inspect and rewriteLogicalPlan
s
§Examples
§Creating a LogicalPlan from SQL:
§Creating a LogicalPlan from the DataFrame API:
§Creating a LogicalPlan programmatically:
§Visiting and Rewriting LogicalPlan
s
Using the tree_node
API, you can recursively walk all nodes in a
LogicalPlan
. For example, to find all column references in a plan:
// Projection(name, salary)
// Filter(salary > 1000)
// TableScan(employee)
let plan = table_scan(Some("employee"), &employee_schema(), None)?
.filter(col("salary").gt(lit(1000)))?
.project(vec![col("name")])?
.build()?;
// use apply to walk the plan and collect all expressions
let mut expressions = HashSet::new();
plan.apply(|node| {
// collect all expressions in the plan
node.apply_expressions(|expr| {
expressions.insert(expr.clone());
Ok(TreeNodeRecursion::Continue) // control walk of expressions
})?;
Ok(TreeNodeRecursion::Continue) // control walk of plan nodes
}).unwrap();
// we found the expression in projection and filter
assert_eq!(expressions.len(), 2);
println!("Found expressions: {:?}", expressions);
// found predicate in the Filter: employee.salary > 1000
let salary = Expr::Column(Column::new(Some("employee"), "salary"));
assert!(expressions.contains(&salary.gt(lit(1000))));
// found projection in the Projection: employee.name
let name = Expr::Column(Column::new(Some("employee"), "name"));
assert!(expressions.contains(&name));
You can also rewrite plans using the tree_node
API. For example, to
replace the filter predicate in a plan:
// Projection(name, salary)
// Filter(salary > 1000)
// TableScan(employee)
use datafusion_common::tree_node::Transformed;
let plan = table_scan(Some("employee"), &employee_schema(), None)?
.filter(col("salary").gt(lit(1000)))?
.project(vec![col("name")])?
.build()?;
// use transform to rewrite the plan
let transformed_result = plan.transform(|node| {
// when we see the filter node
if let LogicalPlan::Filter(mut filter) = node {
// replace predicate with salary < 2000
filter.predicate = Expr::Column(Column::new(Some("employee"), "salary")).lt(lit(2000));
let new_plan = LogicalPlan::Filter(filter);
return Ok(Transformed::yes(new_plan)); // communicate the node was changed
}
// return the node unchanged
Ok(Transformed::no(node))
}).unwrap();
// Transformed result contains rewritten plan and information about
// whether the plan was changed
assert!(transformed_result.transformed);
let rewritten_plan = transformed_result.data;
// we found the filter
assert_eq!(rewritten_plan.display_indent().to_string(),
"Projection: employee.name\
\n Filter: employee.salary < Int32(2000)\
\n TableScan: employee");
Variants§
Projection(Projection)
Evaluates an arbitrary list of expressions (essentially a SELECT with an expression list) on its input.
Filter(Filter)
Filters rows from its input that do not match an expression (essentially a WHERE clause with a predicate expression).
Semantically, <predicate>
is evaluated for each row of the
input; If the value of <predicate>
is true, the input row is
passed to the output. If the value of <predicate>
is false
(or null), the row is discarded.
Window(Window)
Windows input based on a set of window spec and window
function (e.g. SUM or RANK). This is used to implement SQL
window functions, and the OVER
clause.
Aggregate(Aggregate)
Aggregates its input based on a set of grouping and aggregate
expressions (e.g. SUM). This is used to implement SQL aggregates
and GROUP BY
.
Sort(Sort)
Sorts its input according to a list of sort expressions. This
is used to implement SQL ORDER BY
Join(Join)
Join two logical plans on one or more join columns.
This is used to implement SQL JOIN
CrossJoin(CrossJoin)
Apply Cross Join to two logical plans.
This is used to implement SQL CROSS JOIN
Repartition(Repartition)
Repartitions the input based on a partitioning scheme. This is used to add parallelism and is sometimes referred to as an “exchange” operator in other systems
Union(Union)
Union multiple inputs with the same schema into a single
output stream. This is used to implement SQL UNION [ALL]
and
INTERSECT [ALL]
.
TableScan(TableScan)
Produces rows from a TableSource
, used to implement SQL
FROM
tables or views.
EmptyRelation(EmptyRelation)
Produces no rows: An empty relation with an empty schema that
produces 0 or 1 row. This is used to implement SQL SELECT
that has no values in the FROM
clause.
Subquery(Subquery)
Produces the output of running another query. This is used to implement SQL subqueries
SubqueryAlias(SubqueryAlias)
Aliased relation provides, or changes, the name of a relation.
Limit(Limit)
Skip some number of rows, and then fetch some number of rows.
Statement(Statement)
A DataFusion Statement
such as SET VARIABLE
or START TRANSACTION
Values(Values)
Values expression. See
Postgres VALUES
documentation for more details. This is used to implement SQL such as
VALUES (1, 2), (3, 4)
Explain(Explain)
Produces a relation with string representations of
various parts of the plan. This is used to implement SQL EXPLAIN
.
Analyze(Analyze)
Runs the input, and prints annotated physical plan as a string
with execution metric. This is used to implement SQL
EXPLAIN ANALYZE
.
Extension(Extension)
Extension operator defined outside of DataFusion. This is used to extend DataFusion with custom relational operations that
Distinct(Distinct)
Remove duplicate rows from the input. This is used to
implement SQL SELECT DISTINCT ...
.
Prepare(Prepare)
Prepare a statement and find any bind parameters
(e.g. ?
). This is used to implement SQL-prepared statements.
Dml(DmlStatement)
Data Manipulation Language (DML): Insert / Update / Delete
Ddl(DdlStatement)
Data Definition Language (DDL): CREATE / DROP TABLES / VIEWS / SCHEMAS
Copy(CopyTo)
COPY TO
for writing plan results to files
DescribeTable(DescribeTable)
Describe the schema of the table. This is used to implement the
SQL DESCRIBE
command from MySQL.
Unnest(Unnest)
Unnest a column that contains a nested list type such as an
ARRAY. This is used to implement SQL UNNEST
RecursiveQuery(RecursiveQuery)
A variadic query (e.g. “Recursive CTEs”)
Implementations§
source§impl LogicalPlan
impl LogicalPlan
sourcepub fn schema(&self) -> &DFSchemaRef
pub fn schema(&self) -> &DFSchemaRef
Get a reference to the logical plan’s schema
sourcepub fn fallback_normalize_schemas(&self) -> Vec<&DFSchema>
pub fn fallback_normalize_schemas(&self) -> Vec<&DFSchema>
Used for normalizing columns, as the fallback schemas to the main schema of the plan.
sourcepub fn explain_schema() -> SchemaRef
pub fn explain_schema() -> SchemaRef
Returns the (fixed) output schema for explain plans
sourcepub fn describe_schema() -> Schema
pub fn describe_schema() -> Schema
Returns the (fixed) output schema for DESCRIBE
plans
sourcepub fn expressions(self: &LogicalPlan) -> Vec<Expr>
pub fn expressions(self: &LogicalPlan) -> Vec<Expr>
Returns all expressions (non-recursively) evaluated by the current logical plan node. This does not include expressions in any children.
Note this method clone
s all the expressions. When possible, the
tree_node
API should be used instead of this API.
The returned expressions do not necessarily represent or even
contributed to the output schema of this node. For example,
LogicalPlan::Filter
returns the filter expression even though the
output of a Filter has the same columns as the input.
The expressions do contain all the columns that are used by this plan, so if there are columns not referenced by these expressions then DataFusion’s optimizer attempts to optimize them away.
sourcepub fn all_out_ref_exprs(self: &LogicalPlan) -> Vec<Expr>
pub fn all_out_ref_exprs(self: &LogicalPlan) -> Vec<Expr>
Returns all the out reference(correlated) expressions (recursively) in the current logical plan nodes and all its descendant nodes.
pub fn inspect_expressions<F, E>(self: &LogicalPlan, f: F) -> Result<(), E>
apply_expressions
insteadsourcepub fn inputs(&self) -> Vec<&LogicalPlan>
pub fn inputs(&self) -> Vec<&LogicalPlan>
Returns all inputs / children of this LogicalPlan
node.
Note does not include inputs to inputs, or subqueries.
sourcepub fn using_columns(&self) -> Result<Vec<HashSet<Column>>, DataFusionError>
pub fn using_columns(&self) -> Result<Vec<HashSet<Column>>, DataFusionError>
returns all Using
join columns in a logical plan
sourcepub fn head_output_expr(&self) -> Result<Option<Expr>>
pub fn head_output_expr(&self) -> Result<Option<Expr>>
returns the first output expression of this LogicalPlan
node.
sourcepub fn with_new_inputs(&self, inputs: &[LogicalPlan]) -> Result<LogicalPlan>
👎Deprecated since 35.0.0: please use with_new_exprs
instead
pub fn with_new_inputs(&self, inputs: &[LogicalPlan]) -> Result<LogicalPlan>
with_new_exprs
insteadReturns a copy of this LogicalPlan
with the new inputs
sourcepub fn recompute_schema(self) -> Result<Self>
pub fn recompute_schema(self) -> Result<Self>
Recomputes schema and type information for this LogicalPlan if needed.
Some LogicalPlan
s may need to recompute their schema if the number or
type of expressions have been changed (for example due to type
coercion). For example LogicalPlan::Projection
s schema depends on
its expressions.
Some LogicalPlan
s schema is unaffected by any changes to their
expressions. For example LogicalPlan::Filter
schema is always the
same as its input schema.
This is useful after modifying a plans Expr
s (or input plans) via
methods such as Self::map_children and Self::map_expressions. Unlike
Self::with_new_exprs, this method does not require a new set of
expressions or inputs plans.
§Return value
Returns an error if there is some issue recomputing the schema.
§Notes
- Does not recursively recompute schema for input (child) plans.
sourcepub fn with_new_exprs(
&self,
expr: Vec<Expr>,
inputs: Vec<LogicalPlan>,
) -> Result<LogicalPlan>
pub fn with_new_exprs( &self, expr: Vec<Expr>, inputs: Vec<LogicalPlan>, ) -> Result<LogicalPlan>
Returns a new LogicalPlan
based on self
with inputs and
expressions replaced.
Note this method creates an entirely new node, which requires a large
amount of clone’ing. When possible, the tree_node
API should be used
instead of this API.
The exprs correspond to the same order of expressions returned
by Self::expressions
. This function is used by optimizers
to rewrite plans using the following pattern:
let new_inputs = optimize_children(..., plan, props);
// get the plans expressions to optimize
let exprs = plan.expressions();
// potentially rewrite plan expressions
let rewritten_exprs = rewrite_exprs(exprs);
// create new plan using rewritten_exprs in same position
let new_plan = plan.new_with_exprs(rewritten_exprs, new_inputs);
sourcepub fn with_param_values(
self,
param_values: impl Into<ParamValues>,
) -> Result<LogicalPlan>
pub fn with_param_values( self, param_values: impl Into<ParamValues>, ) -> Result<LogicalPlan>
Replaces placeholder param values (like $1
, $2
) in LogicalPlan
with the specified param_values
.
LogicalPlan::Prepare
are
converted to their inner logical plan for execution.
§Example
use datafusion_common::ScalarValue;
// Build SELECT * FROM t1 WHRERE id = $1
let plan = table_scan(Some("t1"), &schema, None).unwrap()
.filter(col("id").eq(placeholder("$1"))).unwrap()
.build().unwrap();
assert_eq!(
"Filter: t1.id = $1\
\n TableScan: t1",
plan.display_indent().to_string()
);
// Fill in the parameter $1 with a literal 3
let plan = plan.with_param_values(vec![
ScalarValue::from(3i32) // value at index 0 --> $1
]).unwrap();
assert_eq!(
"Filter: t1.id = Int32(3)\
\n TableScan: t1",
plan.display_indent().to_string()
);
// Note you can also used named parameters
// Build SELECT * FROM t1 WHRERE id = $my_param
let plan = table_scan(Some("t1"), &schema, None).unwrap()
.filter(col("id").eq(placeholder("$my_param"))).unwrap()
.build().unwrap()
// Fill in the parameter $my_param with a literal 3
.with_param_values(vec![
("my_param", ScalarValue::from(3i32)),
]).unwrap();
assert_eq!(
"Filter: t1.id = Int32(3)\
\n TableScan: t1",
plan.display_indent().to_string()
);
sourcepub fn max_rows(self: &LogicalPlan) -> Option<usize>
pub fn max_rows(self: &LogicalPlan) -> Option<usize>
Returns the maximum number of rows that this plan can output, if known.
If None
, the plan can return any number of rows.
If Some(n)
then the plan can return at most n
rows but may return fewer.
sourcepub fn contains_outer_reference(&self) -> bool
pub fn contains_outer_reference(&self) -> bool
If this node’s expressions contains any references to an outer subquery
source§impl LogicalPlan
impl LogicalPlan
sourcepub fn replace_params_with_values(
self,
param_values: &ParamValues,
) -> Result<LogicalPlan>
pub fn replace_params_with_values( self, param_values: &ParamValues, ) -> Result<LogicalPlan>
Return a LogicalPlan
with all placeholders (e.g $1 $2,
…) replaced with corresponding values provided in
params_values
See Self::with_param_values
for examples and usage with an owned
ParamValues
sourcepub fn get_parameter_types(
&self,
) -> Result<HashMap<String, Option<DataType>>, DataFusionError>
pub fn get_parameter_types( &self, ) -> Result<HashMap<String, Option<DataType>>, DataFusionError>
Walk the logical plan, find any Placeholder
tokens, and return a map of their IDs and DataTypes
sourcepub fn display_indent(&self) -> impl Display + '_
pub fn display_indent(&self) -> impl Display + '_
Return a format
able structure that produces a single line
per node.
§Example
Projection: employee.id
Filter: employee.state Eq Utf8(\"CO\")\
CsvScan: employee projection=Some([0, 3])
use arrow::datatypes::{Field, Schema, DataType};
use datafusion_expr::{lit, col, LogicalPlanBuilder, logical_plan::table_scan};
let schema = Schema::new(vec![
Field::new("id", DataType::Int32, false),
]);
let plan = table_scan(Some("t1"), &schema, None).unwrap()
.filter(col("id").eq(lit(5))).unwrap()
.build().unwrap();
// Format using display_indent
let display_string = format!("{}", plan.display_indent());
assert_eq!("Filter: t1.id = Int32(5)\n TableScan: t1",
display_string);
sourcepub fn display_indent_schema(&self) -> impl Display + '_
pub fn display_indent_schema(&self) -> impl Display + '_
Return a format
able structure that produces a single line
per node that includes the output schema. For example:
Projection: employee.id [id:Int32]\
Filter: employee.state = Utf8(\"CO\") [id:Int32, state:Utf8]\
TableScan: employee projection=[0, 3] [id:Int32, state:Utf8]";
use arrow::datatypes::{Field, Schema, DataType};
use datafusion_expr::{lit, col, LogicalPlanBuilder, logical_plan::table_scan};
let schema = Schema::new(vec![
Field::new("id", DataType::Int32, false),
]);
let plan = table_scan(Some("t1"), &schema, None).unwrap()
.filter(col("id").eq(lit(5))).unwrap()
.build().unwrap();
// Format using display_indent_schema
let display_string = format!("{}", plan.display_indent_schema());
assert_eq!("Filter: t1.id = Int32(5) [id:Int32]\
\n TableScan: t1 [id:Int32]",
display_string);
sourcepub fn display_pg_json(&self) -> impl Display + '_
pub fn display_pg_json(&self) -> impl Display + '_
Return a displayable structure that produces plan in postgresql JSON format.
Users can use this format to visualize the plan in existing plan visualization tools, for example dalibo
sourcepub fn display_graphviz(&self) -> impl Display + '_
pub fn display_graphviz(&self) -> impl Display + '_
Return a format
able structure that produces lines meant for
graphical display using the DOT
language. This format can be
visualized using software from
graphviz
This currently produces two graphs – one with the basic structure, and one with additional details such as schema.
use arrow::datatypes::{Field, Schema, DataType};
use datafusion_expr::{lit, col, LogicalPlanBuilder, logical_plan::table_scan};
let schema = Schema::new(vec![
Field::new("id", DataType::Int32, false),
]);
let plan = table_scan(Some("t1"), &schema, None).unwrap()
.filter(col("id").eq(lit(5))).unwrap()
.build().unwrap();
// Format using display_graphviz
let graphviz_string = format!("{}", plan.display_graphviz());
If graphviz string is saved to a file such as /tmp/example.dot
, the following
commands can be used to render it as a pdf:
dot -Tpdf < /tmp/example.dot > /tmp/example.pdf
sourcepub fn display(&self) -> impl Display + '_
pub fn display(&self) -> impl Display + '_
Return a format
able structure with the a human readable
description of this LogicalPlan node per node, not including
children. For example:
Projection: id
use arrow::datatypes::{Field, Schema, DataType};
use datafusion_expr::{lit, col, LogicalPlanBuilder, logical_plan::table_scan};
let schema = Schema::new(vec![
Field::new("id", DataType::Int32, false),
]);
let plan = table_scan(Some("t1"), &schema, None).unwrap()
.build().unwrap();
// Format using display
let display_string = format!("{}", plan.display());
assert_eq!("TableScan: t1", display_string);
source§impl LogicalPlan
impl LogicalPlan
sourcepub fn apply_expressions<F: FnMut(&Expr) -> Result<TreeNodeRecursion>>(
&self,
f: F,
) -> Result<TreeNodeRecursion>
pub fn apply_expressions<F: FnMut(&Expr) -> Result<TreeNodeRecursion>>( &self, f: F, ) -> Result<TreeNodeRecursion>
Calls f
on all expressions in the current LogicalPlan
node.
§Notes
- Similar to
TreeNode::apply
but for this node’s expressions. - Does not include expressions in input
LogicalPlan
nodes - Visits only the top level expressions (Does not recurse into each expression)
sourcepub fn map_expressions<F: FnMut(Expr) -> Result<Transformed<Expr>>>(
self,
f: F,
) -> Result<Transformed<Self>>
pub fn map_expressions<F: FnMut(Expr) -> Result<Transformed<Expr>>>( self, f: F, ) -> Result<Transformed<Self>>
Rewrites all expressions in the current LogicalPlan
node using f
.
Returns the current node.
§Notes
- Similar to
TreeNode::map_children
but for this node’s expressions. - Visits only the top level expressions (Does not recurse into each expression)
sourcepub fn visit_with_subqueries<V: for<'n> TreeNodeVisitor<'n, Node = Self>>(
&self,
visitor: &mut V,
) -> Result<TreeNodeRecursion>
pub fn visit_with_subqueries<V: for<'n> TreeNodeVisitor<'n, Node = Self>>( &self, visitor: &mut V, ) -> Result<TreeNodeRecursion>
Visits a plan similarly to Self::visit
, including subqueries that
may appear in expressions such as IN (SELECT ...)
.
sourcepub fn rewrite_with_subqueries<R: TreeNodeRewriter<Node = Self>>(
self,
rewriter: &mut R,
) -> Result<Transformed<Self>>
pub fn rewrite_with_subqueries<R: TreeNodeRewriter<Node = Self>>( self, rewriter: &mut R, ) -> Result<Transformed<Self>>
Similarly to Self::rewrite
, rewrites this node and its inputs using f
,
including subqueries that may appear in expressions such as IN (SELECT ...)
.
sourcepub fn apply_with_subqueries<F: FnMut(&Self) -> Result<TreeNodeRecursion>>(
&self,
f: F,
) -> Result<TreeNodeRecursion>
pub fn apply_with_subqueries<F: FnMut(&Self) -> Result<TreeNodeRecursion>>( &self, f: F, ) -> Result<TreeNodeRecursion>
Similarly to Self::apply
, calls f
on this node and all its inputs,
including subqueries that may appear in expressions such as IN (SELECT ...)
.
sourcepub fn transform_with_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>(
self,
f: F,
) -> Result<Transformed<Self>>
pub fn transform_with_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>( self, f: F, ) -> Result<Transformed<Self>>
Similarly to Self::transform
, rewrites this node and its inputs using f
,
including subqueries that may appear in expressions such as IN (SELECT ...)
.
sourcepub fn transform_down_with_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>(
self,
f: F,
) -> Result<Transformed<Self>>
pub fn transform_down_with_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>( self, f: F, ) -> Result<Transformed<Self>>
Similarly to Self::transform_down
, rewrites this node and its inputs using f
,
including subqueries that may appear in expressions such as IN (SELECT ...)
.
sourcepub fn transform_up_with_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>(
self,
f: F,
) -> Result<Transformed<Self>>
pub fn transform_up_with_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>( self, f: F, ) -> Result<Transformed<Self>>
Similarly to Self::transform_up
, rewrites this node and its inputs using f
,
including subqueries that may appear in expressions such as IN (SELECT ...)
.
sourcepub fn transform_down_up_with_subqueries<FD: FnMut(Self) -> Result<Transformed<Self>>, FU: FnMut(Self) -> Result<Transformed<Self>>>(
self,
f_down: FD,
f_up: FU,
) -> Result<Transformed<Self>>
pub fn transform_down_up_with_subqueries<FD: FnMut(Self) -> Result<Transformed<Self>>, FU: FnMut(Self) -> Result<Transformed<Self>>>( self, f_down: FD, f_up: FU, ) -> Result<Transformed<Self>>
Similarly to Self::transform_down
, rewrites this node and its inputs using f
,
including subqueries that may appear in expressions such as IN (SELECT ...)
.
sourcepub fn apply_subqueries<F: FnMut(&Self) -> Result<TreeNodeRecursion>>(
&self,
f: F,
) -> Result<TreeNodeRecursion>
pub fn apply_subqueries<F: FnMut(&Self) -> Result<TreeNodeRecursion>>( &self, f: F, ) -> Result<TreeNodeRecursion>
Similarly to Self::apply
, calls f
on this node and its inputs
including subqueries that may appear in expressions such as IN (SELECT ...)
.
sourcepub fn map_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>(
self,
f: F,
) -> Result<Transformed<Self>>
pub fn map_subqueries<F: FnMut(Self) -> Result<Transformed<Self>>>( self, f: F, ) -> Result<Transformed<Self>>
Similarly to Self::map_children
, rewrites all subqueries that may
appear in expressions such as IN (SELECT ...)
using f
.
Returns the current node.
Trait Implementations§
source§impl Clone for LogicalPlan
impl Clone for LogicalPlan
source§fn clone(&self) -> LogicalPlan
fn clone(&self) -> LogicalPlan
1.0.0 · source§fn clone_from(&mut self, source: &Self)
fn clone_from(&mut self, source: &Self)
source
. Read moresource§impl Debug for LogicalPlan
impl Debug for LogicalPlan
source§impl Default for LogicalPlan
impl Default for LogicalPlan
source§impl Display for LogicalPlan
impl Display for LogicalPlan
source§impl Hash for LogicalPlan
impl Hash for LogicalPlan
source§impl PartialEq for LogicalPlan
impl PartialEq for LogicalPlan
source§impl ToStringifiedPlan for LogicalPlan
impl ToStringifiedPlan for LogicalPlan
source§fn to_stringified(&self, plan_type: PlanType) -> StringifiedPlan
fn to_stringified(&self, plan_type: PlanType) -> StringifiedPlan
source§impl TreeNode for LogicalPlan
impl TreeNode for LogicalPlan
source§fn map_children<F: FnMut(Self) -> Result<Transformed<Self>>>(
self,
f: F,
) -> Result<Transformed<Self>>
fn map_children<F: FnMut(Self) -> Result<Transformed<Self>>>( self, f: F, ) -> Result<Transformed<Self>>
Applies f
to each child (input) of this plan node, rewriting them in place.
§Notes
Inputs include ONLY direct children, not embedded LogicalPlan
s for
subqueries, for example such as are in Expr::Exists
.
source§fn apply_children<'n, F: FnMut(&'n Self) -> Result<TreeNodeRecursion>>(
&'n self,
f: F,
) -> Result<TreeNodeRecursion>
fn apply_children<'n, F: FnMut(&'n Self) -> Result<TreeNodeRecursion>>( &'n self, f: F, ) -> Result<TreeNodeRecursion>
source§fn visit<'n, V>(
&'n self,
visitor: &mut V,
) -> Result<TreeNodeRecursion, DataFusionError>where
V: TreeNodeVisitor<'n, Node = Self>,
fn visit<'n, V>(
&'n self,
visitor: &mut V,
) -> Result<TreeNodeRecursion, DataFusionError>where
V: TreeNodeVisitor<'n, Node = Self>,
TreeNodeVisitor
, performing a
depth-first walk of the node and its children. Read moresource§fn rewrite<R>(
self,
rewriter: &mut R,
) -> Result<Transformed<Self>, DataFusionError>where
R: TreeNodeRewriter<Node = Self>,
fn rewrite<R>(
self,
rewriter: &mut R,
) -> Result<Transformed<Self>, DataFusionError>where
R: TreeNodeRewriter<Node = Self>,
TreeNodeRewriter
, performing a
depth-first walk of the node and its children. Read moresource§fn apply<'n, F>(&'n self, f: F) -> Result<TreeNodeRecursion, DataFusionError>
fn apply<'n, F>(&'n self, f: F) -> Result<TreeNodeRecursion, DataFusionError>
f
to the node then each of its children, recursively (a
top-down, pre-order traversal). Read moresource§fn transform<F>(self, f: F) -> Result<Transformed<Self>, DataFusionError>
fn transform<F>(self, f: F) -> Result<Transformed<Self>, DataFusionError>
f
(a bottom-up post-order traversal). Read moresource§fn transform_down<F>(self, f: F) -> Result<Transformed<Self>, DataFusionError>
fn transform_down<F>(self, f: F) -> Result<Transformed<Self>, DataFusionError>
f
in a top-down (pre-order)
fashion. Read moresource§fn transform_down_mut<F>(
self,
f: &mut F,
) -> Result<Transformed<Self>, DataFusionError>
fn transform_down_mut<F>( self, f: &mut F, ) -> Result<Transformed<Self>, DataFusionError>
transform_down
insteadSelf::transform_down
but with a mutable closure.source§fn transform_up<F>(self, f: F) -> Result<Transformed<Self>, DataFusionError>
fn transform_up<F>(self, f: F) -> Result<Transformed<Self>, DataFusionError>
f
in a bottom-up (post-order)
fashion. Read moresource§fn transform_up_mut<F>(
self,
f: &mut F,
) -> Result<Transformed<Self>, DataFusionError>
fn transform_up_mut<F>( self, f: &mut F, ) -> Result<Transformed<Self>, DataFusionError>
transform_up
insteadSelf::transform_up
but with a mutable closure.source§fn transform_down_up<FD, FU>(
self,
f_down: FD,
f_up: FU,
) -> Result<Transformed<Self>, DataFusionError>where
FD: FnMut(Self) -> Result<Transformed<Self>, DataFusionError>,
FU: FnMut(Self) -> Result<Transformed<Self>, DataFusionError>,
fn transform_down_up<FD, FU>(
self,
f_down: FD,
f_up: FU,
) -> Result<Transformed<Self>, DataFusionError>where
FD: FnMut(Self) -> Result<Transformed<Self>, DataFusionError>,
FU: FnMut(Self) -> Result<Transformed<Self>, DataFusionError>,
f_down
while traversing the tree top-down
(pre-order), and using f_up
while traversing the tree bottom-up
(post-order). Read moreimpl Eq for LogicalPlan
impl StructuralPartialEq for LogicalPlan
Auto Trait Implementations§
impl Freeze for LogicalPlan
impl !RefUnwindSafe for LogicalPlan
impl Send for LogicalPlan
impl Sync for LogicalPlan
impl Unpin for LogicalPlan
impl !UnwindSafe for LogicalPlan
Blanket Implementations§
source§impl<T> BorrowMut<T> for Twhere
T: ?Sized,
impl<T> BorrowMut<T> for Twhere
T: ?Sized,
source§fn borrow_mut(&mut self) -> &mut T
fn borrow_mut(&mut self) -> &mut T
source§impl<T> CloneToUninit for Twhere
T: Clone,
impl<T> CloneToUninit for Twhere
T: Clone,
source§default unsafe fn clone_to_uninit(&self, dst: *mut T)
default unsafe fn clone_to_uninit(&self, dst: *mut T)
clone_to_uninit
)source§impl<Q, K> Equivalent<K> for Q
impl<Q, K> Equivalent<K> for Q
source§impl<Q, K> Equivalent<K> for Q
impl<Q, K> Equivalent<K> for Q
source§fn equivalent(&self, key: &K) -> bool
fn equivalent(&self, key: &K) -> bool
key
and return true
if they are equal.