risingwave_common/hash/consistent_hash/vnode_count.rs
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
// Copyright 2024 RisingWave Labs
//
// 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.
use std::num::NonZeroUsize;
use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType;
use super::vnode::VirtualNode;
/// The different cases of `maybe_vnode_count` field in the protobuf message.
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Default)]
pub enum VnodeCount {
/// The field is a placeholder and has to be filled first before using it.
#[default]
Placeholder,
/// The field is set to a specific value.
Set(NonZeroUsize),
/// The field is unset because the table/fragment is persisted as hash-distributed
/// in an older version.
CompatHash,
/// The field is unset because the table/fragment is persisted as singleton
/// in an older version.
CompatSingleton,
}
impl VnodeCount {
/// Creates a `VnodeCount` set to the given value.
pub fn set(v: impl TryInto<usize> + Copy + std::fmt::Debug) -> Self {
let v = (v.try_into().ok())
.filter(|v| (1..=VirtualNode::MAX_COUNT).contains(v))
.unwrap_or_else(|| panic!("invalid vnode count {v:?}"));
VnodeCount::Set(NonZeroUsize::new(v).unwrap())
}
/// Creates a `VnodeCount` set to the value for testing.
///
/// Equivalent to `VnodeCount::set(VirtualNode::COUNT_FOR_TEST)`.
pub fn for_test() -> Self {
Self::set(VirtualNode::COUNT_FOR_TEST)
}
/// Converts from protobuf representation of `maybe_vnode_count`. If the value is not set,
/// call `compat_is_singleton` to determine whether it should be treated as a singleton
/// when it comes to backward compatibility.
fn from_protobuf(v: Option<u32>, compat_is_singleton: impl FnOnce() -> bool) -> Self {
match v {
Some(0) => VnodeCount::Placeholder,
Some(v) => VnodeCount::set(v as usize),
None => {
if compat_is_singleton() {
VnodeCount::CompatSingleton
} else {
VnodeCount::CompatHash
}
}
}
}
/// Converts to protobuf representation for `maybe_vnode_count`.
pub fn to_protobuf(self) -> Option<u32> {
// Effectively fills the compatibility cases with values.
self.value_opt()
.map_or(Some(0) /* placeholder */, |v| Some(v as _))
}
/// Returns the value of the vnode count, or `None` if it's a placeholder.
pub fn value_opt(self) -> Option<usize> {
match self {
VnodeCount::Placeholder => None,
VnodeCount::Set(v) => Some(v.get()),
VnodeCount::CompatHash => Some(VirtualNode::COUNT_FOR_COMPAT),
VnodeCount::CompatSingleton => Some(1),
}
}
/// Returns the value of the vnode count. Panics if it's a placeholder.
pub fn value(self) -> usize {
self.value_opt()
.expect("vnode count is a placeholder that must be filled by the meta service first")
}
}
/// A trait for checking whether a table/fragment is a singleton.
pub trait IsSingleton {
/// Returns `true` if the table/fragment is a singleton.
///
/// By singleton, we mean that all data read from or written to the storage belongs to
/// the only `SINGLETON_VNODE`. This must be consistent with the behavior of
/// [`TableDistribution`](crate::hash::table_distribution::TableDistribution::new).
/// As a result, the `vnode_count` of such table/fragment can be `1`.
fn is_singleton(&self) -> bool;
}
/// A trait for accessing the vnode count field with backward compatibility.
///
/// # `maybe_`?
///
/// The reason why there's a `maybe_` prefix on the protobuf field is that, a getter
/// method with the same name as the field will be generated for `prost` structs.
/// Directly naming it `vnode_count` will lead to the method `vnode_count()` returning
/// `0` when the field is unset, which can be misleading sometimes.
///
/// Instead, we name the field as `maybe_vnode_count` and provide the method `vnode_count`
/// through this trait, ensuring that backward compatibility is handled properly.
pub trait VnodeCountCompat {
/// Get the `maybe_vnode_count` field.
fn vnode_count_inner(&self) -> VnodeCount;
/// Returns the vnode count if it's set. Otherwise, returns [`VirtualNode::COUNT_FOR_COMPAT`]
/// for distributed tables/fragments, and `1` for singleton tables/fragments, for backward
/// compatibility. Panics if the field is a placeholder.
///
/// See the documentation on the field of the implementing type for more details.
fn vnode_count(&self) -> usize {
self.vnode_count_inner().value()
}
}
impl IsSingleton for risingwave_pb::catalog::Table {
fn is_singleton(&self) -> bool {
self.distribution_key.is_empty()
&& self.dist_key_in_pk.is_empty()
&& self.vnode_col_index.is_none()
}
}
impl VnodeCountCompat for risingwave_pb::catalog::Table {
fn vnode_count_inner(&self) -> VnodeCount {
VnodeCount::from_protobuf(self.maybe_vnode_count, || self.is_singleton())
}
}
impl IsSingleton for risingwave_pb::plan_common::StorageTableDesc {
fn is_singleton(&self) -> bool {
self.dist_key_in_pk_indices.is_empty() && self.vnode_col_idx_in_pk.is_none()
}
}
impl VnodeCountCompat for risingwave_pb::plan_common::StorageTableDesc {
fn vnode_count_inner(&self) -> VnodeCount {
VnodeCount::from_protobuf(self.maybe_vnode_count, || self.is_singleton())
}
}
impl IsSingleton for risingwave_pb::meta::table_fragments::Fragment {
fn is_singleton(&self) -> bool {
matches!(self.distribution_type(), FragmentDistributionType::Single)
}
}
impl VnodeCountCompat for risingwave_pb::meta::table_fragments::Fragment {
fn vnode_count_inner(&self) -> VnodeCount {
VnodeCount::from_protobuf(self.maybe_vnode_count, || self.is_singleton())
}
}