flow/adapter/
worker.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
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
// Copyright 2023 Greptime Team
//
// 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.

//! For single-thread flow worker

use std::collections::{BTreeMap, VecDeque};
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;

use common_telemetry::info;
use enum_as_inner::EnumAsInner;
use hydroflow::scheduled::graph::Hydroflow;
use snafu::ensure;
use tokio::sync::{broadcast, mpsc, oneshot, Mutex};

use crate::adapter::FlowId;
use crate::compute::{Context, DataflowState, ErrCollector};
use crate::error::{Error, FlowAlreadyExistSnafu, InternalSnafu, UnexpectedSnafu};
use crate::expr::{Batch, GlobalId};
use crate::plan::TypedPlan;
use crate::repr::{self, DiffRow};

pub type SharedBuf = Arc<Mutex<VecDeque<DiffRow>>>;

type ReqId = usize;

/// Create both worker(`!Send`) and worker handle(`Send + Sync`)
pub fn create_worker<'a>() -> (WorkerHandle, Worker<'a>) {
    let (itc_client, itc_server) = create_inter_thread_call();
    let worker_handle = WorkerHandle {
        itc_client,
        shutdown: AtomicBool::new(false),
    };
    let worker = Worker {
        task_states: BTreeMap::new(),
        itc_server: Arc::new(Mutex::new(itc_server)),
    };
    (worker_handle, worker)
}

/// ActiveDataflowState is a wrapper around `Hydroflow` and `DataflowState`
pub(crate) struct ActiveDataflowState<'subgraph> {
    df: Hydroflow<'subgraph>,
    state: DataflowState,
    err_collector: ErrCollector,
}

impl std::fmt::Debug for ActiveDataflowState<'_> {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ActiveDataflowState")
            .field("df", &"<Hydroflow>")
            .field("state", &self.state)
            .field("err_collector", &self.err_collector)
            .finish()
    }
}

impl Default for ActiveDataflowState<'_> {
    fn default() -> Self {
        ActiveDataflowState {
            df: Hydroflow::new(),
            state: DataflowState::default(),
            err_collector: ErrCollector::default(),
        }
    }
}

impl<'subgraph> ActiveDataflowState<'subgraph> {
    /// Create a new render context, assigned with given global id
    pub fn new_ctx<'ctx>(&'ctx mut self, global_id: GlobalId) -> Context<'ctx, 'subgraph>
    where
        'subgraph: 'ctx,
    {
        Context {
            id: global_id,
            df: &mut self.df,
            compute_state: &mut self.state,
            err_collector: self.err_collector.clone(),
            input_collection: Default::default(),
            local_scope: Default::default(),
            input_collection_batch: Default::default(),
            local_scope_batch: Default::default(),
        }
    }

    pub fn set_current_ts(&mut self, ts: repr::Timestamp) {
        self.state.set_current_ts(ts);
    }

    pub fn set_last_exec_time(&mut self, ts: repr::Timestamp) {
        self.state.set_last_exec_time(ts);
    }

    /// Run all available subgraph
    ///
    /// return true if any subgraph actually executed
    pub fn run_available(&mut self) -> bool {
        self.state.run_available_with_schedule(&mut self.df)
    }
}

#[derive(Debug)]
pub struct WorkerHandle {
    itc_client: InterThreadCallClient,
    shutdown: AtomicBool,
}

impl WorkerHandle {
    /// create task, return task id
    pub async fn create_flow(&self, create_reqs: Request) -> Result<Option<FlowId>, Error> {
        ensure!(
            matches!(create_reqs, Request::Create { .. }),
            InternalSnafu {
                reason: format!(
                    "Flow Node/Worker itc failed, expect Request::Create, found {create_reqs:?}"
                ),
            }
        );

        let ret = self.itc_client.call_with_resp(create_reqs).await?;
        ret.into_create().map_err(|ret| {
            InternalSnafu {
                reason: format!(
                    "Flow Node/Worker itc failed, expect Response::Create, found {ret:?}"
                ),
            }
            .build()
        })?
    }

    /// remove task, return task id
    pub async fn remove_flow(&self, flow_id: FlowId) -> Result<bool, Error> {
        let req = Request::Remove { flow_id };

        let ret = self.itc_client.call_with_resp(req).await?;

        ret.into_remove().map_err(|ret| {
            InternalSnafu {
                reason: format!("Flow Node/Worker failed, expect Response::Remove, found {ret:?}"),
            }
            .build()
        })
    }

