risingwave_frontend/optimizer/property/
distribution.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
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
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
// 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.

//!   "A -> B" represent A satisfies B
//!                                 x
//!  only as a required property    x  can used as both required
//!                                 x  and provided property
//!                                 x
//!            ┌───┐                x┌──────┐
//!            │Any◄─────────────────┤single│
//!            └─▲─┘                x└──────┘
//!              │                  x
//!              │                  x
//!              │                  x
//!          ┌───┴────┐             x┌──────────┐
//!          │AnyShard◄──────────────┤SomeShard │
//!          └───▲────┘             x└──────────┘
//!              │                  x
//!          ┌───┴───────────┐      x┌──────────────┐ ┌──────────────┐
//!          │ShardByKey(a,b)◄───┬───┤HashShard(a,b)│ │HashShard(b,a)│
//!          └───▲──▲────────┘   │  x└──────────────┘ └┬─────────────┘
//!              │  │            │  x                  │
//!              │  │            └─────────────────────┘
//!              │  │               x
//!              │ ┌┴────────────┐  x┌────────────┐
//!              │ │ShardByKey(a)◄───┤HashShard(a)│
//!              │ └─────────────┘  x└────────────┘
//!              │                  x
//!             ┌┴────────────┐     x┌────────────┐
//!             │ShardByKey(b)◄──────┤HashShard(b)│
//!             └─────────────┘     x└────────────┘
//!                                 x
//!                                 x
use std::collections::HashMap;
use std::fmt;
use std::fmt::Debug;

use fixedbitset::FixedBitSet;
use generic::PhysicalPlanRef;
use itertools::Itertools;
use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector;
use risingwave_common::catalog::{FieldDisplay, Schema, TableId};
use risingwave_common::hash::WorkerSlotId;
use risingwave_pb::batch_plan::exchange_info::{
    ConsistentHashInfo, Distribution as PbDistribution, DistributionMode, HashInfo,
};
use risingwave_pb::batch_plan::ExchangeInfo;

use super::super::plan_node::*;
use crate::catalog::catalog_service::CatalogReader;
use crate::catalog::FragmentId;
use crate::error::Result;
use crate::optimizer::property::Order;

/// the distribution property provided by a operator.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub enum Distribution {
    /// There is only one partition. All records are placed on it.
    Single,
    /// Records are sharded into partitions, and satisfy the `AnyShard` but without any guarantee
    /// about their placement rules.
    SomeShard,
    /// Records are sharded into partitions based on the hash value of some columns, which means
    /// the records with the same hash values must be on the same partition.
    /// `usize` is the index of column used as the distribution key.
    HashShard(Vec<usize>),
    /// A special kind of provided distribution which is almost the same as
    /// [`Distribution::HashShard`], but may have different vnode mapping.
    ///
    /// It exists because the upstream MV can be scaled independently. So we use
    /// `UpstreamHashShard` to **force an exchange to be inserted**.
    ///
    /// Alternatively, [`Distribution::SomeShard`] can also be used to insert an exchange, but
    /// `UpstreamHashShard` contains distribution keys, which might be useful in some cases, e.g.,
    /// two-phase Agg. It also satisfies [`RequiredDist::ShardByKey`].
    ///
    /// `TableId` is used to represent the data distribution(`vnode_mapping`) of this
    /// `UpstreamHashShard`. The scheduler can fetch `TableId`'s corresponding `vnode_mapping` to do
    /// shuffle.
    UpstreamHashShard(Vec<usize>, TableId),
    /// Records are available on all downstream shards.
    Broadcast,
}

/// the distribution property requirement.
#[derive(Debug, Clone, PartialEq)]
pub enum RequiredDist {
    /// with any distribution
    Any,
    /// records are shard on partitions, which means every record should belong to a partition
    AnyShard,
    /// records are shard on partitions based on some keys(order-irrelevance, ShardByKey({a,b}) is
    /// equivalent with ShardByKey({b,a})), which means the records with same keys must be on
    /// the same partition, as required property only.
    ShardByKey(FixedBitSet),
    /// must be the same with the physical distribution
    PhysicalDist(Distribution),
}

