Skip to content
Closed
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
40 changes: 39 additions & 1 deletion dashboard/proto/gen/stream_plan.ts

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

6 changes: 6 additions & 0 deletions proto/stream_plan.proto
Original file line number Diff line number Diff line change
Expand Up @@ -440,6 +440,11 @@ message RowIdGenNode {
uint64 row_id_index = 1;
}

message NowNode {
// Persists emitted 'now'.
catalog.Table state_table = 1;
}

message StreamNode {
oneof node_body {
SourceNode source = 100;
Expand Down Expand Up @@ -471,6 +476,7 @@ message StreamNode {
WatermarkFilterNode watermark_filter = 126;
DmlNode dml = 127;
RowIdGenNode row_id_gen = 128;
NowNode now = 129;
}
// The id for the operator. This is local per mview.
// TODO: should better be a uint32.
Expand Down
2 changes: 2 additions & 0 deletions src/stream/src/executor/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ mod lookup_union;
mod managed_state;
mod merge;
mod mview;
mod now;
mod project;
mod project_set;
mod rearranged_chain;
Expand Down Expand Up @@ -109,6 +110,7 @@ pub use lookup::*;
pub use lookup_union::LookupUnionExecutor;
pub use merge::MergeExecutor;
pub use mview::*;
pub use now::NowExecutor;
pub use project::ProjectExecutor;
pub use project_set::*;
pub use rearranged_chain::RearrangedChainExecutor;
Expand Down
88 changes: 88 additions & 0 deletions src/stream/src/executor/now.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// Copyright 2022 Singularity Data
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use futures::StreamExt;
use futures_async_stream::try_stream;
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::types::DataType;
use risingwave_storage::StateStore;
use tokio::sync::mpsc::UnboundedReceiver;

use super::{
Barrier, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, StreamExecutorError,
};
use crate::common::table::state_table::StateTable;

pub struct NowExecutor<S: StateStore> {
/// Receiver of barrier channel.
barrier_receiver: UnboundedReceiver<Barrier>,

pk_indices: PkIndices,
identity: String,
schema: Schema,
state_table: StateTable<S>,
}

impl<S: StateStore> NowExecutor<S> {
pub fn new(
barrier_receiver: UnboundedReceiver<Barrier>,
executor_id: u64,
state_table: StateTable<S>,
) -> Self {
let schema = Schema::new(vec![Field {
data_type: DataType::Timestamp,
name: String::from("now"),
sub_fields: vec![],
type_name: String::default(),
}]);
Self {
barrier_receiver,
pk_indices: vec![],
identity: format!("NowExecutor {:X}", executor_id),
schema,
state_table,
}
}

#[try_stream(ok = Message, error = StreamExecutorError)]
async fn into_stream(self) {
#[allow(unused_variables)]
let Self {
barrier_receiver,
state_table,
schema,
..
} = self;

todo!("https://github.com/risingwavelabs/risingwave/pull/6408");
}
}

impl<S: StateStore> Executor for NowExecutor<S> {
fn execute(self: Box<Self>) -> BoxedMessageStream {
self.into_stream().boxed()
}

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

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

fn identity(&self) -> &str {
self.identity.as_str()
}
}
3 changes: 3 additions & 0 deletions src/stream/src/from_proto/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ mod lookup;
mod lookup_union;
mod merge;
mod mview;
mod now;
mod project;
mod project_set;
mod row_id_gen;
Expand Down Expand Up @@ -64,6 +65,7 @@ use self::lookup::*;
use self::lookup_union::*;
use self::merge::*;
use self::mview::*;
use self::now::NowExecutorBuilder;
use self::project::*;
use self::project_set::*;
use self::row_id_gen::RowIdGenExecutorBuilder;
Expand Down Expand Up @@ -143,5 +145,6 @@ pub async fn create_executor(
NodeBody::WatermarkFilter => WatermarkFilterBuilder,
NodeBody::Dml => DmlExecutorBuilder,
NodeBody::RowIdGen => RowIdGenExecutorBuilder,
NodeBody::Now => NowExecutorBuilder,
}
}
52 changes: 52 additions & 0 deletions src/stream/src/from_proto/now.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
// Copyright 2022 Singularity Data
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use risingwave_pb::stream_plan::NowNode;
use risingwave_storage::StateStore;
use tokio::sync::mpsc::unbounded_channel;

use super::ExecutorBuilder;
use crate::common::table::state_table::StateTable;
use crate::error::StreamResult;
use crate::executor::{BoxedExecutor, NowExecutor};
use crate::task::{ExecutorParams, LocalStreamManagerCore};

pub struct NowExecutorBuilder;

#[async_trait::async_trait]
impl ExecutorBuilder for NowExecutorBuilder {
type Node = NowNode;

async fn new_boxed_executor(
params: ExecutorParams,
node: &NowNode,
store: impl StateStore,
stream: &mut LocalStreamManagerCore,
) -> StreamResult<BoxedExecutor> {
let (sender, barrier_receiver) = unbounded_channel();
stream
.context
.lock_barrier_manager()
.register_sender(params.actor_context.id, sender);

let state_table =
StateTable::from_table_catalog(node.get_state_table()?, store, None).await;

Ok(Box::new(NowExecutor::new(
barrier_receiver,
params.executor_id,
state_table,
)))
}
}