    /// trigger running the worker, will not block, and will run the worker parallelly
    ///
    /// will set the current timestamp to `now` for all dataflows before running them
    ///
    /// `blocking` indicate whether it will wait til all dataflows are finished computing if true or
    /// just start computing and return immediately if false
    ///
    /// the returned error is unrecoverable, and the worker should be shutdown/rebooted
    pub async fn run_available(&self, now: repr::Timestamp, blocking: bool) -> Result<(), Error> {
        common_telemetry::trace!("Running available with blocking={}", blocking);
        if blocking {
            let resp = self
                .itc_client
                .call_with_resp(Request::RunAvail { now, blocking })
                .await?;
            common_telemetry::trace!("Running available with response={:?}", resp);
            Ok(())
        } else {
            self.itc_client
                .call_no_resp(Request::RunAvail { now, blocking })
        }
    }

    pub async fn contains_flow(&self, flow_id: FlowId) -> Result<bool, Error> {
        let req = Request::ContainTask { flow_id };
        let ret = self.itc_client.call_with_resp(req).await?;

        ret.into_contain_task().map_err(|ret| {
            InternalSnafu {
                reason: format!(
                    "Flow Node/Worker itc failed, expect Response::ContainTask, found {ret:?}"
                ),
            }
            .build()
        })
    }

    /// shutdown the worker
    pub fn shutdown(&self) -> Result<(), Error> {
        if !self.shutdown.fetch_or(true, Ordering::SeqCst) {
            self.itc_client.call_no_resp(Request::Shutdown)
        } else {
            UnexpectedSnafu {
                reason: "Worker already shutdown",
            }
            .fail()
        }
    }

    pub async fn get_state_size(&self) -> Result<BTreeMap<FlowId, usize>, Error> {
        let ret = self
            .itc_client
            .call_with_resp(Request::QueryStateSize)
            .await?;
        ret.into_query_state_size().map_err(|ret| {
            InternalSnafu {
                reason: format!(
                    "Flow Node/Worker itc failed, expect Response::QueryStateSize, found {ret:?}"
                ),
            }
            .build()
        })
    }

    pub async fn get_last_exec_time_map(&self) -> Result<BTreeMap<FlowId, i64>, Error> {
        let ret = self
            .itc_client
            .call_with_resp(Request::QueryLastExecTimeMap)
            .await?;
        ret.into_query_last_exec_time_map().map_err(|ret| {
            InternalSnafu {
                reason: format!(
                    "Flow Node/Worker get_last_exec_time_map failed, expect Response::QueryLastExecTimeMap, found {ret:?}"
                ),
            }
            .build()
        })
    }
}

impl Drop for WorkerHandle {
    fn drop(&mut self) {
        if let Err(ret) = self.shutdown() {
            common_telemetry::error!(
                ret;
                "While dropping Worker Handle, failed to shutdown worker, worker might be in inconsistent state."
            );
        } else {
            info!("Flow Worker shutdown due to Worker Handle dropped.")
        }
    }
}

/// The actual worker that does the work and contain active state
#[derive(Debug)]
pub struct Worker<'subgraph> {
    /// Task states
    pub(crate) task_states: BTreeMap<FlowId, ActiveDataflowState<'subgraph>>,
    itc_server: Arc<Mutex<InterThreadCallServer>>,
}

impl<'s> Worker<'s> {
    #[allow(clippy::too_many_arguments)]
    pub fn create_flow(
        &mut self,
        flow_id: FlowId,
        plan: TypedPlan,
        sink_id: GlobalId,
        sink_sender: mpsc::UnboundedSender<Batch>,
        source_ids: &[GlobalId],
        src_recvs: Vec<broadcast::Receiver<Batch>>,
        // TODO(discord9): set expire duration for all arrangement and compare to sys timestamp instead
        expire_after: Option<repr::Duration>,
        or_replace: bool,
        create_if_not_exists: bool,
        err_collector: ErrCollector,
    ) -> Result<Option<FlowId>, Error> {
        let already_exist = self.task_states.contains_key(&flow_id);
        match (create_if_not_exists, or_replace, already_exist) {
            // if replace, ignore that old flow exists
            (_, true, true) => {
                info!("Replacing flow with id={}", flow_id);
            }
            (false, false, true) => FlowAlreadyExistSnafu { id: flow_id }.fail()?,
            // already exists, and not replace, return None
            (true, false, true) => {
                info!("Flow with id={} already exists, do nothing", flow_id);
                return Ok(None);
            }
            // continue as normal
            (_, _, false) => (),
        }

        let mut cur_task_state = ActiveDataflowState::<'s> {
            err_collector,
            ..Default::default()
        };
        cur_task_state.state.set_expire_after(expire_after);

