risingwave_stream/from_proto/
sync_log_store.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use 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        // Previously, these configs are persisted in the plan node.
57        // Now it's always `None` and we should refer to the job's config override.
58        #[allow(deprecated)]
59        let pause_duration_ms = node.pause_duration_ms.map_or(
60            params.config.developer.sync_log_store_pause_duration_ms,
61            |v| v as usize,
62        );
63        #[allow(deprecated)]
64        let buffer_max_size = node
65            .buffer_size
66            .map_or(params.config.developer.sync_log_store_buffer_size, |v| {
67                v as usize
68            });
69
70        let chunk_size = actor_context.config.developer.chunk_size;
71
72        let executor = SyncedKvLogStoreExecutor::new(
73            actor_context,
74            table_id,
75            metrics,
76            serde,
77            store,
78            buffer_max_size,
79            chunk_size,
80            upstream,
81            Duration::from_millis(pause_duration_ms as _),
82            node.aligned,
83        );
84        Ok((params.info, executor).into())
85    }
86}