risingwave_batch/task/
task_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
// 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::{hash_map, HashMap};
use std::net::SocketAddr;
use std::sync::Arc;

use anyhow::Context;
use parking_lot::Mutex;
use risingwave_common::config::BatchConfig;
use risingwave_common::memory::MemoryContext;
use risingwave_common::util::runtime::BackgroundShutdownRuntime;
use risingwave_common::util::tracing::TracingContext;
use risingwave_pb::batch_plan::{PbTaskId, PbTaskOutputId, PlanFragment};
use risingwave_pb::common::BatchQueryEpoch;
use risingwave_pb::plan_common::ExprContext;
use risingwave_pb::task_service::task_info_response::TaskStatus;
use risingwave_pb::task_service::{GetDataResponse, TaskInfoResponse};
use tokio::sync::mpsc::Sender;
use tonic::Status;

use crate::error::Result;
use crate::monitor::BatchManagerMetrics;
use crate::rpc::service::exchange::GrpcExchangeWriter;
use crate::task::{
    BatchTaskExecution, ComputeNodeContext, StateReporter, TaskId, TaskOutput, TaskOutputId,
};

/// `BatchManager` is responsible for managing all batch tasks.
#[derive(Clone)]
pub struct BatchManager {
    /// Every task id has a corresponding task execution.
    tasks: Arc<Mutex<HashMap<TaskId, Arc<BatchTaskExecution<ComputeNodeContext>>>>>,

    /// Runtime for the batch manager.
    runtime: Arc<BackgroundShutdownRuntime>,

    /// Batch configuration
    config: BatchConfig,

    /// Memory context used for batch tasks in cn.
    mem_context: MemoryContext,

    /// Metrics for batch manager.
    metrics: Arc<BatchManagerMetrics>,
}

impl BatchManager {
    pub fn new(config: BatchConfig, metrics: Arc<BatchManagerMetrics>, mem_limit: u64) -> Self {
        let runtime = {
            let mut builder = tokio::runtime::Builder::new_multi_thread();
            if let Some(worker_threads_num) = config.worker_threads_num {
                builder.worker_threads(worker_threads_num);
            }
            builder
                .thread_name("rw-batch")
                .enable_all()
                .build()
                .unwrap()
        };

        let mem_context = MemoryContext::root(metrics.batch_total_mem.clone(), mem_limit);
        BatchManager {
            tasks: Arc::new(Mutex::new(HashMap::new())),
            runtime: Arc::new(runtime.into()),
            config,
            metrics,
            mem_context,
        }
    }

    pub(crate) fn metrics(&self) -> Arc<BatchManagerMetrics> {
        self.metrics.clone()
    }

    pub fn memory_context_ref(&self) -> MemoryContext {
        self.mem_context.clone()
    }

    pub async fn fire_task(
        self: &Arc<Self>,
        tid: &PbTaskId,
        plan: PlanFragment,
        epoch: BatchQueryEpoch,
        context: ComputeNodeContext,
        state_reporter: StateReporter,
        tracing_context: TracingContext,
        expr_context: ExprContext,
    ) -> Result<()> {
        trace!("Received task id: {:?}, plan: {:?}", tid, plan);
        let task = BatchTaskExecution::new(tid, plan, context, epoch, self.runtime())?;
        let task_id = task.get_task_id().clone();
        let task = Arc::new(task);
        // Here the task id insert into self.tasks is put in front of `.async_execute`, cuz when
        // send `TaskStatus::Running` in `.async_execute`, the query runner may schedule next stage,
        // it's possible do not found parent task id in theory.
        let ret = if let hash_map::Entry::Vacant(e) = self.tasks.lock().entry(task_id.clone()) {
            e.insert(task.clone());

            let this = self.clone();
            let task_id = task_id.clone();
            let state_reporter = state_reporter.clone();
            let heartbeat_join_handle = self.runtime.spawn(async move {
                this.start_task_heartbeat(state_reporter, task_id).await;
            });
            task.set_heartbeat_join_handle(heartbeat_join_handle);

            Ok(())
        } else {
            bail!(
                "can not create duplicate task with the same id: {:?}",
                task_id,
            );
        };
        task.async_execute(Some(state_reporter), tracing_context, expr_context)
            .await
            .inspect_err(|_| {
                self.cancel_task(&task_id.to_prost());
            })?;
        ret
    }

    #[cfg(test)]
    async fn fire_task_for_test(
        self: &Arc<Self>,
        tid: &PbTaskId,
        plan: PlanFragment,
    ) -> Result<()> {
        use risingwave_hummock_sdk::test_batch_query_epoch;

        self.fire_task(
            tid,
            plan,
            test_batch_query_epoch(),
            ComputeNodeContext::for_test(),
            StateReporter::new_with_test(),
            TracingContext::none(),
            ExprContext {
                time_zone: "UTC".to_string(),
            },
        )
        .await
    }

