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
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
// Copyright 2023 Greptime Team
//
// 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.

mod metadata;

use std::collections::BTreeMap;

use api::v1::flow::flow_request::Body as PbFlowRequest;
use api::v1::flow::{CreateRequest, FlowRequest, FlowRequestHeader};
use api::v1::ExpireAfter;
use async_trait::async_trait;
use common_catalog::format_full_flow_name;
use common_procedure::error::{FromJsonSnafu, ToJsonSnafu};
use common_procedure::{
    Context as ProcedureContext, LockKey, Procedure, Result as ProcedureResult, Status,
};
use common_telemetry::info;
use common_telemetry::tracing_context::TracingContext;
use futures::future::join_all;
use itertools::Itertools;
use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt};
use strum::AsRefStr;
use table::metadata::TableId;

use super::utils::add_peer_context_if_needed;
use crate::cache_invalidator::Context;
use crate::ddl::utils::handle_retry_error;
use crate::ddl::DdlContext;
use crate::error::{self, Result};
use crate::instruction::{CacheIdent, CreateFlow};
use crate::key::flow::flow_info::FlowInfoValue;
use crate::key::flow::flow_route::FlowRouteValue;
use crate::key::table_name::TableNameKey;
use crate::key::{FlowId, FlowPartitionId};
use crate::lock_key::{CatalogLock, FlowNameLock, TableNameLock};
use crate::peer::Peer;
use crate::rpc::ddl::{CreateFlowTask, QueryContext};
use crate::{metrics, ClusterId};

/// The procedure of flow creation.
pub struct CreateFlowProcedure {
    pub context: DdlContext,
    pub data: CreateFlowData,
}

impl CreateFlowProcedure {
    pub const TYPE_NAME: &'static str = "metasrv-procedure::CreateFlow";

    /// Returns a new [CreateFlowProcedure].
    pub fn new(
        cluster_id: ClusterId,
        task: CreateFlowTask,
        query_context: QueryContext,
        context: DdlContext,
    ) -> Self {
        Self {
            context,
            data: CreateFlowData {
                cluster_id,
                task,
                flow_id: None,
                peers: vec![],
                source_table_ids: vec![],
                query_context,
                state: CreateFlowState::Prepare,
            },
        }
    }

    /// Deserializes from `json`.
    pub fn from_json(json: &str, context: DdlContext) -> ProcedureResult<Self> {
        let data = serde_json::from_str(json).context(FromJsonSnafu)?;
        Ok(CreateFlowProcedure { context, data })
    }

    pub(crate) async fn on_prepare(&mut self) -> Result<Status> {
        let catalog_name = &self.data.task.catalog_name;
        let flow_name = &self.data.task.flow_name;
        let sink_table_name = &self.data.task.sink_table_name;
        let create_if_not_exists = self.data.task.create_if_not_exists;

        let flow_name_value = self
            .context
            .flow_metadata_manager
            .flow_name_manager()
            .get(catalog_name, flow_name)
            .await?;

        if let Some(value) = flow_name_value {
            ensure!(
                create_if_not_exists,
                error::FlowAlreadyExistsSnafu {
                    flow_name: format_full_flow_name(catalog_name, flow_name),
                }
            );

            let flow_id = value.flow_id();
            return Ok(Status::done_with_output(flow_id));
        }

        //  Ensures sink table doesn't exist.
        let exists = self
            .context
            .table_metadata_manager
            .table_name_manager()
            .exists(TableNameKey::new(
                &sink_table_name.catalog_name,
                &sink_table_name.schema_name,
                &sink_table_name.table_name,
            ))
            .await?;
        // TODO(discord9): due to undefined behavior in flow's plan in how to transform types in mfp, sometime flow can't deduce correct schema
        // and require manually create sink table
        if exists {
            common_telemetry::warn!("Table already exists, table: {}", sink_table_name);
        }

        self.collect_source_tables().await?;
        self.allocate_flow_id().await?;
        self.data.state = CreateFlowState::CreateFlows;

        Ok(Status::executing(true))
    }

    async fn on_flownode_create_flows(&mut self) -> Result<Status> {
        // Safety: must be allocated.
        let mut create_flow = Vec::with_capacity(self.data.peers.len());
        for peer in &self.data.peers {
            let requester = self.context.node_manager.flownode(peer).await;
            let request = FlowRequest {
                header: Some(FlowRequestHeader {
                    tracing_context: TracingContext::from_current_span().to_w3c(),
                    query_context: Some(self.data.query_context.clone().into()),
                }),
                body: Some(PbFlowRequest::Create((&self.data).into())),
            };
            create_flow.push(async move {
                requester
                    .handle(request)
                    .await
                    .map_err(add_peer_context_if_needed(peer.clone()))
            });
        }

        join_all(create_flow)
            .await
            .into_iter()
            .collect::<Result<Vec<_>>>()?;

        self.data.state = CreateFlowState::CreateMetadata;
        Ok(Status::executing(true))
    }

