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_stream_node_body!(SyncLogStore(SyncLogStoreNode) => SyncLogStoreExecutorBuilder);
29
30impl ExecutorBuilder for SyncLogStoreExecutorBuilder {
31 type Node = SyncLogStoreNode;
32
33 async fn new_boxed_executor(
34 params: ExecutorParams,
35 node: &Self::Node,
36 store: impl StateStore,
37 ) -> StreamResult<Executor> {
38 let actor_context = params.actor_context.clone();
39 let table = node.log_store_table.as_ref().unwrap().clone();
40 let table_id = table.id;
41
42 let metrics = {
43 let streaming_metrics = actor_context.streaming_metrics.as_ref();
44 let actor_id = actor_context.id;
45 let fragment_id = params.fragment_id;
46 let name = "sync_log_store";
47 let target = "unaligned_hash_join";
48 SyncedKvLogStoreMetrics::new(streaming_metrics, actor_id, fragment_id, name, target)
49 };
50
51 let serde = LogStoreRowSerde::new(
52 &table,
53 params.vnode_bitmap.map(|b| b.into()),
54 &KV_LOG_STORE_V2_INFO,
55 );
56 let [upstream] = params.input.try_into().unwrap();
57
58 #[expect(deprecated)]
61 let pause_duration_ms = node.pause_duration_ms.map_or(
62 params.config.developer.sync_log_store_pause_duration_ms,
63 |v| v as usize,
64 );
65 #[expect(deprecated)]
66 let buffer_max_size = node
67 .buffer_size
68 .map_or(params.config.developer.sync_log_store_buffer_size, |v| {
69 v as usize
70 });
71
72 let chunk_size = actor_context.config.developer.chunk_size;
73
74 let executor = SyncedKvLogStoreExecutor::new(
75 actor_context,
76 table_id,
77 metrics,
78 serde,
79 store,
80 buffer_max_size,
81 chunk_size,
82 upstream,
83 Duration::from_millis(pause_duration_ms as _),
84 node.aligned,
85 );
86 Ok((params.info, executor).into())
87 }
88}