        {
            let mut ctx = cur_task_state.new_ctx(sink_id);
            for (source_id, src_recv) in source_ids.iter().zip(src_recvs) {
                let bundle = ctx.render_source_batch(src_recv)?;
                ctx.insert_global_batch(*source_id, bundle);
            }

            let rendered = ctx.render_plan_batch(plan)?;
            ctx.render_unbounded_sink_batch(rendered, sink_sender);
        }
        self.task_states.insert(flow_id, cur_task_state);
        Ok(Some(flow_id))
    }

    /// remove task, return true if a task is removed
    pub fn remove_flow(&mut self, flow_id: FlowId) -> bool {
        self.task_states.remove(&flow_id).is_some()
    }

    /// Run the worker, blocking, until shutdown signal is received
    pub fn run(&mut self) {
        loop {
            let (req, ret_tx) = if let Some(ret) = self.itc_server.blocking_lock().blocking_recv() {
                ret
            } else {
                common_telemetry::error!(
                    "Worker's itc server has been closed unexpectedly, shutting down worker now."
                );
                break;
            };

            let ret = self.handle_req(req);
            match (ret, ret_tx) {
                (Ok(Some(resp)), Some(ret_tx)) => {
                    if let Err(err) = ret_tx.send(resp) {
                        common_telemetry::error!(
                            err;
                            "Result receiver is dropped, can't send result"
                        );
                    };
                }
                (Ok(None), None) => continue,
                (Ok(Some(resp)), None) => {
                    common_telemetry::error!(
                        "Expect no result for current request, but found {resp:?}"
                    )
                }
                (Ok(None), Some(_)) => {
                    common_telemetry::error!("Expect result for current request, but found nothing")
                }
                (Err(()), _) => {
                    break;
                }
            }
        }
    }

    /// run with tick acquired from tick manager(usually means system time)
    /// TODO(discord9): better tick management
    pub fn run_tick(&mut self, now: repr::Timestamp) {
        for (_flow_id, task_state) in self.task_states.iter_mut() {
            task_state.set_current_ts(now);
            task_state.set_last_exec_time(now);
            task_state.run_available();
        }
    }
    /// handle request, return response if any, Err if receive shutdown signal
    ///
    /// return `Err(())` if receive shutdown request
    fn handle_req(&mut self, req: Request) -> Result<Option<Response>, ()> {
        let ret = match req {
            Request::Create {
                flow_id,
                plan,
                sink_id,
                sink_sender,
                source_ids,
                src_recvs,
                expire_after,
                or_replace,
                create_if_not_exists,
                err_collector,
            } => {
                let task_create_result = self.create_flow(
                    flow_id,
                    plan,
                    sink_id,
                    sink_sender,
                    &source_ids,
                    src_recvs,
                    expire_after,
                    or_replace,
                    create_if_not_exists,
                    err_collector,
                );
                Some(Response::Create {
                    result: task_create_result,
                })
            }
            Request::Remove { flow_id } => {
                let ret = self.remove_flow(flow_id);
                Some(Response::Remove { result: ret })
            }
            Request::RunAvail { now, blocking } => {
                self.run_tick(now);
                if blocking {
                    Some(Response::RunAvail)
                } else {
                    None
                }
            }
            Request::ContainTask { flow_id } => {
                let ret = self.task_states.contains_key(&flow_id);
                Some(Response::ContainTask { result: ret })
            }
            Request::Shutdown => return Err(()),
            Request::QueryStateSize => {
                let mut ret = BTreeMap::new();
                for (flow_id, task_state) in self.task_states.iter() {
                    ret.insert(*flow_id, task_state.state.get_state_size());
                }
                Some(Response::QueryStateSize { result: ret })
            }
            Request::QueryLastExecTimeMap => {
                let mut ret = BTreeMap::new();
                for (flow_id, task_state) in self.task_states.iter() {
                    if let Some(last_exec_time) = task_state.state.last_exec_time() {
                        ret.insert(*flow_id, last_exec_time);
                    }
                }
                Some(Response::QueryLastExecTimeMap { result: ret })
            }
        };
        Ok(ret)
    }
}

#[derive(Debug, EnumAsInner)]
pub enum Request {
    Create {
        flow_id: FlowId,
        plan: TypedPlan,
        sink_id: GlobalId,
        sink_sender: mpsc::UnboundedSender<Batch>,
        source_ids: Vec<GlobalId>,
        src_recvs: Vec<broadcast::Receiver<Batch>>,
        expire_after: Option<repr::Duration>,
        or_replace: bool,
        create_if_not_exists: bool,
        err_collector: ErrCollector,
    },
    Remove {
        flow_id: FlowId,
    },
    /// Trigger the worker to run, useful after input buffer is full
    RunAvail {
        now: repr::Timestamp,
        blocking: bool,
    },
    ContainTask {
        flow_id: FlowId,
    },
    Shutdown,
    QueryStateSize,
    QueryLastExecTimeMap,
}

