risingwave_stream/executor/source/
source_backfill_state_table.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
// Copyright 2024 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 std::ops::Bound;

use futures::{pin_mut, StreamExt};
use risingwave_common::row;
use risingwave_common::row::{OwnedRow, Row};
use risingwave_common::types::{ScalarImpl, ScalarRef, ScalarRefImpl};
use risingwave_common::util::epoch::EpochPair;
use risingwave_connector::source::SplitId;
use risingwave_pb::catalog::PbTable;
use risingwave_storage::StateStore;

use super::source_backfill_executor::{BackfillStateWithProgress, BackfillStates};
use crate::common::table::state_table::StateTable;
use crate::executor::error::StreamExecutorError;
use crate::executor::StreamExecutorResult;

pub struct BackfillStateTableHandler<S: StateStore> {
    pub state_store: StateTable<S>,
}

impl<S: StateStore> BackfillStateTableHandler<S> {
    /// See also [`super::SourceStateTableHandler::from_table_catalog`] for how the state table looks like.
    pub async fn from_table_catalog(table_catalog: &PbTable, store: S) -> Self {
        Self {
            state_store: StateTable::from_table_catalog(table_catalog, store, None).await,
        }
    }

    pub async fn init_epoch(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
        self.state_store.init_epoch(epoch).await
    }

    fn string_to_scalar(rhs: impl Into<String>) -> ScalarImpl {
        ScalarImpl::Utf8(rhs.into().into_boxed_str())
    }

    pub(crate) async fn get(&self, key: &SplitId) -> StreamExecutorResult<Option<OwnedRow>> {
        self.state_store
            .get_row(row::once(Some(Self::string_to_scalar(key.as_ref()))))
            .await
            .map_err(StreamExecutorError::from)
    }

    /// XXX: we might get stale data for other actors' writes, but it's fine?
    pub async fn scan(&self) -> StreamExecutorResult<Vec<BackfillStateWithProgress>> {
        let sub_range: &(Bound<OwnedRow>, Bound<OwnedRow>) = &(Bound::Unbounded, Bound::Unbounded);

        let state_table_iter = self
            .state_store
            .iter_with_prefix(None::<OwnedRow>, sub_range, Default::default())
            .await?;
        pin_mut!(state_table_iter);

        let mut ret = vec![];
        while let Some(item) = state_table_iter.next().await {
            let row = item?.into_owned_row();
            let state = match row.datum_at(1) {
                Some(ScalarRefImpl::Jsonb(jsonb_ref)) => {
                    BackfillStateWithProgress::restore_from_json(jsonb_ref.to_owned_scalar())?
                }
                _ => unreachable!(),
            };
            ret.push(state);
        }
        tracing::trace!("scan SourceBackfill state table: {:?}", ret);
        Ok(ret)
    }

    async fn set(
        &mut self,
        key: SplitId,
        state: BackfillStateWithProgress,
    ) -> StreamExecutorResult<()> {
        let row = [
            Some(Self::string_to_scalar(key.as_ref())),
            Some(ScalarImpl::Jsonb(state.encode_to_json())),
        ];
        match self.get(&key).await? {
            Some(prev_row) => {
                self.state_store.update(prev_row, row);
            }
            None => {
                self.state_store.insert(row);
            }
        }
        Ok(())
    }

    pub async fn delete(&mut self, key: &SplitId) -> StreamExecutorResult<()> {
        if let Some(prev_row) = self.get(key).await? {
            self.state_store.delete(prev_row);
        }

        Ok(())
    }

    pub async fn set_states(&mut self, states: BackfillStates) -> StreamExecutorResult<()> {
        for (split_id, state) in states {
            self.set(split_id, state).await?;
        }
        Ok(())
    }

    pub async fn trim_state(
        &mut self,
        to_trim: impl IntoIterator<Item = SplitId>,
    ) -> StreamExecutorResult<()> {
        for split_id in to_trim {
            tracing::info!("trimming source state for split {}", split_id);
            self.delete(&split_id).await?;
        }

        Ok(())
    }

    pub async fn try_recover_from_state_store(
        &mut self,
        split_id: &SplitId,
    ) -> StreamExecutorResult<Option<BackfillStateWithProgress>> {
        Ok(self
            .get(split_id)
            .await?
            .map(|row| match row.datum_at(1) {
                Some(ScalarRefImpl::Jsonb(jsonb_ref)) => {
                    BackfillStateWithProgress::restore_from_json(jsonb_ref.to_owned_scalar())
                }
                _ => unreachable!(),
            })
            .transpose()?)
    }
}