impl Distribution {
    pub fn to_prost(
        &self,
        output_count: u32,
        catalog_reader: &CatalogReader,
        worker_node_manager: &WorkerNodeSelector,
    ) -> Result<ExchangeInfo> {
        let exchange_info = ExchangeInfo {
            mode: match self {
                Distribution::Single => DistributionMode::Single,
                Distribution::HashShard(_) => DistributionMode::Hash,
                // TODO: add round robin DistributionMode
                Distribution::SomeShard => DistributionMode::Single,
                Distribution::Broadcast => DistributionMode::Broadcast,
                Distribution::UpstreamHashShard(_, _) => DistributionMode::ConsistentHash,
            } as i32,
            distribution: match self {
                Distribution::Single => None,
                Distribution::HashShard(key) => {
                    assert!(
                        !key.is_empty(),
                        "hash key should not be empty, use `Single` instead"
                    );
                    Some(PbDistribution::HashInfo(HashInfo {
                        output_count,
                        key: key.iter().map(|num| *num as u32).collect(),
                    }))
                }
                // TODO: add round robin distribution
                Distribution::SomeShard => None,
                Distribution::Broadcast => None,
                Distribution::UpstreamHashShard(key, table_id) => {
                    assert!(
                        !key.is_empty(),
                        "hash key should not be empty, use `Single` instead"
                    );

                    let vnode_mapping = worker_node_manager
                        .fragment_mapping(Self::get_fragment_id(catalog_reader, table_id)?)?;

                    let worker_slot_to_id_map: HashMap<WorkerSlotId, u32> = vnode_mapping
                        .iter_unique()
                        .enumerate()
                        .map(|(i, worker_slot_id)| (worker_slot_id, i as u32))
                        .collect();

                    Some(PbDistribution::ConsistentHashInfo(ConsistentHashInfo {
                        vmap: vnode_mapping
                            .iter()
                            .map(|id| worker_slot_to_id_map[&id])
                            .collect_vec(),
                        key: key.iter().map(|num| *num as u32).collect(),
                    }))
                }
            },
        };
        Ok(exchange_info)
    }

    /// check if the distribution satisfies other required distribution
    pub fn satisfies(&self, required: &RequiredDist) -> bool {
        match required {
            RequiredDist::Any => true,
            RequiredDist::AnyShard => {
                matches!(
                    self,
                    Distribution::SomeShard
                        | Distribution::HashShard(_)
                        | Distribution::UpstreamHashShard(_, _)
                        | Distribution::Broadcast
                )
            }
            RequiredDist::ShardByKey(required_key) => match self {
                Distribution::HashShard(hash_key)
                | Distribution::UpstreamHashShard(hash_key, _) => {
                    hash_key.iter().all(|idx| required_key.contains(*idx))
                }
                _ => false,
            },
            RequiredDist::PhysicalDist(other) => self == other,
        }
    }

    /// Get distribution column indices. After optimization, only `HashShard` and `Single` are
    /// valid.
    pub fn dist_column_indices(&self) -> &[usize] {
        match self {
            Distribution::Single | Distribution::SomeShard | Distribution::Broadcast => {
                Default::default()
            }
            Distribution::HashShard(dists) | Distribution::UpstreamHashShard(dists, _) => dists,
        }
    }

    #[inline(always)]
    fn get_fragment_id(catalog_reader: &CatalogReader, table_id: &TableId) -> Result<FragmentId> {
        catalog_reader
            .read_guard()
            .get_any_table_by_id(table_id)
            .map(|table| table.fragment_id)
            .map_err(Into::into)
    }
}

impl fmt::Display for Distribution {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.write_str("[")?;
        match self {
            Self::Single => f.write_str("Single")?,
            Self::SomeShard => f.write_str("SomeShard")?,
            Self::Broadcast => f.write_str("Broadcast")?,
            Self::HashShard(vec) | Self::UpstreamHashShard(vec, _) => {
                for key in vec {
                    std::fmt::Debug::fmt(&key, f)?;
                }
            }
        }
        f.write_str("]")
    }
}

pub struct DistributionDisplay<'a> {
    pub distribution: &'a Distribution,
    pub input_schema: &'a Schema,
}

impl DistributionDisplay<'_> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        let that = self.distribution;
        match that {
            Distribution::Single => f.write_str("Single"),
            Distribution::SomeShard => f.write_str("SomeShard"),
            Distribution::Broadcast => f.write_str("Broadcast"),
            Distribution::HashShard(vec) | Distribution::UpstreamHashShard(vec, _) => {
                if let Distribution::HashShard(_) = that {
                    f.write_str("HashShard(")?;
                } else {
                    f.write_str("UpstreamHashShard(")?;
                }
                for (pos, key) in vec.iter().copied().with_position() {
                    std::fmt::Debug::fmt(
                        &FieldDisplay(self.input_schema.fields.get(key).unwrap()),
                        f,
                    )?;
                    match pos {
                        itertools::Position::First | itertools::Position::Middle => {
                            f.write_str(", ")?;
                        }
                        _ => {}
                    }
                }
                f.write_str(")")
            }
        }
    }
}

impl fmt::Debug for DistributionDisplay<'_> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        self.fmt(f)
    }
}

impl fmt::Display for DistributionDisplay<'_> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        self.fmt(f)
    }
}

impl RequiredDist {
    pub fn single() -> Self {
        Self::PhysicalDist(Distribution::Single)
    }

    pub fn shard_by_key(tot_col_num: usize, key: &[usize]) -> Self {
        let mut cols = FixedBitSet::with_capacity(tot_col_num);
        for i in key {
            cols.insert(*i);
        }
        assert!(!cols.is_clear());
        Self::ShardByKey(cols)
    }

