forked from risingwavelabs/risingwave
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(streaming): separate
BarrierRecv
executor (risingwavelabs#8595)
Signed-off-by: Bugen Zhao <i@bugenzhao.com>
- Loading branch information
Showing
9 changed files
with
229 additions
and
5 deletions.
There are no files selected for viewing
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,107 @@ | ||
// Copyright 2023 RisingWave Labs | ||
// | ||
// 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 risingwave_common::catalog::Schema; | ||
use tokio::sync::mpsc::UnboundedReceiver; | ||
use tokio_stream::wrappers::UnboundedReceiverStream; | ||
|
||
use super::{ | ||
ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Executor, Message, PkIndicesRef, | ||
StreamExecutorError, | ||
}; | ||
|
||
/// The executor only for receiving barrier from the meta service. It always resides in the leaves | ||
/// of the streaming graph. | ||
pub struct BarrierRecvExecutor { | ||
_ctx: ActorContextRef, | ||
identity: String, | ||
|
||
/// The barrier receiver registered in the local barrier manager. | ||
barrier_receiver: UnboundedReceiver<Barrier>, | ||
} | ||
|
||
impl BarrierRecvExecutor { | ||
pub fn new( | ||
ctx: ActorContextRef, | ||
barrier_receiver: UnboundedReceiver<Barrier>, | ||
executor_id: u64, | ||
) -> Self { | ||
Self { | ||
_ctx: ctx, | ||
identity: format!("BarrierRecvExecutor {:X}", executor_id), | ||
barrier_receiver, | ||
} | ||
} | ||
|
||
pub fn for_test(barrier_receiver: UnboundedReceiver<Barrier>) -> Self { | ||
Self::new(ActorContext::create(0), barrier_receiver, 0) | ||
} | ||
} | ||
|
||
impl Executor for BarrierRecvExecutor { | ||
fn execute(self: Box<Self>) -> BoxedMessageStream { | ||
UnboundedReceiverStream::new(self.barrier_receiver) | ||
.map(|barrier| Ok(Message::Barrier(barrier))) | ||
.chain(futures::stream::once(async { | ||
// We do not use the stream termination as the control message, and this line should | ||
// never be reached in normal cases. So we just return an error here. | ||
Err(StreamExecutorError::channel_closed("barrier receiver")) | ||
})) | ||
.boxed() | ||
} | ||
|
||
fn schema(&self) -> &Schema { | ||
Schema::empty() | ||
} | ||
|
||
fn pk_indices(&self) -> PkIndicesRef<'_> { | ||
&[] | ||
} | ||
|
||
fn identity(&self) -> &str { | ||
&self.identity | ||
} | ||
} | ||
|
||
#[cfg(test)] | ||
mod tests { | ||
use futures::pin_mut; | ||
use tokio::sync::mpsc; | ||
|
||
use super::*; | ||
use crate::executor::test_utils::StreamExecutorTestExt; | ||
|
||
#[tokio::test] | ||
async fn test_barrier_recv() { | ||
let (barrier_tx, barrier_rx) = mpsc::unbounded_channel(); | ||
|
||
let barrier_recv = BarrierRecvExecutor::for_test(barrier_rx).boxed(); | ||
let stream = barrier_recv.execute(); | ||
pin_mut!(stream); | ||
|
||
barrier_tx.send(Barrier::new_test_barrier(114)).unwrap(); | ||
barrier_tx.send(Barrier::new_test_barrier(514)).unwrap(); | ||
|
||
let barrier_1 = stream.next_unwrap_ready_barrier().unwrap(); | ||
assert_eq!(barrier_1.epoch.curr, 114); | ||
let barrier_2 = stream.next_unwrap_ready_barrier().unwrap(); | ||
assert_eq!(barrier_2.epoch.curr, 514); | ||
|
||
stream.next_unwrap_pending(); | ||
|
||
drop(barrier_tx); | ||
assert!(stream.next_unwrap_ready().is_err()); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
// Copyright 2023 RisingWave Labs | ||
// | ||
// 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::BarrierRecvNode; | ||
use tokio::sync::mpsc::unbounded_channel; | ||
|
||
use super::*; | ||
use crate::executor::BarrierRecvExecutor; | ||
|
||
pub struct BarrierRecvExecutorBuilder; | ||
|
||
#[async_trait::async_trait] | ||
impl ExecutorBuilder for BarrierRecvExecutorBuilder { | ||
type Node = BarrierRecvNode; | ||
|
||
async fn new_boxed_executor( | ||
params: ExecutorParams, | ||
_node: &Self::Node, | ||
_store: impl StateStore, | ||
stream: &mut LocalStreamManagerCore, | ||
) -> StreamResult<BoxedExecutor> { | ||
assert!( | ||
params.input.is_empty(), | ||
"barrier receiver should not have input" | ||
); | ||
|
||
let (sender, barrier_receiver) = unbounded_channel(); | ||
stream | ||
.context | ||
.lock_barrier_manager() | ||
.register_sender(params.actor_context.id, sender); | ||
|
||
Ok( | ||
BarrierRecvExecutor::new(params.actor_context, barrier_receiver, params.executor_id) | ||
.boxed(), | ||
) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters