Skip to main content

common_meta/key/flow/
flow_info.rs

1// Copyright 2023 Greptime Team
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 std::collections::{BTreeMap, HashMap};
16use std::sync::Arc;
17
18use chrono::{DateTime, Utc};
19use futures::TryStreamExt;
20use futures::stream::BoxStream;
21use lazy_static::lazy_static;
22use regex::Regex;
23use serde::{Deserialize, Serialize};
24use snafu::OptionExt;
25use table::metadata::TableId;
26use table::table_name::TableName;
27
28use crate::FlownodeId;
29use crate::error::{self, Result};
30use crate::key::flow::FlowScoped;
31use crate::key::txn_helper::TxnOpGetResponseSet;
32use crate::key::{
33    BytesAdapter, DeserializedValueWithBytes, FlowId, FlowPartitionId, MetadataKey, MetadataValue,
34};
35use crate::kv_backend::KvBackendRef;
36use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
37use crate::range_stream::{DEFAULT_PAGE_SIZE, PaginationStream};
38use crate::rpc::KeyValue;
39use crate::rpc::store::RangeRequest;
40
41pub const FLOW_INFO_KEY_PREFIX: &str = "info";
42
43/// The lifecycle status of a flow stored in metadata.
44#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Default)]
45pub enum FlowStatus {
46    /// The flow metadata exists, but at least one source table did not exist at create time.
47    PendingSources,
48    /// The flow has resolved source tables and can be scheduled on flownodes.
49    #[default]
50    Active,
51}
52
53lazy_static! {
54    static ref FLOW_INFO_KEY_PATTERN: Regex =
55        Regex::new(&format!("^{FLOW_INFO_KEY_PREFIX}/([0-9]+)$")).unwrap();
56}
57
58/// The key stores the metadata of the flow.
59///
60/// The layout: `__flow/info/{flow_id}`.
61pub struct FlowInfoKey(FlowScoped<FlowInfoKeyInner>);
62
63pub type FlowInfoDecodeResult = Result<Option<DeserializedValueWithBytes<FlowInfoValue>>>;
64
65impl<'a> MetadataKey<'a, FlowInfoKey> for FlowInfoKey {
66    fn to_bytes(&self) -> Vec<u8> {
67        self.0.to_bytes()
68    }
69
70    fn from_bytes(bytes: &'a [u8]) -> Result<FlowInfoKey> {
71        Ok(FlowInfoKey(FlowScoped::<FlowInfoKeyInner>::from_bytes(
72            bytes,
73        )?))
74    }
75}
76
77impl FlowInfoKey {
78    /// Returns the [FlowInfoKey].
79    pub fn new(flow_id: FlowId) -> FlowInfoKey {
80        let inner = FlowInfoKeyInner::new(flow_id);
81        FlowInfoKey(FlowScoped::new(inner))
82    }
83
84    /// Returns the [FlowId].
85    pub fn flow_id(&self) -> FlowId {
86        self.0.flow_id
87    }
88}
89
90/// The key of flow metadata.
91#[derive(Debug, Clone, Copy, PartialEq)]
92struct FlowInfoKeyInner {
93    flow_id: FlowId,
94}
95
96impl FlowInfoKeyInner {
97    /// Returns a [FlowInfoKey] with the specified `flow_id`.
98    pub fn new(flow_id: FlowId) -> FlowInfoKeyInner {
99        FlowInfoKeyInner { flow_id }
100    }
101}
102
103impl<'a> MetadataKey<'a, FlowInfoKeyInner> for FlowInfoKeyInner {
104    fn to_bytes(&self) -> Vec<u8> {
105        format!("{FLOW_INFO_KEY_PREFIX}/{}", self.flow_id).into_bytes()
106    }
107
108    fn from_bytes(bytes: &'a [u8]) -> Result<FlowInfoKeyInner> {
109        let key = std::str::from_utf8(bytes).map_err(|e| {
110            error::InvalidMetadataSnafu {
111                err_msg: format!(
112                    "FlowInfoKeyInner '{}' is not a valid UTF8 string: {e}",
113                    String::from_utf8_lossy(bytes)
114                ),
115            }
116            .build()
117        })?;
118        let captures =
119            FLOW_INFO_KEY_PATTERN
120                .captures(key)
121                .context(error::InvalidMetadataSnafu {
122                    err_msg: format!("Invalid FlowInfoKeyInner '{key}'"),
123                })?;
124        // Safety: pass the regex check above
125        let flow_id = captures[1].parse::<FlowId>().unwrap();
126        Ok(FlowInfoKeyInner { flow_id })
127    }
128}
129
130// The metadata of the flow.
131#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
132pub struct FlowInfoValue {
133    /// The source tables used by the flow.
134    #[serde(default)]
135    pub source_table_ids: Vec<TableId>,
136    #[serde(default)]
137    pub all_source_table_names: Vec<TableName>,
138    #[serde(default)]
139    pub unresolved_source_table_names: Vec<TableName>,
140    /// The sink table used by the flow.
141    pub sink_table_name: TableName,
142    /// Which flow nodes this flow is running on.
143    pub flownode_ids: BTreeMap<FlowPartitionId, FlownodeId>,
144    /// The catalog name.
145    pub catalog_name: String,
146    /// The query context used when create flow.
147    /// Although flow doesn't belong to any schema, this query_context is needed to remember
148    /// the query context when `create_flow` is executed
149    /// for recovering flow using the same sql&query_context after db restart.
150    /// if none, should use default query context
151    #[serde(default)]
152    pub query_context: Option<crate::rpc::ddl::QueryContext>,
153    /// The flow name.
154    pub flow_name: String,
155    /// The raw sql.
156    pub raw_sql: String,
157    /// The expr of expire.
158    /// Duration in seconds as `i64`.
159    pub expire_after: Option<i64>,
160    /// The eval interval.
161    /// Duration in seconds as `i64`.
162    /// If `None`, will automatically decide when to evaluate the flow.
163    /// If `Some`, it will be evaluated every `eval_interval` seconds.
164    #[serde(default)]
165    pub eval_interval_secs: Option<i64>,
166    /// The comment.
167    pub comment: String,
168    /// The options.
169    pub options: HashMap<String, String>,
170    #[serde(default)]
171    pub status: FlowStatus,
172    /// The created time
173    #[serde(default)]
174    pub created_time: DateTime<Utc>,
175    /// The updated time.
176    #[serde(default)]
177    pub updated_time: DateTime<Utc>,
178}
179
180impl FlowInfoValue {
181    pub fn is_pending(&self) -> bool {
182        self.status == FlowStatus::PendingSources
183    }
184
185    pub fn is_active(&self) -> bool {
186        self.status == FlowStatus::Active
187    }
188
189    /// Returns the `flownode_id`.
190    pub fn flownode_ids(&self) -> &BTreeMap<FlowPartitionId, FlownodeId> {
191        &self.flownode_ids
192    }
193
194    /// Insert a new flownode id for a partition.
195    pub fn insert_flownode_id(
196        &mut self,
197        partition: FlowPartitionId,
198        node: FlownodeId,
199    ) -> Option<FlownodeId> {
200        self.flownode_ids.insert(partition, node)
201    }
202
203    /// Returns the `source_table`.
204    pub fn source_table_ids(&self) -> &[TableId] {
205        &self.source_table_ids
206    }
207
208    pub fn all_source_table_names(&self) -> &[TableName] {
209        &self.all_source_table_names
210    }
211
212    pub fn unresolved_source_table_names(&self) -> &[TableName] {
213        &self.unresolved_source_table_names
214    }
215
216    pub fn catalog_name(&self) -> &String {
217        &self.catalog_name
218    }
219
220    pub fn query_context(&self) -> &Option<crate::rpc::ddl::QueryContext> {
221        &self.query_context
222    }
223
224    pub fn flow_name(&self) -> &String {
225        &self.flow_name
226    }
227
228    pub fn sink_table_name(&self) -> &TableName {
229        &self.sink_table_name
230    }
231
232    pub fn raw_sql(&self) -> &String {
233        &self.raw_sql
234    }
235
236    pub fn expire_after(&self) -> Option<i64> {
237        self.expire_after
238    }
239
240    pub fn eval_interval(&self) -> Option<i64> {
241        self.eval_interval_secs
242    }
243
244    pub fn comment(&self) -> &String {
245        &self.comment
246    }
247
248    pub fn options(&self) -> &HashMap<String, String> {
249        &self.options
250    }
251
252    pub fn status(&self) -> &FlowStatus {
253        &self.status
254    }
255
256    pub fn created_time(&self) -> &DateTime<Utc> {
257        &self.created_time
258    }
259
260    pub fn updated_time(&self) -> &DateTime<Utc> {
261        &self.updated_time
262    }
263}
264
265pub type FlowInfoManagerRef = Arc<FlowInfoManager>;
266
267/// The manager of [FlowInfoKey].
268pub struct FlowInfoManager {
269    kv_backend: KvBackendRef,
270}
271
272pub fn flow_info_decoder(kv: KeyValue) -> Result<(FlowInfoKey, FlowInfoValue)> {
273    let key = FlowInfoKey::from_bytes(&kv.key)?;
274    let value = FlowInfoValue::try_from_raw_value(&kv.value)?;
275    Ok((key, value))
276}
277
278impl FlowInfoManager {
279    /// Returns a new [FlowInfoManager].
280    pub fn new(kv_backend: KvBackendRef) -> Self {
281        Self { kv_backend }
282    }
283
284    /// Returns the [FlowInfoValue] of specified `flow_id`.
285    pub async fn get(&self, flow_id: FlowId) -> Result<Option<FlowInfoValue>> {
286        let key = FlowInfoKey::new(flow_id).to_bytes();
287        self.kv_backend
288            .get(&key)
289            .await?
290            .map(|x| FlowInfoValue::try_from_raw_value(&x.value))
291            .transpose()
292    }
293
294    /// Returns the [FlowInfoValue] with original bytes of specified `flow_id`.
295    pub async fn get_raw(
296        &self,
297        flow_id: FlowId,
298    ) -> Result<Option<DeserializedValueWithBytes<FlowInfoValue>>> {
299        let key = FlowInfoKey::new(flow_id).to_bytes();
300        self.kv_backend
301            .get(&key)
302            .await?
303            .map(|x| DeserializedValueWithBytes::from_inner_slice(&x.value))
304            .transpose()
305    }
306
307    pub fn flow_infos(&self) -> BoxStream<'static, Result<(FlowId, FlowInfoValue)>> {
308        let start_key = FlowScoped::new(BytesAdapter::from(
309            format!("{FLOW_INFO_KEY_PREFIX}/").into_bytes(),
310        ))
311        .to_bytes();
312        let req = RangeRequest::new().with_prefix(start_key);
313        let stream = PaginationStream::new(
314            self.kv_backend.clone(),
315            req,
316            DEFAULT_PAGE_SIZE,
317            flow_info_decoder,
318        )
319        .into_stream();
320
321        Box::pin(stream.map_ok(|(key, value)| (key.flow_id(), value)))
322    }
323
324    /// Builds a create flow transaction.
325    /// It is expected that the `__flow/info/{flow_id}` wasn't occupied.
326    /// Otherwise, the transaction will retrieve existing value.
327    pub(crate) fn build_create_txn(
328        &self,
329        flow_id: FlowId,
330        flow_value: &FlowInfoValue,
331    ) -> Result<(
332        Txn,
333        impl FnOnce(&mut TxnOpGetResponseSet) -> FlowInfoDecodeResult,
334    )> {
335        let key = FlowInfoKey::new(flow_id).to_bytes();
336        let txn = Txn::put_if_not_exists(key.clone(), flow_value.try_as_raw_value()?);
337
338        Ok((
339            txn,
340            TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
341        ))
342    }
343
344    /// Builds a update flow transaction.
345    /// It is expected that the `__flow/info/{flow_id}` IS ALREADY occupied and equal to `prev_flow_value`,
346    /// but the new value can be the same, so to allow replace operation to happen even when the value is the same.
347    /// Otherwise, the transaction will retrieve existing value and fail.
348    pub(crate) fn build_update_txn(
349        &self,
350        flow_id: FlowId,
351        current_flow_value: &DeserializedValueWithBytes<FlowInfoValue>,
352        new_flow_value: &FlowInfoValue,
353    ) -> Result<(
354        Txn,
355        impl FnOnce(&mut TxnOpGetResponseSet) -> FlowInfoDecodeResult,
356    )> {
357        let key = FlowInfoKey::new(flow_id).to_bytes();
358        let raw_value = new_flow_value.try_as_raw_value()?;
359        let prev_value = current_flow_value.get_raw_bytes();
360        let txn = Txn::new()
361            .when(vec![Compare::new(
362                key.clone(),
363                CompareOp::Equal,
364                Some(prev_value),
365            )])
366            .and_then(vec![TxnOp::Put(key.clone(), raw_value)])
367            .or_else(vec![TxnOp::Get(key.clone())]);
368
369        Ok((
370            txn,
371            TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
372        ))
373    }
374}
375
376#[cfg(test)]
377mod tests {
378    use super::*;
379
380    #[test]
381    fn test_key_serialization() {
382        let flow_info = FlowInfoKey::new(2);
383        assert_eq!(b"__flow/info/2".to_vec(), flow_info.to_bytes());
384    }
385
386    #[test]
387    fn test_key_deserialization() {
388        let bytes = b"__flow/info/2".to_vec();
389        let key = FlowInfoKey::from_bytes(&bytes).unwrap();
390        assert_eq!(key.flow_id(), 2);
391    }
392}