risingwave_common/hash/consistent_hash/
vnode_count.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::NonZeroUsize;
16
17use super::vnode::VirtualNode;
18
19/// The different cases of `maybe_vnode_count` field in the protobuf message.
20#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Default)]
21pub enum VnodeCount {
22    /// The field is a placeholder and has to be filled first before using it.
23    #[default]
24    Placeholder,
25    /// The table/fragment is a singleton, thus the value should always be interpreted as `1`.
26    Singleton,
27    /// The field is set to a specific value.
28    Set(NonZeroUsize),
29    /// The field is unset because the table/fragment is persisted as hash-distributed
30    /// in an older version.
31    CompatHash,
32}
33
34impl VnodeCount {
35    /// Creates a `VnodeCount` set to the given value.
36    pub fn set(v: impl TryInto<usize> + Copy + std::fmt::Debug) -> Self {
37        let v = v.try_into().ok();
38        if v == Some(0) {
39            return VnodeCount::Placeholder;
40        }
41
42        let v = v
43            .filter(|v| (1..=VirtualNode::MAX_COUNT).contains(v))
44            .unwrap_or_else(|| panic!("invalid vnode count {v:?}"));
45
46        VnodeCount::Set(NonZeroUsize::new(v).unwrap())
47    }
48
49    /// Creates a `VnodeCount` set to the value for testing.
50    ///
51    /// Equivalent to `VnodeCount::set(VirtualNode::COUNT_FOR_TEST)`.
52    pub fn for_test() -> Self {
53        Self::set(VirtualNode::COUNT_FOR_TEST)
54    }
55
56    /// Converts from protobuf representation of `maybe_vnode_count`.
57    ///
58    /// The value will be ignored if `is_singleton` returns `true`.
59    pub fn from_protobuf(v: Option<u32>, is_singleton: impl FnOnce() -> bool) -> Self {
60        match v {
61            Some(0) => VnodeCount::Placeholder,
62            _ => {
63                if is_singleton() {
64                    if let Some(v) = v
65                        && v != 1
66                    {
67                        tracing::debug!(
68                            vnode_count = v,
69                            "singleton has vnode count set to non-1, \
70                            ignoring as it could be due to backward compatibility"
71                        );
72                    }
73                    VnodeCount::Singleton
74                } else {
75                    v.map_or(VnodeCount::CompatHash, VnodeCount::set)
76                }
77            }
78        }
79    }
80
81    /// Converts to protobuf representation for `maybe_vnode_count`.
82    pub fn to_protobuf(self) -> Option<u32> {
83        // Effectively fills the compatibility cases with values.
84        self.value_opt()
85            .map_or(Some(0) /* placeholder */, |v| Some(v as _))
86    }
87
88    /// Returns the value of the vnode count, or `None` if it's a placeholder.
89    pub fn value_opt(self) -> Option<usize> {
90        match self {
91            VnodeCount::Placeholder => None,
92            VnodeCount::Singleton => Some(1),
93            VnodeCount::Set(v) => Some(v.get()),
94            VnodeCount::CompatHash => Some(VirtualNode::COUNT_FOR_COMPAT),
95        }
96    }
97
98    /// Returns the value of the vnode count. Panics if it's a placeholder.
99    pub fn value(self) -> usize {
100        self.value_opt()
101            .expect("vnode count is a placeholder that must be filled by the meta service first")
102    }
103}
104
105/// A trait for checking whether a table/fragment is a singleton.
106pub trait IsSingleton {
107    /// Returns `true` if the table/fragment is a singleton.
108    ///
109    /// By singleton, we mean that all data read from or written to the storage belongs to
110    /// the only `SINGLETON_VNODE`. This must be consistent with the behavior of
111    /// [`TableDistribution`](crate::hash::table_distribution::TableDistribution::new).
112    /// As a result, the `vnode_count` of such table/fragment can be `1`.
113    fn is_singleton(&self) -> bool;
114}
115
116/// A trait for accessing the vnode count field with backward compatibility.
117///
118/// # `maybe_`?
119///
120/// The reason why there's a `maybe_` prefix on the protobuf field is that, a getter
121/// method with the same name as the field will be generated for `prost` structs.
122/// Directly naming it `vnode_count` will lead to the method `vnode_count()` returning
123/// `0` when the field is unset, which can be misleading sometimes.
124///
125/// Instead, we name the field as `maybe_vnode_count` and provide the method `vnode_count`
126/// through this trait, ensuring that backward compatibility is handled properly.
127pub trait VnodeCountCompat {
128    /// Get the `maybe_vnode_count` field.
129    fn vnode_count_inner(&self) -> VnodeCount;
130
131    /// Returns the vnode count if it's set. Otherwise, returns [`VirtualNode::COUNT_FOR_COMPAT`]
132    /// for distributed tables/fragments, and `1` for singleton tables/fragments, for backward
133    /// compatibility. Panics if the field is a placeholder.
134    ///
135    /// See the documentation on the field of the implementing type for more details.
136    fn vnode_count(&self) -> usize {
137        self.vnode_count_inner().value()
138    }
139}
140
141impl IsSingleton for risingwave_pb::catalog::Table {
142    fn is_singleton(&self) -> bool {
143        self.distribution_key.is_empty()
144            && self.dist_key_in_pk.is_empty()
145            && self.vnode_col_index.is_none()
146    }
147}
148impl VnodeCountCompat for risingwave_pb::catalog::Table {
149    fn vnode_count_inner(&self) -> VnodeCount {
150        VnodeCount::from_protobuf(self.maybe_vnode_count, || self.is_singleton())
151    }
152}
153
154impl IsSingleton for risingwave_pb::plan_common::StorageTableDesc {
155    fn is_singleton(&self) -> bool {
156        self.dist_key_in_pk_indices.is_empty() && self.vnode_col_idx_in_pk.is_none()
157    }
158}
159impl VnodeCountCompat for risingwave_pb::plan_common::StorageTableDesc {
160    fn vnode_count_inner(&self) -> VnodeCount {
161        VnodeCount::from_protobuf(self.maybe_vnode_count, || self.is_singleton())
162    }
163}