    /// Creates flow metadata.
    ///
    /// Abort(not-retry):
    /// - Failed to create table metadata.
    async fn on_create_metadata(&mut self) -> Result<Status> {
        // Safety: The flow id must be allocated.
        let flow_id = self.data.flow_id.unwrap();
        // TODO(weny): Support `or_replace`.
        let (flow_info, flow_routes) = (&self.data).into();
        self.context
            .flow_metadata_manager
            .create_flow_metadata(flow_id, flow_info, flow_routes)
            .await?;
        info!("Created flow metadata for flow {flow_id}");
        self.data.state = CreateFlowState::InvalidateFlowCache;
        Ok(Status::executing(true))
    }

    async fn on_broadcast(&mut self) -> Result<Status> {
        // Safety: The flow id must be allocated.
        let flow_id = self.data.flow_id.unwrap();
        let ctx = Context {
            subject: Some("Invalidate flow cache by creating flow".to_string()),
        };

        self.context
            .cache_invalidator
            .invalidate(
                &ctx,
                &[CacheIdent::CreateFlow(CreateFlow {
                    source_table_ids: self.data.source_table_ids.clone(),
                    flownodes: self.data.peers.clone(),
                })],
            )
            .await?;

        Ok(Status::done_with_output(flow_id))
    }
}

#[async_trait]
impl Procedure for CreateFlowProcedure {
    fn type_name(&self) -> &str {
        Self::TYPE_NAME
    }

    async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
        let state = &self.data.state;

        let _timer = metrics::METRIC_META_PROCEDURE_CREATE_FLOW
            .with_label_values(&[state.as_ref()])
            .start_timer();

        match state {
            CreateFlowState::Prepare => self.on_prepare().await,
            CreateFlowState::CreateFlows => self.on_flownode_create_flows().await,
            CreateFlowState::CreateMetadata => self.on_create_metadata().await,
            CreateFlowState::InvalidateFlowCache => self.on_broadcast().await,
        }
        .map_err(handle_retry_error)
    }

    fn dump(&self) -> ProcedureResult<String> {
        serde_json::to_string(&self.data).context(ToJsonSnafu)
    }

    fn lock_key(&self) -> LockKey {
        let catalog_name = &self.data.task.catalog_name;
        let flow_name = &self.data.task.flow_name;
        let sink_table_name = &self.data.task.sink_table_name;

        LockKey::new(vec![
            CatalogLock::Read(catalog_name).into(),
            TableNameLock::new(
                &sink_table_name.catalog_name,
                &sink_table_name.schema_name,
                &sink_table_name.catalog_name,
            )
            .into(),
            FlowNameLock::new(catalog_name, flow_name).into(),
        ])
    }
}

/// The state of [CreateFlowProcedure].
#[derive(Debug, Clone, Serialize, Deserialize, AsRefStr, PartialEq)]
pub enum CreateFlowState {
    /// Prepares to create the flow.
    Prepare,
    /// Creates flows on the flownode.
    CreateFlows,
    /// Invalidate flow cache.
    InvalidateFlowCache,
    /// Create metadata.
    CreateMetadata,
}

/// The serializable data.
#[derive(Debug, Serialize, Deserialize)]
pub struct CreateFlowData {
    pub(crate) cluster_id: ClusterId,
    pub(crate) state: CreateFlowState,
    pub(crate) task: CreateFlowTask,
    pub(crate) flow_id: Option<FlowId>,
    pub(crate) peers: Vec<Peer>,
    pub(crate) source_table_ids: Vec<TableId>,
    pub(crate) query_context: QueryContext,
}

impl From<&CreateFlowData> for CreateRequest {
    fn from(value: &CreateFlowData) -> Self {
        let flow_id = value.flow_id.unwrap();
        let source_table_ids = &value.source_table_ids;

        CreateRequest {
            flow_id: Some(api::v1::FlowId { id: flow_id }),
            source_table_ids: source_table_ids
                .iter()
                .map(|table_id| api::v1::TableId { id: *table_id })
                .collect_vec(),
            sink_table_name: Some(value.task.sink_table_name.clone().into()),
            // Always be true
            create_if_not_exists: true,
            expire_after: value.task.expire_after.map(|value| ExpireAfter { value }),
            comment: value.task.comment.clone(),
            sql: value.task.sql.clone(),
            flow_options: value.task.flow_options.clone(),
        }
    }
}

impl From<&CreateFlowData> for (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteValue)>) {
    fn from(value: &CreateFlowData) -> Self {
        let CreateFlowTask {
            catalog_name,
            flow_name,
            sink_table_name,
            expire_after,
            comment,
            sql,
            flow_options: options,
            ..
        } = value.task.clone();

        let flownode_ids = value
            .peers
            .iter()
            .enumerate()
            .map(|(idx, peer)| (idx as u32, peer.id))
            .collect::<BTreeMap<_, _>>();
        let flow_routes = value
            .peers
            .iter()
            .enumerate()
            .map(|(idx, peer)| (idx as u32, FlowRouteValue { peer: peer.clone() }))
            .collect::<Vec<_>>();

        (
            FlowInfoValue {
                source_table_ids: value.source_table_ids.clone(),
                sink_table_name,
                flownode_ids,
                catalog_name,
                flow_name,
                raw_sql: sql,
                expire_after,
                comment,
                options,
            },
            flow_routes,
        )
    }
}