risingwave_batch/worker_manager/
worker_node_manager.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
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
// 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::collections::{HashMap, HashSet};
use std::sync::{Arc, RwLock, RwLockReadGuard};
use std::time::Duration;

use rand::seq::SliceRandom;
use risingwave_common::bail;
use risingwave_common::catalog::OBJECT_ID_PLACEHOLDER;
use risingwave_common::hash::{WorkerSlotId, WorkerSlotMapping};
use risingwave_common::vnode_mapping::vnode_placement::place_vnode;
use risingwave_pb::common::{WorkerNode, WorkerType};

use crate::error::{BatchError, Result};

pub(crate) type FragmentId = u32;

/// `WorkerNodeManager` manages live worker nodes and table vnode mapping information.
pub struct WorkerNodeManager {
    inner: RwLock<WorkerNodeManagerInner>,
    /// Temporarily make worker invisible from serving cluster.
    worker_node_mask: Arc<RwLock<HashSet<u32>>>,
}

struct WorkerNodeManagerInner {
    worker_nodes: Vec<WorkerNode>,
    /// fragment vnode mapping info for streaming
    streaming_fragment_vnode_mapping: HashMap<FragmentId, WorkerSlotMapping>,
    /// fragment vnode mapping info for serving
    serving_fragment_vnode_mapping: HashMap<FragmentId, WorkerSlotMapping>,
}

pub type WorkerNodeManagerRef = Arc<WorkerNodeManager>;

impl Default for WorkerNodeManager {
    fn default() -> Self {
        Self::new()
    }
}

impl WorkerNodeManager {
    pub fn new() -> Self {
        Self {
            inner: RwLock::new(WorkerNodeManagerInner {
                worker_nodes: Default::default(),
                streaming_fragment_vnode_mapping: Default::default(),
                serving_fragment_vnode_mapping: Default::default(),
            }),
            worker_node_mask: Arc::new(Default::default()),
        }
    }

    /// Used in tests.
    pub fn mock(worker_nodes: Vec<WorkerNode>) -> Self {
        let inner = RwLock::new(WorkerNodeManagerInner {
            worker_nodes,
            streaming_fragment_vnode_mapping: HashMap::new(),
            serving_fragment_vnode_mapping: HashMap::new(),
        });
        Self {
            inner,
            worker_node_mask: Arc::new(Default::default()),
        }
    }

