pub trait ToStream {
// Required methods
fn logical_rewrite_for_stream(
&self,
ctx: &mut RewriteStreamContext,
) -> Result<(PlanRef, ColIndexMapping), RwError>;
fn to_stream(&self, ctx: &mut ToStreamContext) -> Result<PlanRef, RwError>;
// Provided method
fn to_stream_with_dist_required(
&self,
required_dist: &RequiredDist,
ctx: &mut ToStreamContext,
) -> Result<PlanRef, RwError> { ... }
}
Expand description
ToStream
converts a logical plan node to streaming physical node
with an optional required distribution.
when implement this trait you can choose the two ways
- Implement
to_stream
and use the default implementation ofto_stream_with_dist_required
- Or, if the required distribution is given, there will be a better plan. For example a hash
join with hash-key(a,b) and the plan is required hash-distributed by (a,b,c). you can
implement
to_stream_with_dist_required
, and implementto_stream
withto_stream_with_dist_required(RequiredDist::Any)
. you can seeLogicalProject
as an example.
Required Methods§
sourcefn logical_rewrite_for_stream(
&self,
ctx: &mut RewriteStreamContext,
) -> Result<(PlanRef, ColIndexMapping), RwError>
fn logical_rewrite_for_stream( &self, ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping), RwError>
logical_rewrite_for_stream
will rewrite the logical node, and return (new_plan_node
,
col_mapping
), the col_mapping
is for original columns have been changed into some other
position.
Now it is used to:
- ensure every plan node’s output having pk column
- add
row_count
() in every Agg
Provided Methods§
sourcefn to_stream_with_dist_required(
&self,
required_dist: &RequiredDist,
ctx: &mut ToStreamContext,
) -> Result<PlanRef, RwError>
fn to_stream_with_dist_required( &self, required_dist: &RequiredDist, ctx: &mut ToStreamContext, ) -> Result<PlanRef, RwError>
convert the plan to streaming physical plan and satisfy the required distribution