risingwave_stream/from_proto/
sync_log_store.rs1use risingwave_pb::stream_plan::SyncLogStoreNode;
16use risingwave_storage::StateStore;
17use tokio::time::Duration;
18
19use crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V2_INFO;
20use crate::common::log_store_impl::kv_log_store::serde::LogStoreRowSerde;
21use crate::error::StreamResult;
22use crate::executor::{Executor, SyncedKvLogStoreExecutor, SyncedKvLogStoreMetrics};
23use crate::from_proto::ExecutorBuilder;
24use crate::task::ExecutorParams;
25
26pub struct SyncLogStoreExecutorBuilder;
27
28impl ExecutorBuilder for SyncLogStoreExecutorBuilder {
29 type Node = SyncLogStoreNode;
30
31 async fn new_boxed_executor(
32 params: ExecutorParams,
33 node: &Self::Node,
34 store: impl StateStore,
35 ) -> StreamResult<Executor> {
36 let actor_context = params.actor_context.clone();
37 let table = node.log_store_table.as_ref().unwrap().clone();
38 let table_id = table.id;
39
40 let metrics = {
41 let streaming_metrics = actor_context.streaming_metrics.as_ref();
42 let actor_id = actor_context.id;
43 let join_fragment_id = 0;
44 let name = "sync_log_store";
45 let target = "unaligned_hash_join";
46 SyncedKvLogStoreMetrics::new(
47 streaming_metrics,
48 actor_id,
49 join_fragment_id,
50 name,
51 target,
52 )
53 };
54
55 let serde = LogStoreRowSerde::new(
56 &table,
57 params.vnode_bitmap.map(|b| b.into()),
58 &KV_LOG_STORE_V2_INFO,
59 );
60 let [upstream] = params.input.try_into().unwrap();
61
62 let pause_duration_ms = node.pause_duration_ms as _;
63 let buffer_max_size = node.buffer_size as usize;
64 let chunk_size = actor_context.streaming_config.developer.chunk_size as u32;
65
66 let executor = SyncedKvLogStoreExecutor::new(
67 actor_context,
68 table_id,
69 metrics,
70 serde,
71 store,
72 buffer_max_size,
73 chunk_size,
74 upstream,
75 Duration::from_millis(pause_duration_ms),
76 );
77 Ok((params.info, executor).into())
78 }
79}