    pub fn list_worker_nodes(&self) -> Vec<WorkerNode> {
        self.inner
            .read()
            .unwrap()
            .worker_nodes
            .iter()
            .filter(|w| w.r#type() == WorkerType::ComputeNode)
            .cloned()
            .collect()
    }

    fn list_serving_worker_nodes(&self) -> Vec<WorkerNode> {
        self.list_worker_nodes()
            .into_iter()
            .filter(|w| w.property.as_ref().map_or(false, |p| p.is_serving))
            .collect()
    }

    fn list_streaming_worker_nodes(&self) -> Vec<WorkerNode> {
        self.list_worker_nodes()
            .into_iter()
            .filter(|w| w.property.as_ref().map_or(false, |p| p.is_streaming))
            .collect()
    }

    pub fn add_worker_node(&self, node: WorkerNode) {
        let mut write_guard = self.inner.write().unwrap();
        match write_guard
            .worker_nodes
            .iter_mut()
            .find(|w| w.id == node.id)
        {
            None => {
                // insert
                write_guard.worker_nodes.push(node);
            }
            Some(w) => {
                // update
                *w = node;
            }
        }
    }

    pub fn remove_worker_node(&self, node: WorkerNode) {
        let mut write_guard = self.inner.write().unwrap();
        write_guard.worker_nodes.retain(|x| x.id != node.id);
    }

    pub fn refresh(
        &self,
        nodes: Vec<WorkerNode>,
        streaming_mapping: HashMap<FragmentId, WorkerSlotMapping>,
        serving_mapping: HashMap<FragmentId, WorkerSlotMapping>,
    ) {
        let mut write_guard = self.inner.write().unwrap();
        tracing::debug!("Refresh worker nodes {:?}.", nodes);
        tracing::debug!(
            "Refresh streaming vnode mapping for fragments {:?}.",
            streaming_mapping.keys()
        );
        tracing::debug!(
            "Refresh serving vnode mapping for fragments {:?}.",
            serving_mapping.keys()
        );
        write_guard.worker_nodes = nodes;
        write_guard.streaming_fragment_vnode_mapping = streaming_mapping;
        write_guard.serving_fragment_vnode_mapping = serving_mapping;
    }

    /// If worker slot ids is empty, the scheduler may fail to schedule any task and stuck at
    /// schedule next stage. If we do not return error in this case, needs more complex control
    /// logic above. Report in this function makes the schedule root fail reason more clear.
    pub fn get_workers_by_worker_slot_ids(
        &self,
        worker_slot_ids: &[WorkerSlotId],
    ) -> Result<Vec<WorkerNode>> {
        if worker_slot_ids.is_empty() {
            return Err(BatchError::EmptyWorkerNodes);
        }

        let guard = self.inner.read().unwrap();

        let worker_index: HashMap<_, _> = guard.worker_nodes.iter().map(|w| (w.id, w)).collect();

        let mut workers = Vec::with_capacity(worker_slot_ids.len());

        for worker_slot_id in worker_slot_ids {
            match worker_index.get(&worker_slot_id.worker_id()) {
                Some(worker) => workers.push((*worker).clone()),
                None => bail!(
                    "No worker node found for worker slot id: {}",
                    worker_slot_id
                ),
            }
        }

        Ok(workers)
    }

    pub fn get_streaming_fragment_mapping(
        &self,
        fragment_id: &FragmentId,
    ) -> Result<WorkerSlotMapping> {
        self.inner
            .read()
            .unwrap()
            .streaming_fragment_vnode_mapping
            .get(fragment_id)
            .cloned()
            .ok_or_else(|| BatchError::StreamingVnodeMappingNotFound(*fragment_id))
    }

    pub fn insert_streaming_fragment_mapping(
        &self,
        fragment_id: FragmentId,
        vnode_mapping: WorkerSlotMapping,
    ) {
        self.inner
            .write()
            .unwrap()
            .streaming_fragment_vnode_mapping
            .try_insert(fragment_id, vnode_mapping)
            .unwrap();
    }

    pub fn update_streaming_fragment_mapping(
        &self,
        fragment_id: FragmentId,
        vnode_mapping: WorkerSlotMapping,
    ) {
        let mut guard = self.inner.write().unwrap();
        guard
            .streaming_fragment_vnode_mapping
            .insert(fragment_id, vnode_mapping)
            .unwrap();
    }

    pub fn remove_streaming_fragment_mapping(&self, fragment_id: &FragmentId) {
        let mut guard = self.inner.write().unwrap();

        let res = guard.streaming_fragment_vnode_mapping.remove(fragment_id);
        match &res {
            Some(_) => {}
            None if OBJECT_ID_PLACEHOLDER == *fragment_id => {
                // Do nothing for placeholder fragment.
            }
            None => {
                tracing::warn!(fragment_id, "Streaming vnode mapping not found");
            }
        };
    }

    /// Returns fragment's vnode mapping for serving.
    fn serving_fragment_mapping(&self, fragment_id: FragmentId) -> Result<WorkerSlotMapping> {
        self.inner
            .read()
            .unwrap()
            .get_serving_fragment_mapping(fragment_id)
            .ok_or_else(|| BatchError::ServingVnodeMappingNotFound(fragment_id))
    }

    pub fn set_serving_fragment_mapping(&self, mappings: HashMap<FragmentId, WorkerSlotMapping>) {
        let mut guard = self.inner.write().unwrap();
        tracing::debug!(
            "Set serving vnode mapping for fragments {:?}",
            mappings.keys()
        );
        guard.serving_fragment_vnode_mapping = mappings;
    }

    pub fn upsert_serving_fragment_mapping(
        &self,
        mappings: HashMap<FragmentId, WorkerSlotMapping>,
    ) {
        let mut guard = self.inner.write().unwrap();
        tracing::debug!(
            "Upsert serving vnode mapping for fragments {:?}",
            mappings.keys()
        );
        for (fragment_id, mapping) in mappings {
            guard
                .serving_fragment_vnode_mapping
                .insert(fragment_id, mapping);
        }
    }

    pub fn remove_serving_fragment_mapping(&self, fragment_ids: &[FragmentId]) {
        let mut guard = self.inner.write().unwrap();
        tracing::debug!(
            "Delete serving vnode mapping for fragments {:?}",
            fragment_ids
        );
        for fragment_id in fragment_ids {
            guard.serving_fragment_vnode_mapping.remove(fragment_id);
        }
    }

    fn worker_node_mask(&self) -> RwLockReadGuard<'_, HashSet<u32>> {
        self.worker_node_mask.read().unwrap()
    }

    pub fn mask_worker_node(&self, worker_node_id: u32, duration: Duration) {
        tracing::info!(
            "Mask worker node {} for {:?} temporarily",
            worker_node_id,
            duration
        );
        let mut worker_node_mask = self.worker_node_mask.write().unwrap();
        if worker_node_mask.contains(&worker_node_id) {
            return;
        }
        worker_node_mask.insert(worker_node_id);
        let worker_node_mask_ref = self.worker_node_mask.clone();
        tokio::spawn(async move {
            tokio::time::sleep(duration).await;
            worker_node_mask_ref
                .write()
                .unwrap()
                .remove(&worker_node_id);
        });
    }
}

impl WorkerNodeManagerInner {
    fn get_serving_fragment_mapping(&self, fragment_id: FragmentId) -> Option<WorkerSlotMapping> {
        self.serving_fragment_vnode_mapping
            .get(&fragment_id)
            .cloned()
    }
}

/// Selects workers for query according to `enable_barrier_read`
#[derive(Clone)]
pub struct WorkerNodeSelector {
    pub manager: WorkerNodeManagerRef,
    enable_barrier_read: bool,
}