    async fn start_task_heartbeat(&self, mut state_reporter: StateReporter, task_id: TaskId) {
        let _metric_guard = scopeguard::guard((), |_| {
            tracing::debug!("heartbeat worker for task {:?} stopped", task_id);
            self.metrics.batch_heartbeat_worker_num.dec();
        });
        tracing::debug!("heartbeat worker for task {:?} started", task_id);
        self.metrics.batch_heartbeat_worker_num.inc();
        // The heartbeat is to ensure task cancellation when frontend's cancellation request fails
        // to reach compute node (for any reason like RPC fails, frontend crashes).
        let mut heartbeat_interval = tokio::time::interval(core::time::Duration::from_secs(60));
        heartbeat_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay);
        heartbeat_interval.reset();
        loop {
            heartbeat_interval.tick().await;
            if !self.tasks.lock().contains_key(&task_id) {
                break;
            }
            if state_reporter
                .send(TaskInfoResponse {
                    task_id: Some(task_id.to_prost()),
                    task_status: TaskStatus::Ping.into(),
                    error_message: "".to_string(),
                })
                .await
                .is_err()
            {
                tracing::warn!("try to cancel task {:?} due to heartbeat", task_id);
                // Task may have been cancelled, but it's fine to `cancel_task` again.
                self.cancel_task(&task_id.to_prost());
                break;
            }
        }
    }

    pub fn get_data(
        &self,
        tx: Sender<std::result::Result<GetDataResponse, Status>>,
        peer_addr: SocketAddr,
        pb_task_output_id: &PbTaskOutputId,
    ) -> Result<()> {
        let task_id = TaskOutputId::try_from(pb_task_output_id)?;
        tracing::debug!(target: "events::compute::exchange", peer_addr = %peer_addr, from = ?task_id, "serve exchange RPC");
        let mut task_output = self.take_output(pb_task_output_id)?;
        self.runtime.spawn(async move {
            let mut writer = GrpcExchangeWriter::new(tx.clone());
            match task_output.take_data(&mut writer).await {
                Ok(_) => {
                    tracing::trace!(
                        from = ?task_id,
                        "exchanged {} chunks",
                        writer.written_chunks(),
                    );
                    Ok(())
                }
                Err(e) => tx.send(Err(e.into())).await,
            }
        });
        Ok(())
    }

    pub fn take_output(&self, output_id: &PbTaskOutputId) -> Result<TaskOutput> {
        let task_id = TaskId::from(output_id.get_task_id()?);
        self.tasks
            .lock()
            .get(&task_id)
            .with_context(|| format!("task {:?} not found", task_id))?
            .get_task_output(output_id)
    }

    pub fn cancel_task(&self, sid: &PbTaskId) {
        let sid = TaskId::from(sid);
        match self.tasks.lock().remove(&sid) {
            Some(task) => {
                tracing::trace!("Removed task: {:?}", task.get_task_id());
                // Use `cancel` rather than `abort` here since this is not an error which should be
                // propagated to upstream.
                task.cancel();
                if let Some(heartbeat_join_handle) = task.heartbeat_join_handle() {
                    heartbeat_join_handle.abort();
                }
            }
            None => {
                warn!("Task {:?} not found for cancel", sid)
            }
        };
    }

    /// Returns error if task is not running.
    pub fn check_if_task_running(&self, task_id: &TaskId) -> Result<()> {
        match self.tasks.lock().get(task_id) {
            Some(task) => task.check_if_running(),
            None => bail!("task {:?} not found", task_id),
        }
    }

    pub fn check_if_task_aborted(&self, task_id: &TaskId) -> Result<bool> {
        match self.tasks.lock().get(task_id) {
            Some(task) => task.check_if_aborted(),
            None => bail!("task {:?} not found", task_id),
        }
    }

    #[cfg(test)]
    async fn wait_until_task_aborted(&self, task_id: &TaskId) -> Result<()> {
        use std::time::Duration;
        loop {
            match self.tasks.lock().get(task_id) {
                Some(task) => {
                    let ret = task.check_if_aborted();
                    match ret {
                        Ok(true) => return Ok(()),
                        Ok(false) => {}
                        Err(err) => return Err(err),
                    }
                }
                None => bail!("task {:?} not found", task_id),
            }
            tokio::time::sleep(Duration::from_millis(100)).await
        }
    }

    pub fn runtime(&self) -> Arc<BackgroundShutdownRuntime> {
        self.runtime.clone()
    }

    pub fn config(&self) -> &BatchConfig {
        &self.config
    }
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;

    use risingwave_common::config::BatchConfig;
    use risingwave_pb::batch_plan::exchange_info::DistributionMode;
    use risingwave_pb::batch_plan::plan_node::NodeBody;
    use risingwave_pb::batch_plan::{
        ExchangeInfo, PbTaskId, PbTaskOutputId, PlanFragment, PlanNode, ValuesNode,
    };

    use crate::monitor::BatchManagerMetrics;
    use crate::task::{BatchManager, TaskId};

    #[tokio::test]
    async fn test_task_not_found() {
        let manager = Arc::new(BatchManager::new(
            BatchConfig::default(),
            BatchManagerMetrics::for_test(),
            u64::MAX,
        ));
        let task_id = TaskId {
            task_id: 0,
            stage_id: 0,
            query_id: "abc".to_string(),
        };

        let error = manager.check_if_task_running(&task_id).unwrap_err();
        assert!(error.to_string().contains("not found"), "{:?}", error);

        let output_id = PbTaskOutputId {
            task_id: Some(risingwave_pb::batch_plan::TaskId {
                stage_id: 0,
                task_id: 0,
                query_id: "".to_owned(),
            }),
            output_id: 0,
        };
        let error = manager.take_output(&output_id).unwrap_err();
        assert!(error.to_string().contains("not found"), "{:?}", error);
    }

    #[tokio::test]
    async fn test_task_id_conflict() {
        let manager = Arc::new(BatchManager::new(
            BatchConfig::default(),
            BatchManagerMetrics::for_test(),
            u64::MAX,
        ));
        let plan = PlanFragment {
            root: Some(PlanNode {
                children: vec![],
                identity: "".to_string(),
                node_body: Some(NodeBody::Values(ValuesNode {
                    tuples: vec![],
                    fields: vec![],
                })),
            }),
            exchange_info: Some(ExchangeInfo {
                mode: DistributionMode::Single as i32,
                distribution: None,
            }),
        };
        let task_id = PbTaskId {
            query_id: "".to_string(),
            stage_id: 0,
            task_id: 0,
        };
        manager
            .fire_task_for_test(&task_id, plan.clone())
            .await
            .unwrap();
        let err = manager
            .fire_task_for_test(&task_id, plan)
            .await
            .unwrap_err();
        assert!(err
            .to_string()
            .contains("can not create duplicate task with the same id"));
    }

    #[tokio::test]
    // see https://github.com/risingwavelabs/risingwave/issues/11979
    #[ignore]
    async fn test_task_cancel_for_busy_loop() {
        let manager = Arc::new(BatchManager::new(
            BatchConfig::default(),
            BatchManagerMetrics::for_test(),
            u64::MAX,
        ));
        let plan = PlanFragment {
            root: Some(PlanNode {
                children: vec![],
                identity: "".to_string(),
                node_body: Some(NodeBody::BusyLoopExecutor(true)),
            }),
            exchange_info: Some(ExchangeInfo {
                mode: DistributionMode::Single as i32,
                distribution: None,
            }),
        };
        let task_id = PbTaskId {
            query_id: "".to_string(),
            stage_id: 0,
            task_id: 0,
        };
        manager.fire_task_for_test(&task_id, plan).await.unwrap();
        manager.cancel_task(&task_id);
        let task_id = TaskId::from(&task_id);
        assert!(!manager.tasks.lock().contains_key(&task_id));
    }

    #[tokio::test]
    // see https://github.com/risingwavelabs/risingwave/issues/11979
    #[ignore]
    async fn test_task_abort_for_busy_loop() {
        let manager = Arc::new(BatchManager::new(
            BatchConfig::default(),
            BatchManagerMetrics::for_test(),
            u64::MAX,
        ));
        let plan = PlanFragment {
            root: Some(PlanNode {
                children: vec![],
                identity: "".to_string(),
                node_body: Some(NodeBody::BusyLoopExecutor(true)),
            }),
            exchange_info: Some(ExchangeInfo {
                mode: DistributionMode::Single as i32,
                distribution: None,
            }),
        };
        let task_id = PbTaskId {
            query_id: "".to_string(),
            stage_id: 0,
            task_id: 0,
        };
        manager.fire_task_for_test(&task_id, plan).await.unwrap();
        let task_id = TaskId::from(&task_id);
        manager
            .tasks
            .lock()
            .get(&task_id)
            .unwrap()
            .abort("Abort Test".to_owned());
        assert!(manager.wait_until_task_aborted(&task_id).await.is_ok());
    }
}