Skip to content

Commit

Permalink
using as_ref() instead of directly cloning it
Browse files Browse the repository at this point in the history
  • Loading branch information
jatin510 committed Nov 2, 2024
1 parent b907ccc commit 0c2e077
Show file tree
Hide file tree
Showing 7 changed files with 13 additions and 9 deletions.
4 changes: 4 additions & 0 deletions datafusion/physical-expr-common/src/sort_expr.rs
Original file line number Diff line number Diff line change
Expand Up @@ -356,6 +356,10 @@ impl LexOrdering {
self.inner.capacity()
}

pub fn from_ref(lex_ordering_ref: &LexOrdering) -> Self {
Self::new(lex_ordering_ref.to_vec())
}

pub fn clear(&mut self) {
self.inner.clear()
}
Expand Down
2 changes: 1 addition & 1 deletion datafusion/physical-expr/src/window/aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ impl PlainAggregateWindowExpr {
Self {
aggregate,
partition_by: partition_by.to_vec(),
order_by: order_by.clone(),
order_by: LexOrdering::from_ref(order_by),
window_frame,
}
}
Expand Down
2 changes: 1 addition & 1 deletion datafusion/physical-expr/src/window/built_in.rs
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ impl BuiltInWindowExpr {
Self {
expr,
partition_by: partition_by.to_vec(),
order_by: order_by.clone(),
order_by: LexOrdering::from_ref(order_by),
window_frame,
}
}
Expand Down
2 changes: 1 addition & 1 deletion datafusion/physical-expr/src/window/sliding_aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ impl SlidingAggregateWindowExpr {
Self {
aggregate,
partition_by: partition_by.to_vec(),
order_by: order_by.clone(),
order_by: LexOrdering::from_ref(order_by),
window_frame,
}
}
Expand Down
8 changes: 4 additions & 4 deletions datafusion/physical-plan/src/joins/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -483,7 +483,7 @@ fn offset_ordering(
options: sort_expr.options,
})
.collect(),
_ => ordering.clone(),
_ => LexOrdering::from_ref(ordering),
}
}

Expand All @@ -503,23 +503,23 @@ pub fn calculate_join_output_ordering(
if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) {
replace_on_columns_of_right_ordering(
on_columns,
&mut right_ordering.clone(),
&mut LexOrdering::from_ref(right_ordering),
)
.ok()?;
merge_vectors(
left_ordering,
&offset_ordering(right_ordering, &join_type, left_columns_len),
)
} else {
left_ordering.clone()
LexOrdering::from_ref(left_ordering)
}
}
[false, true] => {
// Special case, we can prefix ordering of left side with the ordering of right side.
if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) {
replace_on_columns_of_right_ordering(
on_columns,
&mut right_ordering.clone(),
&mut LexOrdering::from_ref(right_ordering),
)
.ok()?;
merge_vectors(
Expand Down
2 changes: 1 addition & 1 deletion datafusion/physical-plan/src/repartition/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -503,7 +503,7 @@ impl DisplayAs for RepartitionExec {
}

if let Some(sort_exprs) = self.sort_exprs() {
write!(f, ", sort_exprs={sort_exprs}")?;
write!(f, ", sort_exprs={}", LexOrdering::from_ref(sort_exprs))?;
}
Ok(())
}
Expand Down
2 changes: 1 addition & 1 deletion datafusion/proto/src/physical_plan/to_proto.rs
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ pub fn serialize_physical_aggr_expr(
) -> Result<protobuf::PhysicalExprNode> {
let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?;
let ordering_req = match aggr_expr.order_bys() {
Some(order) => order.clone(),
Some(order) => LexOrdering::from_ref(order),
None => LexOrdering::default(),
};
let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?;
Expand Down

0 comments on commit 0c2e077

Please sign in to comment.