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 fragment_id = params.fragment_id;
44 let name = "sync_log_store";
45 let target = "unaligned_hash_join";
46 SyncedKvLogStoreMetrics::new(streaming_metrics, actor_id, fragment_id, name, target)
47 };
48
49 let serde = LogStoreRowSerde::new(
50 &table,
51 params.vnode_bitmap.map(|b| b.into()),
52 &KV_LOG_STORE_V2_INFO,
53 );
54 let [upstream] = params.input.try_into().unwrap();
55
56 let pause_duration_ms = node.pause_duration_ms as _;
57 let buffer_max_size = node.buffer_size as usize;
58 let chunk_size = actor_context.streaming_config.developer.chunk_size;
59
60 let executor = SyncedKvLogStoreExecutor::new(
61 actor_context,
62 table_id,
63 metrics,
64 serde,
65 store,
66 buffer_max_size,
67 chunk_size,
68 upstream,
69 Duration::from_millis(pause_duration_ms),
70 node.aligned,
71 );
72 Ok((params.info, executor).into())
73 }
74}