Skip to content

Commit

Permalink
change function declaration
Browse files Browse the repository at this point in the history
  • Loading branch information
st1page committed Sep 25, 2023
1 parent fd4f89f commit ea3ea6b
Show file tree
Hide file tree
Showing 3 changed files with 10 additions and 10 deletions.
2 changes: 1 addition & 1 deletion src/frontend/src/optimizer/plan_node/generic/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ pub(super) use impl_distill_unit_from_fields;

pub trait GenericPlanRef: Eq + Hash {
fn schema(&self) -> &Schema;
fn stream_key(&self) -> &[usize];
fn stream_key(&self) -> Option<&[usize]>;
fn functional_dependency(&self) -> &FunctionalDependencySet;
fn ctx(&self) -> OptimizerContextRef;
}
Expand Down
8 changes: 4 additions & 4 deletions src/frontend/src/optimizer/plan_node/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -435,8 +435,8 @@ impl GenericPlanRef for PlanRef {
&self.plan_base().schema
}

fn stream_key(&self) -> &[usize] {
&self.plan_base().stream_key
fn stream_key(&self) -> Option<&[usize]> {
self.plan_base().stream_key()
}

fn ctx(&self) -> OptimizerContextRef {
Expand Down Expand Up @@ -514,8 +514,8 @@ impl dyn PlanNode {
&self.plan_base().schema
}

pub fn stream_key(&self) -> &[usize] {
&self.plan_base().stream_key
pub fn stream_key(&self) -> Option<&[usize]> {
self.plan_base().stream_key()
}

pub fn order(&self) -> &Order {
Expand Down
10 changes: 5 additions & 5 deletions src/frontend/src/optimizer/plan_node/plan_base.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ pub struct PlanBase {
pub ctx: OptimizerContextRef,
pub schema: Schema,
/// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key
pub stream_key: Vec<usize>,
pub stream_key: Option<Vec<usize>>,
/// The order property of the PlanNode's output, store an `&Order::any()` here will not affect
/// correctness, but insert unnecessary sort in plan
pub order: Order,
Expand All @@ -59,8 +59,8 @@ impl generic::GenericPlanRef for PlanBase {
&self.schema
}

fn stream_key(&self) -> &[usize] {
&self.stream_key
fn stream_key(&self) -> Option<&[usize]> {
self.stream_key.as_deref()
}

fn ctx(&self) -> OptimizerContextRef {
Expand Down Expand Up @@ -233,8 +233,8 @@ macro_rules! impl_base_delegate {
pub fn schema(&self) -> &Schema {
&self.plan_base().schema
}
pub fn stream_key(&self) -> &[usize] {
&self.plan_base().stream_key
pub fn stream_key(&self) -> Option<&[usize]> {
&self.plan_base().stream_key()
}
pub fn order(&self) -> &Order {
&self.plan_base().order
Expand Down

0 comments on commit ea3ea6b

Please sign in to comment.