#[derive(Debug, EnumAsInner)]
enum Response {
    Create {
        result: Result<Option<FlowId>, Error>,
        // TODO(discord9): add flow err_collector
    },
    Remove {
        result: bool,
    },
    ContainTask {
        result: bool,
    },
    RunAvail,
    QueryStateSize {
        /// each flow tasks' state size
        result: BTreeMap<FlowId, usize>,
    },
    QueryLastExecTimeMap {
        /// each flow tasks' last execution time
        result: BTreeMap<FlowId, i64>,
    },
}

fn create_inter_thread_call() -> (InterThreadCallClient, InterThreadCallServer) {
    let (arg_send, arg_recv) = mpsc::unbounded_channel();
    let client = InterThreadCallClient {
        arg_sender: arg_send,
    };
    let server = InterThreadCallServer { arg_recv };
    (client, server)
}

#[derive(Debug)]
struct InterThreadCallClient {
    arg_sender: mpsc::UnboundedSender<(Request, Option<oneshot::Sender<Response>>)>,
}

impl InterThreadCallClient {
    /// call without response
    fn call_no_resp(&self, req: Request) -> Result<(), Error> {
        self.arg_sender.send((req, None)).map_err(from_send_error)
    }

    /// call with response
    async fn call_with_resp(&self, req: Request) -> Result<Response, Error> {
        let (tx, rx) = oneshot::channel();
        self.arg_sender
            .send((req, Some(tx)))
            .map_err(from_send_error)?;
        rx.await.map_err(|_| {
            InternalSnafu {
                reason: "Sender is dropped",
            }
            .build()
        })
    }
}

#[derive(Debug)]
struct InterThreadCallServer {
    pub arg_recv: mpsc::UnboundedReceiver<(Request, Option<oneshot::Sender<Response>>)>,
}

impl InterThreadCallServer {
    pub async fn recv(&mut self) -> Option<(Request, Option<oneshot::Sender<Response>>)> {
        self.arg_recv.recv().await
    }

    pub fn blocking_recv(&mut self) -> Option<(Request, Option<oneshot::Sender<Response>>)> {
        self.arg_recv.blocking_recv()
    }
}

fn from_send_error<T>(err: mpsc::error::SendError<T>) -> Error {
    InternalSnafu {
        // this `err` will simply display `channel closed`
        reason: format!(
            "Worker's receiver channel have been closed unexpected: {}",
            err
        ),
    }
    .build()
}

#[cfg(test)]
mod test {
    use tokio::sync::oneshot;

    use super::*;
    use crate::expr::Id;
    use crate::plan::Plan;
    use crate::repr::RelationType;

    #[test]
    fn drop_handle() {
        let (tx, rx) = oneshot::channel();
        let worker_thread_handle = std::thread::spawn(move || {
            let (handle, mut worker) = create_worker();
            tx.send(handle).unwrap();
            worker.run();
        });
        let handle = rx.blocking_recv().unwrap();
        drop(handle);
        worker_thread_handle.join().unwrap();
    }

    #[tokio::test]
    pub async fn test_simple_get_with_worker_and_handle() {
        let (tx, rx) = oneshot::channel();
        let worker_thread_handle = std::thread::spawn(move || {
            let (handle, mut worker) = create_worker();
            tx.send(handle).unwrap();
            worker.run();
        });
        let handle = rx.await.unwrap();
        let src_ids = vec![GlobalId::User(1)];
        let (tx, rx) = broadcast::channel::<Batch>(1024);
        let (sink_tx, mut sink_rx) = mpsc::unbounded_channel::<Batch>();
        let (flow_id, plan) = (
            1,
            TypedPlan {
                plan: Plan::Get {
                    id: Id::Global(GlobalId::User(1)),
                },
                schema: RelationType::new(vec![]).into_unnamed(),
            },
        );
        let create_reqs = Request::Create {
            flow_id,
            plan,
            sink_id: GlobalId::User(1),
            sink_sender: sink_tx,
            source_ids: src_ids,
            src_recvs: vec![rx],
            expire_after: None,
            or_replace: false,
            create_if_not_exists: true,
            err_collector: ErrCollector::default(),
        };
        assert_eq!(
            handle.create_flow(create_reqs).await.unwrap(),
            Some(flow_id)
        );
        tx.send(Batch::empty()).unwrap();
        handle.run_available(0, true).await.unwrap();
        assert_eq!(handle.get_state_size().await.unwrap().len(), 1);
        assert_eq!(sink_rx.recv().await.unwrap(), Batch::empty());
        drop(handle);
        worker_thread_handle.join().unwrap();
    }
}