risingwave_common/types/
fields.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.
14use super::DataType;
15use crate::row::OwnedRow;
16use crate::util::chunk_coalesce::DataChunkBuilder;
17
18/// A struct can implements `Fields` when if can be represented as a relational Row.
19///
20/// # Derivable
21///
22/// This trait can be automatically derived with [`#[derive(Fields)]`](derive@super::Fields).
23/// Type of the fields must implement [`WithDataType`](super::WithDataType) and [`ToOwnedDatum`](super::ToOwnedDatum).
24///
25/// ```
26/// # use risingwave_common::types::Fields;
27///
28/// #[derive(Fields)]
29/// struct Data {
30///     v1: i16,
31///     v2: i32,
32/// }
33/// ```
34///
35/// You can add `#[primary_key]` attribute to one of the fields to specify the primary key of the table.
36///
37/// ```
38/// # use risingwave_common::types::Fields;
39///
40/// #[derive(Fields)]
41/// struct Data {
42///     #[primary_key]
43///     v1: i16,
44///     v2: i32,
45/// }
46/// ```
47///
48/// If the primary key is composite, you can add `#[primary_key(...)]` attribute to the struct to specify the order of the fields.
49///
50/// ```
51/// # use risingwave_common::types::Fields;
52///
53/// #[derive(Fields)]
54/// #[primary_key(v2, v1)]
55/// struct Data {
56///     v1: i16,
57///     v2: i32,
58/// }
59/// ```
60pub trait Fields {
61    /// The primary key of the table.
62    ///
63    /// - `None` if the primary key is not applicable.
64    /// - `Some(&[])` if the primary key is empty, i.e., there'll be at most one row in the table.
65    const PRIMARY_KEY: Option<&'static [usize]>;
66
67    /// Return the schema of the struct.
68    fn fields() -> Vec<(&'static str, DataType)>;
69
70    /// Convert the struct to an `OwnedRow`.
71    fn into_owned_row(self) -> OwnedRow;
72
73    /// Create a [`DataChunkBuilder`](crate::util::chunk_coalesce::DataChunkBuilder) with the schema of the struct.
74    fn data_chunk_builder(capacity: usize) -> DataChunkBuilder {
75        DataChunkBuilder::new(
76            Self::fields().into_iter().map(|(_, ty)| ty).collect(),
77            capacity,
78        )
79    }
80}
81
82#[cfg(test)]
83mod tests {
84    use bytes::Bytes;
85
86    use super::*;
87    use crate::types::{F32, Fields, StructType, Timestamp, Timestamptz};
88
89    #[test]
90    #[allow(dead_code)]
91    fn test_macro() {
92        #[derive(Fields)]
93        struct Sub {
94            v12: Timestamptz,
95            v13: Bytes,
96        }
97
98        #[derive(Fields)]
99        struct Data {
100            v1: i16,
101            v2: std::primitive::i32,
102            v3: bool,
103            v4: f32,
104            v5: F32,
105            v6: Option<f64>,
106            v7: Vec<u8>,
107            v8: std::vec::Vec<i16>,
108            v9: Option<Vec<i64>>,
109            v10: std::option::Option<Vec<Option<F32>>>,
110            v11: Timestamp,
111            v14: Sub,
112        }
113
114        assert_eq!(
115            Data::fields(),
116            vec![
117                ("v1", DataType::Int16),
118                ("v2", DataType::Int32),
119                ("v3", DataType::Boolean),
120                ("v4", DataType::Float32),
121                ("v5", DataType::Float32),
122                ("v6", DataType::Float64),
123                ("v7", DataType::Bytea),
124                ("v8", DataType::List(Box::new(DataType::Int16))),
125                ("v9", DataType::List(Box::new(DataType::Int64))),
126                ("v10", DataType::List(Box::new(DataType::Float32))),
127                ("v11", DataType::Timestamp),
128                (
129                    "v14",
130                    DataType::Struct(StructType::new(vec![
131                        ("v12", DataType::Timestamptz),
132                        ("v13", DataType::Bytea)
133                    ]))
134                )
135            ]
136        )
137    }
138}