    pub fn hash_shard(key: &[usize]) -> Self {
        assert!(!key.is_empty());
        Self::PhysicalDist(Distribution::HashShard(key.to_vec()))
    }

    pub fn enforce_if_not_satisfies(
        &self,
        mut plan: PlanRef,
        required_order: &Order,
    ) -> Result<PlanRef> {
        if let Convention::Batch = plan.convention() {
            plan = required_order.enforce_if_not_satisfies(plan)?;
        }
        if !plan.distribution().satisfies(self) {
            Ok(self.enforce(plan, required_order))
        } else {
            Ok(plan)
        }
    }

    pub fn no_shuffle(plan: PlanRef) -> PlanRef {
        match plan.convention() {
            Convention::Stream => StreamExchange::new_no_shuffle(plan).into(),
            Convention::Logical | Convention::Batch => unreachable!(),
        }
    }

    /// check if the distribution satisfies other required distribution
    pub fn satisfies(&self, required: &RequiredDist) -> bool {
        match self {
            RequiredDist::Any => matches!(required, RequiredDist::Any),
            RequiredDist::AnyShard => {
                matches!(required, RequiredDist::Any | RequiredDist::AnyShard)
            }
            RequiredDist::ShardByKey(key) => match required {
                RequiredDist::Any | RequiredDist::AnyShard => true,
                RequiredDist::ShardByKey(required_key) => key.is_subset(required_key),
                _ => false,
            },
            RequiredDist::PhysicalDist(dist) => dist.satisfies(required),
        }
    }

    pub fn enforce(&self, plan: PlanRef, required_order: &Order) -> PlanRef {
        let dist = self.to_dist();
        match plan.convention() {
            Convention::Batch => BatchExchange::new(plan, required_order.clone(), dist).into(),
            Convention::Stream => StreamExchange::new(plan, dist).into(),
            _ => unreachable!(),
        }
    }

    fn to_dist(&self) -> Distribution {
        match self {
            // all the distribution satisfy the Any, and the function can be only called by
            // `enforce_if_not_satisfies`
            RequiredDist::Any => unreachable!(),
            // TODO: add round robin distributed type
            RequiredDist::AnyShard => todo!(),
            RequiredDist::ShardByKey(required_keys) => {
                Distribution::HashShard(required_keys.ones().collect())
            }
            RequiredDist::PhysicalDist(dist) => dist.clone(),
        }
    }
}

#[cfg(test)]
mod tests {
    use super::{Distribution, RequiredDist};

    #[test]
    fn hash_shard_satisfy() {
        let d1 = Distribution::HashShard(vec![0, 1]);
        let d2 = Distribution::HashShard(vec![1, 0]);
        let d3 = Distribution::HashShard(vec![0]);
        let d4 = Distribution::HashShard(vec![1]);

        let r1 = RequiredDist::shard_by_key(2, &[0, 1]);
        let r3 = RequiredDist::shard_by_key(2, &[0]);
        let r4 = RequiredDist::shard_by_key(2, &[1]);
        assert!(d1.satisfies(&RequiredDist::PhysicalDist(d1.clone())));
        assert!(d2.satisfies(&RequiredDist::PhysicalDist(d2.clone())));
        assert!(d3.satisfies(&RequiredDist::PhysicalDist(d3.clone())));
        assert!(d4.satisfies(&RequiredDist::PhysicalDist(d4.clone())));

        assert!(!d2.satisfies(&RequiredDist::PhysicalDist(d1.clone())));
        assert!(!d3.satisfies(&RequiredDist::PhysicalDist(d1.clone())));
        assert!(!d4.satisfies(&RequiredDist::PhysicalDist(d1.clone())));

        assert!(!d1.satisfies(&RequiredDist::PhysicalDist(d3.clone())));
        assert!(!d2.satisfies(&RequiredDist::PhysicalDist(d3.clone())));
        assert!(!d1.satisfies(&RequiredDist::PhysicalDist(d4.clone())));
        assert!(!d2.satisfies(&RequiredDist::PhysicalDist(d4.clone())));

        assert!(d1.satisfies(&r1));
        assert!(d2.satisfies(&r1));
        assert!(d3.satisfies(&r1));
        assert!(d4.satisfies(&r1));

        assert!(!d1.satisfies(&r3));
        assert!(!d2.satisfies(&r3));
        assert!(d3.satisfies(&r3));
        assert!(!d4.satisfies(&r3));

        assert!(!d1.satisfies(&r4));
        assert!(!d2.satisfies(&r4));
        assert!(!d3.satisfies(&r4));
        assert!(d4.satisfies(&r4));

        assert!(r3.satisfies(&r1));
        assert!(r4.satisfies(&r1));
        assert!(!r1.satisfies(&r3));
        assert!(!r1.satisfies(&r4));
        assert!(!r3.satisfies(&r4));
        assert!(!r4.satisfies(&r3));
    }
}