risingwave_frontend/optimizer/plan_node/
stream_vector_index_write.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 std::num::NonZeroU32;
16
17use itertools::Itertools;
18use pretty_xmlish::{Pretty, XmlNode};
19use risingwave_common::catalog::{
20    ColumnCatalog, ConflictBehavior, CreateType, Engine, OBJECT_ID_PLACEHOLDER, StreamJobStatus,
21    TableId,
22};
23use risingwave_common::hash::VnodeCount;
24use risingwave_common::util::iter_util::ZipEqFast;
25use risingwave_pb::catalog::PbVectorIndexInfo;
26use risingwave_pb::stream_plan::stream_node::PbNodeBody;
27
28use crate::TableCatalog;
29use crate::catalog::table_catalog::TableType;
30use crate::catalog::{DatabaseId, SchemaId};
31use crate::error::ErrorCode;
32use crate::optimizer::StreamOptimizedLogicalPlanRoot;
33use crate::optimizer::plan_node::derive::{derive_columns, derive_pk};
34use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
35use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef};
36use crate::optimizer::plan_node::stream::StreamPlanNodeMetadata;
37use crate::optimizer::plan_node::utils::{Distill, childless_record, plan_can_use_background_ddl};
38use crate::optimizer::plan_node::{
39    ExprRewritable, PlanBase, PlanNodeMeta, PlanTreeNodeUnary, Stream, StreamNode,
40    StreamPlanRef as PlanRef, reorganize_elements_id,
41};
42use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist, StreamKind};
43use crate::stream_fragmenter::BuildFragmentGraphState;
44
45#[derive(Debug, Clone, PartialEq, Eq, Hash)]
46pub struct StreamVectorIndexWrite {
47    pub base: PlanBase<Stream>,
48    /// Child of Vector Index Write plan
49    input: PlanRef,
50    table: TableCatalog,
51}
52
53impl StreamVectorIndexWrite {
54    fn new(input: PlanRef, table: TableCatalog) -> crate::error::Result<Self> {
55        if input.stream_kind() != StreamKind::AppendOnly {
56            return Err(ErrorCode::NotSupported(
57                "cannot create vector index on non-append-only workflow".to_owned(),
58                "try create vector index on append only workflow".to_owned(),
59            )
60            .into());
61        }
62        let base = PlanBase::new_stream(
63            input.ctx(),
64            input.schema().clone(),
65            Some(table.stream_key.clone()),
66            input.functional_dependency().clone(),
67            input.distribution().clone(),
68            StreamKind::AppendOnly,
69            input.emit_on_window_close(),
70            input.watermark_columns().clone(),
71            input.columns_monotonicity().clone(),
72        );
73        Ok(Self { base, input, table })
74    }
75
76    pub fn create(
77        StreamOptimizedLogicalPlanRoot {
78            plan: input,
79            required_order: user_order_by,
80            out_fields: user_cols,
81            out_names,
82            ..
83        }: StreamOptimizedLogicalPlanRoot,
84        name: String,
85        database_id: DatabaseId,
86        schema_id: SchemaId,
87        definition: String,
88        cardinality: Cardinality,
89        retention_seconds: Option<NonZeroU32>,
90        vector_index_info: PbVectorIndexInfo,
91    ) -> crate::error::Result<Self> {
92        let input = RequiredDist::PhysicalDist(Distribution::Single)
93            .streaming_enforce_if_not_satisfies(input)?;
94        // the hidden column name might refer some expr id
95        let input = reorganize_elements_id(input);
96        let columns = derive_columns(input.schema(), out_names, &user_cols)?;
97
98        let create_type = if input.ctx().session_ctx().config().background_ddl()
99            && plan_can_use_background_ddl(&input)
100        {
101            CreateType::Background
102        } else {
103            CreateType::Foreground
104        };
105
106        let table = Self::derive_table_catalog(
107            input.clone(),
108            name,
109            database_id,
110            schema_id,
111            user_order_by,
112            columns,
113            definition,
114            cardinality,
115            retention_seconds,
116            create_type,
117            vector_index_info,
118        )?;
119
120        Self::new(input, table)
121    }
122
123    #[expect(clippy::too_many_arguments)]
124    fn derive_table_catalog(
125        rewritten_input: PlanRef,
126        name: String,
127        database_id: DatabaseId,
128        schema_id: SchemaId,
129        user_order_by: Order,
130        columns: Vec<ColumnCatalog>,
131        definition: String,
132        cardinality: Cardinality,
133        retention_seconds: Option<NonZeroU32>,
134        create_type: CreateType,
135        vector_index_info: PbVectorIndexInfo,
136    ) -> crate::error::Result<TableCatalog> {
137        let input = rewritten_input;
138
139        let value_indices = (0..columns.len()).collect_vec();
140        assert_eq!(input.distribution(), &Distribution::Single);
141        let distribution_key = vec![];
142        let append_only = input.append_only();
143        // TODO(rc): In `TableCatalog` we still use `FixedBitSet` for watermark columns, ignoring the watermark group information.
144        // We will record the watermark group information in `TableCatalog` in the future. For now, let's flatten the watermark columns.
145        let watermark_columns = input.watermark_columns().indices().collect();
146
147        let (table_pk, stream_key) = derive_pk(input, user_order_by, &columns);
148        // assert: `stream_key` is a subset of `table_pk`
149
150        let read_prefix_len_hint = table_pk.len();
151        // We don't need to fill in table id for table in frontend. The id will be generated on
152        // meta catalog service.
153        Ok(TableCatalog {
154            id: TableId::placeholder(),
155            schema_id,
156            database_id,
157            associated_source_id: None,
158            name,
159            columns,
160            pk: table_pk,
161            stream_key,
162            distribution_key,
163            table_type: TableType::VectorIndex,
164            append_only,
165            owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID,
166            fragment_id: OBJECT_ID_PLACEHOLDER,
167            dml_fragment_id: None,
168            vnode_col_index: None,
169            row_id_index: None,
170            value_indices,
171            definition,
172            conflict_behavior: ConflictBehavior::NoCheck,
173            version_column_indices: vec![],
174            read_prefix_len_hint,
175            version: None,
176            watermark_columns,
177            dist_key_in_pk: vec![],
178            cardinality,
179            created_at_epoch: None,
180            initialized_at_epoch: None,
181            cleaned_by_watermark: false,
182            create_type,
183            stream_job_status: StreamJobStatus::Creating,
184            description: None,
185            initialized_at_cluster_version: None,
186            created_at_cluster_version: None,
187            retention_seconds: retention_seconds.map(|i| i.into()),
188            cdc_table_id: None,
189            vnode_count: VnodeCount::Singleton, // will be filled in by the meta service later
190            webhook_info: None,
191            job_id: None,
192            engine: Engine::Hummock,
193            clean_watermark_index_in_pk: None,
194            refreshable: false,
195            vector_index_info: Some(vector_index_info),
196            cdc_table_type: None,
197        })
198    }
199
200    pub(crate) fn table(&self) -> &TableCatalog {
201        &self.table
202    }
203}
204
205impl Distill for StreamVectorIndexWrite {
206    fn distill<'a>(&self) -> XmlNode<'a> {
207        let table = self.table();
208
209        let vector_column_name = Pretty::from(table.columns[0].name().to_owned());
210
211        let column_names = (table.columns.iter())
212            .map(|col| col.name_with_hidden().to_string())
213            .map(Pretty::from)
214            .collect();
215
216        let stream_key = (table.stream_key.iter())
217            .map(|&k| table.columns[k].name().to_owned())
218            .map(Pretty::from)
219            .collect();
220
221        let vec = vec![
222            ("vector_column", vector_column_name),
223            ("columns", Pretty::Array(column_names)),
224            ("stream_key", Pretty::Array(stream_key)),
225        ];
226
227        childless_record("StreamVectorIndexWrite", vec)
228    }
229}
230
231impl PlanTreeNodeUnary<Stream> for StreamVectorIndexWrite {
232    fn input(&self) -> PlanRef {
233        self.input.clone()
234    }
235
236    fn clone_with_input(&self, input: PlanRef) -> Self {
237        let new = Self::new(input, self.table().clone()).unwrap();
238        new.base
239            .schema()
240            .fields
241            .iter()
242            .zip_eq_fast(self.base.schema().fields.iter())
243            .for_each(|(a, b)| {
244                assert_eq!(a.data_type, b.data_type);
245            });
246        assert_eq!(new.plan_base().stream_key(), self.plan_base().stream_key());
247        new
248    }
249}
250
251impl_plan_tree_node_for_unary! { Stream, StreamVectorIndexWrite }
252
253impl StreamNode for StreamVectorIndexWrite {
254    fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody {
255        use risingwave_pb::stream_plan::*;
256
257        let table = self.table();
258        PbNodeBody::VectorIndexWrite(Box::new(VectorIndexWriteNode {
259            table: Some(table.to_prost()),
260        }))
261    }
262}
263
264impl ExprRewritable<Stream> for StreamVectorIndexWrite {}
265
266impl ExprVisitable for StreamVectorIndexWrite {}