impl WorkerNodeSelector {
    pub fn new(manager: WorkerNodeManagerRef, enable_barrier_read: bool) -> Self {
        Self {
            manager,
            enable_barrier_read,
        }
    }

    pub fn worker_node_count(&self) -> usize {
        if self.enable_barrier_read {
            self.manager.list_streaming_worker_nodes().len()
        } else {
            self.apply_worker_node_mask(self.manager.list_serving_worker_nodes())
                .len()
        }
    }

    pub fn schedule_unit_count(&self) -> usize {
        let worker_nodes = if self.enable_barrier_read {
            self.manager.list_streaming_worker_nodes()
        } else {
            self.apply_worker_node_mask(self.manager.list_serving_worker_nodes())
        };
        worker_nodes.iter().map(|node| node.parallelism()).sum()
    }

    pub fn fragment_mapping(&self, fragment_id: FragmentId) -> Result<WorkerSlotMapping> {
        if self.enable_barrier_read {
            self.manager.get_streaming_fragment_mapping(&fragment_id)
        } else {
            let mapping = (self.manager.serving_fragment_mapping(fragment_id)).or_else(|_| {
                tracing::warn!(
                    fragment_id,
                    "Serving fragment mapping not found, fall back to streaming one."
                );
                self.manager.get_streaming_fragment_mapping(&fragment_id)
            })?;

            // Filter out unavailable workers.
            if self.manager.worker_node_mask().is_empty() {
                Ok(mapping)
            } else {
                let workers = self.apply_worker_node_mask(self.manager.list_serving_worker_nodes());
                // If it's a singleton, set max_parallelism=1 for place_vnode.
                let max_parallelism = mapping.to_single().map(|_| 1);
                let masked_mapping =
                    place_vnode(Some(&mapping), &workers, max_parallelism, mapping.len())
                        .ok_or_else(|| BatchError::EmptyWorkerNodes)?;
                Ok(masked_mapping)
            }
        }
    }

    pub fn next_random_worker(&self) -> Result<WorkerNode> {
        let worker_nodes = if self.enable_barrier_read {
            self.manager.list_streaming_worker_nodes()
        } else {
            self.apply_worker_node_mask(self.manager.list_serving_worker_nodes())
        };
        worker_nodes
            .choose(&mut rand::thread_rng())
            .ok_or_else(|| BatchError::EmptyWorkerNodes)
            .map(|w| (*w).clone())
    }

    fn apply_worker_node_mask(&self, origin: Vec<WorkerNode>) -> Vec<WorkerNode> {
        let mask = self.manager.worker_node_mask();
        if origin.iter().all(|w| mask.contains(&w.id)) {
            return origin;
        }
        origin
            .into_iter()
            .filter(|w| !mask.contains(&w.id))
            .collect()
    }
}

#[cfg(test)]
mod tests {

    use risingwave_common::util::addr::HostAddr;
    use risingwave_pb::common::worker_node;
    use risingwave_pb::common::worker_node::Property;

    #[test]
    fn test_worker_node_manager() {
        use super::*;

        let manager = WorkerNodeManager::mock(vec![]);
        assert_eq!(manager.list_serving_worker_nodes().len(), 0);
        assert_eq!(manager.list_streaming_worker_nodes().len(), 0);
        assert_eq!(manager.list_worker_nodes(), vec![]);

        let worker_nodes = vec![
            WorkerNode {
                id: 1,
                r#type: WorkerType::ComputeNode as i32,
                host: Some(HostAddr::try_from("127.0.0.1:1234").unwrap().to_protobuf()),
                state: worker_node::State::Running as i32,
                parallelism: 0,
                property: Some(Property {
                    is_unschedulable: false,
                    is_serving: true,
                    is_streaming: true,
                    internal_rpc_host_addr: "".to_string(),
                }),
                transactional_id: Some(1),
                ..Default::default()
            },
            WorkerNode {
                id: 2,
                r#type: WorkerType::ComputeNode as i32,
                host: Some(HostAddr::try_from("127.0.0.1:1235").unwrap().to_protobuf()),
                state: worker_node::State::Running as i32,
                parallelism: 0,
                property: Some(Property {
                    is_unschedulable: false,
                    is_serving: true,
                    is_streaming: false,
                    internal_rpc_host_addr: "".to_string(),
                }),
                transactional_id: Some(2),
                ..Default::default()
            },
        ];
        worker_nodes
            .iter()
            .for_each(|w| manager.add_worker_node(w.clone()));
        assert_eq!(manager.list_serving_worker_nodes().len(), 2);
        assert_eq!(manager.list_streaming_worker_nodes().len(), 1);
        assert_eq!(manager.list_worker_nodes(), worker_nodes);

        manager.remove_worker_node(worker_nodes[0].clone());
        assert_eq!(manager.list_serving_worker_nodes().len(), 1);
        assert_eq!(manager.list_streaming_worker_nodes().len(), 0);
        assert_eq!(
            manager.list_worker_nodes(),
            worker_nodes.as_slice()[1..].to_vec()
        );
    }
}