Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(stream): refine TopNExecutorBase #1822

Merged
merged 1 commit into from
Apr 13, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions src/stream/src/executor_v2/top_n.rs
Original file line number Diff line number Diff line change
Expand Up @@ -434,6 +434,18 @@ impl<S: StateStore> TopNExecutorBase for InnerTopNExecutor<S> {
async fn flush_data(&mut self, epoch: u64) -> StreamExecutorResult<()> {
self.flush_inner(epoch).await
}

fn schema(&self) -> &Schema {
&self.schema
}

fn pk_indices(&self) -> PkIndicesRef {
&self.pk_indices
}

fn identity(&self) -> &str {
&self.info.identity
}
}

#[cfg(test)]
Expand Down
33 changes: 13 additions & 20 deletions src/stream/src/executor_v2/top_n_appendonly.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ use crate::executor::managed_state::top_n::variants::TOP_N_MAX;
use crate::executor::managed_state::top_n::ManagedTopNState;
use crate::executor_v2::error::{StreamExecutorError, StreamExecutorResult};
use crate::executor_v2::top_n_executor::{generate_output, TopNExecutorBase, TopNExecutorWrapper};
use crate::executor_v2::{BoxedMessageStream, Executor, ExecutorInfo, PkIndices, PkIndicesRef};
use crate::executor_v2::{Executor, ExecutorInfo, PkIndices, PkIndicesRef};

/// If the input contains only append, `AppendOnlyTopNExecutor` does not need
/// to keep all the data records/rows that have been seen. As long as a record
Expand Down Expand Up @@ -180,25 +180,6 @@ impl<S: StateStore> InnerAppendOnlyTopNExecutor<S> {
}
}

#[async_trait]
impl<S: StateStore> Executor for InnerAppendOnlyTopNExecutor<S> {
fn execute(self: Box<Self>) -> BoxedMessageStream {
panic!("Should execute by wrapper");
}

fn schema(&self) -> &Schema {
&self.schema
}

fn pk_indices(&self) -> PkIndicesRef {
&self.pk_indices
}

fn identity(&self) -> &str {
&self.info.identity
}
}

#[async_trait]
impl<S: StateStore> TopNExecutorBase for InnerAppendOnlyTopNExecutor<S> {
async fn apply_chunk(
Expand Down Expand Up @@ -324,6 +305,18 @@ impl<S: StateStore> TopNExecutorBase for InnerAppendOnlyTopNExecutor<S> {
async fn flush_data(&mut self, epoch: u64) -> StreamExecutorResult<()> {
self.flush_inner(epoch).await
}

fn schema(&self) -> &Schema {
&self.schema
}

fn pk_indices(&self) -> PkIndicesRef {
&self.pk_indices
}

fn identity(&self) -> &str {
&self.info.identity
}
}

#[cfg(test)]
Expand Down
29 changes: 10 additions & 19 deletions src/stream/src/executor_v2/top_n_executor.rs
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,15 @@ pub trait TopNExecutorBase: Send + 'static {

/// Flush the buffered chunk to the storage backend.
async fn flush_data(&mut self, epoch: u64) -> StreamExecutorResult<()>;

/// See [`Executor::schema`].
fn schema(&self) -> &Schema;

/// See [`Executor::pk_indices`].
fn pk_indices(&self) -> PkIndicesRef;

/// See [`Executor::identity`].
fn identity(&self) -> &str;
}

/// The struct wraps a [`TopNExecutorBase`]
Expand All @@ -49,7 +58,7 @@ pub struct TopNExecutorWrapper<E> {

impl<E> Executor for TopNExecutorWrapper<E>
where
E: TopNExecutorBase + Executor,
E: TopNExecutorBase,
{
fn execute(self: Box<Self>) -> BoxedMessageStream {
self.top_n_executor_execute().boxed()
Expand All @@ -68,24 +77,6 @@ where
}
}

#[async_trait]
impl<E> TopNExecutorBase for TopNExecutorWrapper<E>
where
E: TopNExecutorBase,
{
async fn apply_chunk(
&mut self,
chunk: StreamChunk,
epoch: u64,
) -> StreamExecutorResult<StreamChunk> {
self.inner.apply_chunk(chunk, epoch).await
}

async fn flush_data(&mut self, epoch: u64) -> StreamExecutorResult<()> {
self.inner.flush_data(epoch).await
}
}

impl<E> TopNExecutorWrapper<E>
where
E: TopNExecutorBase,
Expand Down