numaflow 0.5.0

Rust SDK for Numaflow
Documentation
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
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
use chrono::{DateTime, Utc};
use std::collections::HashMap;

use std::sync::Arc;
use tokio::sync::mpsc::{Sender, channel};
use tokio::sync::{mpsc, oneshot};
use tokio_stream::StreamExt;
use tokio_stream::wrappers::ReceiverStream;
use tokio_util::sync::CancellationToken;
use tonic::{Request, Response, Status, async_trait};

use crate::error::{Error, ErrorKind};
pub use crate::proto::reduce as proto;
use crate::shared;
use shared::{ContainerType, build_panic_status, get_panic_info, prost_timestamp_from_utc};
use tracing::error;
/// Default socket address for reduce service
pub const SOCK_ADDR: &str = "/var/run/numaflow/reduce.sock";

/// Default server info file for reduce service
pub const SERVER_INFO_FILE: &str = "/var/run/numaflow/reducer-server-info";

const KEY_JOIN_DELIMITER: &str = ":";

struct ReduceService<C> {
    creator: Arc<C>,
    shutdown_tx: Sender<()>,
    cancellation_token: CancellationToken, // used to cancel all the tasks
}

/// `ReducerCreator` is a trait for creating a new instance of a `Reducer`.
pub trait ReducerCreator {
    /// Each type that implements `ReducerCreator` must also specify an associated type `R` that implements the `Reducer` trait.
    /// The `create` method is used to create a new instance of this `Reducer` type.
    ///
    /// # Example
    ///
    /// Below is an example of how to implement the `ReducerCreator` trait for a specific type `MyReducerCreator`.
    /// `MyReducerCreator` creates instances of `MyReducer`, which is a type that implements the `Reducer` trait.
    ///
    /// ```rust
    /// use numaflow::reduce::{Reducer, ReducerCreator, ReduceRequest, Metadata, Message};
    /// use tokio::sync::mpsc::Receiver;
    /// use tonic::async_trait;
    ///
    /// pub struct MyReducer;
    ///
    /// #[async_trait]
    /// impl Reducer for MyReducer {
    ///     async fn reduce(
    ///         &self,
    ///         keys: Vec<String>,
    ///         mut input: Receiver<ReduceRequest>,
    ///         md: &Metadata,
    ///     ) -> Vec<Message> {
    ///         // Implementation of the reduce method goes here.
    ///         vec![]
    ///     }
    /// }
    ///
    /// pub struct MyReducerCreator;
    ///
    /// impl ReducerCreator for MyReducerCreator {
    ///     type R = MyReducer;
    ///
    ///     fn create(&self) -> Self::R {
    ///         MyReducer
    ///     }
    /// }
    /// ```
    type R: Reducer + Send + Sync + 'static;
    fn create(&self) -> Self::R;
}

/// Reducer trait for implementing Reduce handler.
#[async_trait]
pub trait Reducer {
    /// reduce_handle is provided with a set of keys, a channel of [`ReduceRequest`], and [`Metadata`]. It
    /// returns 0, 1, or more results as a [`Vec`] of [`Message`]. Reduce is a stateful operation and
    /// the channel is for the collection of keys and for that time [Window].
    /// You can read more about reduce [here](https://numaflow.numaproj.io/user-guide/user-defined-functions/reduce/reduce/).
    ///
    /// # Example
    ///
    /// Below is a reduce code to count the number of elements for a given set of keys and window.
    ///
    /// ```no_run
    /// use numaflow::reduce;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), Box<dyn std::error::Error + Send + Sync>> {
    /// let handler_creator = counter::CounterCreator{};
    ///     reduce::Server::new(handler_creator).start().await?;
    ///     Ok(())
    /// }
    /// mod counter {
    ///     use numaflow::reduce::{Message, ReduceRequest};
    ///     use numaflow::reduce::{Reducer, Metadata};
    ///     use tokio::sync::mpsc::Receiver;
    ///     use tonic::async_trait;
    ///     use numaflow::reduce::proto::reduce_server::Reduce;
    ///     pub(crate) struct Counter {}
    ///
    ///     pub(crate) struct CounterCreator {}
    ///
    ///    impl numaflow::reduce::ReducerCreator for CounterCreator {
    ///        type R = Counter;
    ///
    ///        fn create(&self) -> Self::R {
    ///           Counter::new()
    ///       }
    ///     }
    ///
    ///     impl Counter {
    ///         pub(crate) fn new() -> Self {
    ///             Self {}
    ///         }
    ///     }
    ///     #[async_trait]
    ///     impl Reducer for Counter {
    ///         async fn reduce(
    ///             &self,
    ///             keys: Vec<String>,
    ///             mut input: Receiver<ReduceRequest>,
    ///             md: &Metadata,
    ///         ) -> Vec<Message> {
    ///             let mut counter = 0;
    ///             // the loop exits when input is closed which will happen only on close of book.
    ///             while input.recv().await.is_some() {
    ///                 counter += 1;
    ///             }
    ///             let message=Message::new(counter.to_string().into_bytes()).with_tags(vec![]).with_keys(keys.clone());
    ///             vec![message]
    ///         }
    ///     }
    /// }
    ///```
    /// [Window]: https://numaflow.numaproj.io/user-guide/user-defined-functions/reduce/windowing/windowing/
    async fn reduce(
        &self,
        keys: Vec<String>,
        input: mpsc::Receiver<ReduceRequest>,
        md: &Metadata,
    ) -> Vec<Message>;
}

/// IntervalWindow is the start and end boundary of the window.
#[derive(Default, Clone, Debug)]
pub struct IntervalWindow {
    // start time of the window
    pub start_time: DateTime<Utc>,
    // end time of the window
    pub end_time: DateTime<Utc>,
}

impl IntervalWindow {
    fn new(start_time: DateTime<Utc>, end_time: DateTime<Utc>) -> Self {
        Self {
            start_time,
            end_time,
        }
    }
}

impl Metadata {
    pub fn new(interval_window: IntervalWindow) -> Self {
        Self { interval_window }
    }
}

#[derive(Debug, Clone)]
/// Metadata are additional information passed into the [`Reducer::reduce`].
pub struct Metadata {
    pub interval_window: IntervalWindow,
}

/// Message is the response from the user's [`Reducer::reduce`].
#[derive(Debug, PartialEq)]
pub struct Message {
    /// Keys are a collection of strings which will be passed on to the next vertex as is. It can
    /// be an empty collection. It is mainly used in creating a partition in [`Reducer::reduce`].
    pub keys: Option<Vec<String>>,
    /// Value is the value passed to the next vertex.
    pub value: Vec<u8>,
    /// Tags are used for [conditional forwarding](https://numaflow.numaproj.io/user-guide/reference/conditional-forwarding/).
    pub tags: Option<Vec<String>>,
}

/// Represents a message that can be modified and forwarded.
impl Message {
    /// Creates a new message with the specified value.
    ///
    /// This constructor initializes the message with no keys, tags, or specific event time.
    ///
    /// # Arguments
    ///
    /// * `value` - A vector of bytes representing the message's payload.
    ///
    /// # Examples
    ///
    /// ```
    /// use numaflow::reduce::Message;
    /// let message = Message::new(vec![1, 2, 3, 4]);
    /// ```
    pub fn new(value: Vec<u8>) -> Self {
        Self {
            value,
            keys: None,
            tags: None,
        }
    }
    /// Marks the message to be dropped by creating a new `Message` with an empty value and a special "DROP" tag.
    ///
    /// # Examples
    ///
    /// ```
    /// use numaflow::reduce::Message;
    /// let dropped_message = Message::message_to_drop();
    /// ```
    pub fn message_to_drop() -> crate::map::Message {
        crate::map::Message::message_to_drop()
    }

    /// Sets or replaces the keys associated with this message.
    ///
    /// # Arguments
    ///
    /// * `keys` - A vector of strings representing the keys.
    ///
    /// # Examples
    ///
    /// ```
    ///  use numaflow::reduce::Message;
    /// let message = Message::new(vec![1, 2, 3]).with_keys(vec!["key1".to_string(), "key2".to_string()]);
    /// ```
    pub fn with_keys(mut self, keys: Vec<String>) -> Self {
        self.keys = Some(keys);
        self
    }

    /// Sets or replaces the tags associated with this message.
    ///
    /// # Arguments
    ///
    /// * `tags` - A vector of strings representing the tags.
    ///
    /// # Examples
    ///
    /// ```
    ///  use numaflow::reduce::Message;
    /// let message = Message::new(vec![1, 2, 3]).with_tags(vec!["tag1".to_string(), "tag2".to_string()]);
    /// ```
    pub fn with_tags(mut self, tags: Vec<String>) -> Self {
        self.tags = Some(tags);
        self
    }
}

/// Incoming request into the reducer handler of [`Reducer`].
pub struct ReduceRequest {
    /// Set of keys in the (key, value) terminology of map/reduce paradigm.
    pub keys: Vec<String>,
    /// The value in the (key, value) terminology of map/reduce paradigm.    /// The value in the (key, value) terminology of map/reduce paradigm.
    pub value: Vec<u8>,
    /// [watermark](https://numaflow.numaproj.io/core-concepts/watermarks/) represented by time is a guarantee that we will not see an element older than this time.    /// [watermark](https://numaflow.numaproj.io/core-concepts/watermarks/) represented by time is a guarantee that we will not see an element older than this time.
    pub watermark: DateTime<Utc>,
    /// Time of the element as seen at source or aligned after a reduce operation.
    pub eventtime: DateTime<Utc>,
    /// Headers for the message.
    pub headers: HashMap<String, String>,
}

#[async_trait]
impl<C> proto::reduce_server::Reduce for ReduceService<C>
where
    C: ReducerCreator + Send + Sync + 'static,
{
    type ReduceFnStream = ReceiverStream<Result<proto::ReduceResponse, Status>>;
    async fn reduce_fn(
        &self,
        request: Request<tonic::Streaming<proto::ReduceRequest>>,
    ) -> Result<Response<Self::ReduceFnStream>, Status> {
        // Clone the creator and shutdown_tx to be used in the spawned tasks.
        let creator = Arc::clone(&self.creator);
        let shutdown_tx = self.shutdown_tx.clone();

        // Create a channel to send the response back to the grpc client.
        let (grpc_response_tx, grpc_response_rx) =
            channel::<Result<proto::ReduceResponse, Status>>(1);

        // Internal response channel which will be used by the task set and tasks to send the response after
        // executing the user defined function. It's a result type so in case of error, we can send the error
        // back to the client.
        //
        // NOTE: we are using a separate channel instead of the grpc_response_tx because in case of errors,
        // we have to do graceful shutdown.
        let (response_tx, mut response_rx) = channel::<Result<proto::ReduceResponse, Error>>(1);

        // Start a task executor to handle the incoming ReduceRequests from the client, returns a tx to send
        // commands to the task executor and an oneshot tx to abort all the tasks.
        let (task_tx, abort_tx) = TaskSet::start_task_executor(creator, response_tx.clone());

        // Spawn a new task to handle the incoming ReduceRequests from the client
        let reader_handle = tokio::spawn(async move {
            let mut stream = request.into_inner();
            loop {
                match stream.next().await {
                    Some(Ok(rr)) => {
                        task_tx
                            .send(TaskCommand::HandleReduceRequest(rr))
                            .await
                            .expect("task_tx send failed");
                    }
                    Some(Err(e)) => {
                        response_tx
                            .send(Err(Error::ReduceError(ErrorKind::InternalError(format!(
                                "Failed to receive request: {}",
                                e
                            )))))
                            .await
                            .expect("error_tx send failed");
                        break;
                    }
                    // COB
                    None => {
                        task_tx
                            .send(TaskCommand::Close)
                            .await
                            .expect("task_tx send failed");
                        break;
                    }
                }
            }
        });

        // Spawn a new task to listen to the response channel and send the response back to the grpc client.
        // In case of error, it propagates the error back to the client in grpc status format and sends a shutdown
        // signal to the grpc server. It also listens to the cancellation signal and aborts all the tasks.
        let response_task_token = self.cancellation_token.clone();
        tokio::spawn(async move {
            loop {
                tokio::select! {
                    result = response_rx.recv() => {
                        match result {
                            Some(Ok(response)) => {
                                let eof = response.eof;
                                grpc_response_tx
                                    .send(Ok(response))
                                    .await
                                    .expect("send to grpc response channel failed");
                                 // all the tasks are done (COB has happened and we have closed the tx for the tasks)
                                if eof {
                                    break;
                                }
                            }
                            Some(Err(error)) => {
                                error!("Error from task: {:?}", error);
                                grpc_response_tx
                                    .send(Err(error.into_status()))
                                    .await
                                    .expect("send to grpc response channel failed");
                                // stop reading new messages from the stream.
                                reader_handle.abort();
                                // Send a shutdown signal to the grpc server.
                                shutdown_tx.send(()).await.expect("shutdown_tx send failed");
                            }
                            None => {
                                // we break at eof, None should not happen
                                unreachable!()
                            }
                        }
                    }
                    _ = response_task_token.cancelled() => {
                        // stop reading new messages from stream.
                        reader_handle.abort();
                        // Send an abort signal to the task executor to abort all the tasks.
                        abort_tx.send(()).expect("task_tx send failed");
                        break;
                    }
                }
            }
        });

        // return the rx as the streaming endpoint
        Ok(Response::new(ReceiverStream::new(grpc_response_rx)))
    }

    async fn is_ready(&self, _: Request<()>) -> Result<Response<proto::ReadyResponse>, Status> {
        Ok(Response::new(proto::ReadyResponse { ready: true }))
    }
}

// The `Task` struct represents a task in the reduce service. It is responsible for executing the
// user defined function. We will a separate task for each keyed window. The task will be created
// when the first message for a given key arrives and will be closed when the window is closed.
struct Task {
    udf_tx: Sender<ReduceRequest>,
    response_tx: Sender<Result<proto::ReduceResponse, Error>>,
    done_rx: oneshot::Receiver<()>,
    handle: tokio::task::JoinHandle<()>,
}

// we only have one slot
const SLOT_0: &str = "slot-0";

impl Task {
    // Creates a new task with the given reducer, keys, metadata, and response channel.
    async fn new<R: Reducer + Send + Sync + 'static>(
        reducer: R,
        keys: Vec<String>,
        md: Metadata,
        response_tx: Sender<Result<proto::ReduceResponse, Error>>,
    ) -> Self {
        let (udf_tx, udf_rx) = channel::<ReduceRequest>(1);
        let (done_tx, done_rx) = oneshot::channel();

        let udf_response_tx = response_tx.clone();
        let task_join_handler = tokio::spawn(async move {
            // execute user code
            let messages = reducer.reduce(keys, udf_rx, &md).await;

            // forward the responses
            for message in messages {
                let send_result = udf_response_tx
                    .send(Ok(proto::ReduceResponse {
                        result: Some(proto::reduce_response::Result {
                            keys: message.keys.unwrap_or_default(),
                            value: message.value,
                            tags: message.tags.unwrap_or_default(),
                        }),
                        window: Some(proto::Window {
                            start: prost_timestamp_from_utc(md.interval_window.start_time),
                            end: prost_timestamp_from_utc(md.interval_window.end_time),
                            slot: SLOT_0.to_string(),
                        }),
                        eof: false,
                    }))
                    .await;

                if let Err(e) = send_result {
                    let _ = udf_response_tx
                        .send(Err(Error::ReduceError(ErrorKind::InternalError(format!(
                            "Failed to send response back: {}",
                            e
                        )))))
                        .await;
                    return;
                }
            }
        });

        // We spawn a separate task to await the join handler so that in case of any unhandled errors in the user-defined
        // code will immediately be propagated to the client.
        let handler_tx = response_tx.clone();
        let handle = tokio::spawn(async move {
            if let Err(e) = task_join_handler.await {
                error!("Failed to run reduce function: {e:?}");

                // Check if this is a panic or a regular error
                if let Some(panic_info) = get_panic_info() {
                    // This is a panic - send detailed panic information
                    let status = build_panic_status(&panic_info);
                    let _ = handler_tx.send(Err(Error::GrpcStatus(status))).await;
                } else {
                    // This is a non-panic error
                    let _ = handler_tx
                        .send(Err(Error::ReduceError(ErrorKind::UserDefinedError(
                            format!("Reduce task execution failed: {}", e),
                        ))))
                        .await;
                }
            }

            // Send a message indicating that the task has finished
            let _ = done_tx.send(());
        });

        Self {
            udf_tx,
            response_tx,
            done_rx,
            // We store the task join handle so that we can abort the task if needed, we only need the second task handle because
            // if the second task is aborted, the first task's handle will be dropped and the task will be aborted.
            handle,
        }
    }

    // Sends the request to the user defined function's input channel.
    async fn send(&self, rr: ReduceRequest) {
        if let Err(e) = self.udf_tx.send(rr).await {
            self.response_tx
                .send(Err(Error::ReduceError(ErrorKind::InternalError(format!(
                    "Failed to send message to task: {}",
                    e
                )))))
                .await
                .expect("failed to send message to error channel");
        }
    }

    // Closes the task and waits for it to finish.
    async fn close(self) {
        // drop the sender to close the task
        drop(self.udf_tx);

        // Wait for the task to finish
        let _ = self.done_rx.await;
    }

    // Aborts the task by calling abort on join handler.
    async fn abort(self) {
        self.handle.abort();
    }
}

// The `TaskSet` struct represents a set of tasks that are executing the user defined function. It is responsible
// for creating new tasks, writing messages to the tasks, closing the tasks, and aborting the tasks.
struct TaskSet<C> {
    tasks: HashMap<String, Task>,
    response_tx: Sender<Result<proto::ReduceResponse, Error>>,
    creator: Arc<C>,
    window: IntervalWindow,
}

enum TaskCommand {
    HandleReduceRequest(proto::ReduceRequest),
    Close,
}

impl<C> TaskSet<C>
where
    C: ReducerCreator + Send + Sync + 'static,
{
    // Starts a new task executor which listens to incoming commands and executes them.
    // returns a tx to send commands to the task executor and oneshot tx to abort all
    // the tasks to gracefully shut down the task executor.
    fn start_task_executor(
        creator: Arc<C>,
        response_tx: Sender<Result<proto::ReduceResponse, Error>>,
    ) -> (Sender<TaskCommand>, oneshot::Sender<()>) {
        let (task_tx, mut task_rx) = channel::<TaskCommand>(1);
        let (abort_tx, mut abort_rx) = oneshot::channel();

        let mut task_set = TaskSet {
            tasks: HashMap::new(),
            response_tx,
            creator,
            window: IntervalWindow::default(),
        };

        // Start a new task to listen to incoming commands and execute them, it will also listen to the abort signal.
        tokio::spawn(async move {
            loop {
                tokio::select! {
                    cmd = task_rx.recv() => {
                        match cmd {
                            Some(TaskCommand::HandleReduceRequest(rr)) => {
                               // Extract the keys from the ReduceRequest.
                                let keys = match rr.payload.as_ref() {
                                    Some(payload) => payload.keys.clone(),
                                    None => {
                                        task_set
                                            .handle_error(Error::ReduceError(ErrorKind::InternalError(
                                                "Invalid ReduceRequest".to_string(),
                                            )))
                                            .await;
                                        continue;
                                    }
                                };

                                // Check if the task already exists, if it does, write the ReduceRequest to the task,
                                // otherwise create a new task and write the ReduceRequest to the task.
                                if task_set.tasks.contains_key(&keys.join(KEY_JOIN_DELIMITER)) {
                                    task_set.write_to_task(keys, rr).await;
                                } else {
                                    task_set.create_and_write(keys, rr).await;
                                }
                            }
                            Some(TaskCommand::Close) => task_set.close().await,
                            // COB
                            None => break,
                        }
                    }
                    _ = &mut abort_rx => {
                        task_set.abort().await;
                        break;
                    }
                }
            }
        });

        (task_tx, abort_tx)
    }

    // Creates a new task with the given keys and `ReduceRequest`.
    // It creates a new reducer and assigns it to the task to execute the user defined function.
    async fn create_and_write(&mut self, keys: Vec<String>, rr: proto::ReduceRequest) {
        // validate
        let (reduce_request, interval_window) = match self.validate_and_extract(rr).await {
            Some(value) => value,
            None => return,
        };

        self.window = interval_window.clone();

        // Create a new reducer
        let reducer = self.creator.create();

        // Create Metadata with the extracted start and end time
        let md = Metadata::new(interval_window);

        // Create a new Task with the reducer, keys, and metadata
        let task = Task::new(reducer, keys.clone(), md, self.response_tx.clone()).await;

        // track the task in the task set
        self.tasks.insert(keys.join(KEY_JOIN_DELIMITER), task);

        // send the request inside the proto payload to the task
        // if the task does not exist, send an error to the stream
        if let Some(task) = self.tasks.get(&keys.join(KEY_JOIN_DELIMITER)) {
            task.send(reduce_request).await;
        } else {
            self.handle_error(Error::ReduceError(ErrorKind::InternalError(
                "Task not found".to_string(),
            )))
            .await;
        }
    }

    // Writes the ReduceRequest to the task with the given keys.
    async fn write_to_task(&mut self, keys: Vec<String>, rr: proto::ReduceRequest) {
        // validate the request
        let (reduce_request, _) = match self.validate_and_extract(rr).await {
            Some(value) => value,
            None => return,
        };

        // Get the task name from the keys
        let task_name = keys.join(KEY_JOIN_DELIMITER);

        // If the task exists, send the ReduceRequest to the task
        if let Some(task) = self.tasks.get(&task_name) {
            task.send(reduce_request).await;
        } else {
            self.handle_error(Error::ReduceError(ErrorKind::InternalError(
                "Task not found".to_string(),
            )))
            .await;
        }
    }

    // Validates the ReduceRequest and extracts the payload and window information.
    // If the ReduceRequest is invalid, it sends an error to the response stream and returns None.
    async fn validate_and_extract(
        &self,
        rr: proto::ReduceRequest,
    ) -> Option<(ReduceRequest, IntervalWindow)> {
        // Extract the payload and window information from the ReduceRequest
        let (payload, windows) = match (rr.payload, rr.operation) {
            (Some(payload), Some(operation)) => (payload, operation.windows),
            _ => {
                self.handle_error(Error::ReduceError(ErrorKind::InternalError(
                    "Invalid ReduceRequest".to_string(),
                )))
                .await;
                return None;
            }
        };

        // Check if there is exactly one window in the ReduceRequest
        if windows.len() != 1 {
            self.handle_error(Error::ReduceError(ErrorKind::InternalError(
                "Exactly one window is required".to_string(),
            )))
            .await;
            return None;
        }

        // Extract the start and end time from the window
        let window = &windows[0];
        let (start_time, end_time) = (
            shared::utc_from_timestamp(window.start),
            shared::utc_from_timestamp(window.end),
        );

        // Create the IntervalWindow
        let interval_window = IntervalWindow::new(start_time, end_time);

        // Create the ReduceRequest
        let reduce_request = ReduceRequest {
            keys: payload.keys,
            value: payload.value,
            watermark: shared::utc_from_timestamp(payload.watermark),
            eventtime: shared::utc_from_timestamp(payload.event_time),
            headers: payload.headers,
        };

        Some((reduce_request, interval_window))
    }

    // Closes all tasks in the task set and sends an EOF message to the response stream.
    async fn close(&mut self) {
        for (_, task) in self.tasks.drain() {
            task.close().await;
        }

        // after all the tasks have been closed, send an EOF message to the response stream
        let send_eof = self
            .response_tx
            .send(Ok(proto::ReduceResponse {
                result: None,
                window: Some(proto::Window {
                    start: prost_timestamp_from_utc(self.window.start_time),
                    end: prost_timestamp_from_utc(self.window.end_time),
                    slot: "slot-0".to_string(),
                }),
                eof: true,
            }))
            .await;

        if let Err(e) = send_eof {
            self.handle_error(Error::ReduceError(ErrorKind::InternalError(format!(
                "Failed to send EOF message: {}",
                e
            ))))
            .await;
        }
    }

    // Aborts all tasks in the task set.
    async fn abort(&mut self) {
        for (_, task) in self.tasks.drain() {
            task.abort().await;
        }
    }

    // Sends an error to the response stream.
    async fn handle_error(&self, error: Error) {
        self.response_tx
            .send(Err(error))
            .await
            .expect("error_tx send failed");
    }
}

/// gRPC server to start a reduce service
#[derive(Debug)]
pub struct Server<C> {
    inner: shared::Server<C>,
}

impl<C> shared::ServerExtras<C> for Server<C> {
    fn transform_inner<F>(self, f: F) -> Self
    where
        F: FnOnce(shared::Server<C>) -> shared::Server<C>,
    {
        Self {
            inner: f(self.inner),
        }
    }

    fn inner_ref(&self) -> &shared::Server<C> {
        &self.inner
    }
}

impl<C> Server<C> {
    /// Create a new Server with the given reduce service
    pub fn new(creator: C) -> Self {
        Self {
            inner: shared::Server::new(creator, ContainerType::Reduce, SOCK_ADDR, SERVER_INFO_FILE),
        }
    }

    /// Starts the gRPC server. When message is received on the `shutdown` channel, graceful shutdown of the gRPC server will be initiated.
    pub async fn start_with_shutdown(
        self,
        user_shutdown_rx: oneshot::Receiver<()>,
    ) -> Result<(), Box<dyn std::error::Error + Send + Sync>>
    where
        C: ReducerCreator + Send + Sync + 'static,
    {
        self.inner
            .start_with_shutdown(
                user_shutdown_rx,
                |creator, max_message_size, shutdown_tx, cln_token| {
                    let reduce_svc = ReduceService {
                        creator: Arc::new(creator),
                        shutdown_tx,
                        cancellation_token: cln_token,
                    };

                    let reduce_svc = proto::reduce_server::ReduceServer::new(reduce_svc)
                        .max_encoding_message_size(max_message_size)
                        .max_decoding_message_size(max_message_size);

                    tonic::transport::Server::builder().add_service(reduce_svc)
                },
            )
            .await
    }

    /// Starts the gRPC server. Automatically registers signal handlers for SIGINT and SIGTERM and initiates
    /// graceful shutdown of gRPC server when either one of the signal arrives.
    pub async fn start(self) -> Result<(), Box<dyn std::error::Error + Send + Sync>>
    where
        C: ReducerCreator + Send + Sync + 'static,
    {
        self.inner
            .start(|creator, max_message_size, shutdown_tx, cln_token| {
                let reduce_svc = ReduceService {
                    creator: Arc::new(creator),
                    shutdown_tx,
                    cancellation_token: cln_token,
                };

                let reduce_svc = proto::reduce_server::ReduceServer::new(reduce_svc)
                    .max_encoding_message_size(max_message_size)
                    .max_decoding_message_size(max_message_size);

                tonic::transport::Server::builder().add_service(reduce_svc)
            })
            .await
    }
}

#[cfg(test)]
mod tests {
    use crate::shared::ServerExtras;
    use std::path::PathBuf;
    use std::{error::Error, time::Duration};

    use prost_types::Timestamp;
    use tempfile::TempDir;
    use tokio::net::UnixStream;
    use tokio::sync::{mpsc, oneshot};
    use tokio::time::sleep;
    use tokio_stream::wrappers::ReceiverStream;
    use tonic::Request;
    use tonic::transport::Uri;
    use tower::service_fn;

    use crate::reduce;
    use crate::reduce::proto::reduce_client::ReduceClient;

    struct Sum;

    #[tonic::async_trait]
    impl reduce::Reducer for Sum {
        async fn reduce(
            &self,
            _keys: Vec<String>,
            mut input: mpsc::Receiver<reduce::ReduceRequest>,
            _md: &reduce::Metadata,
        ) -> Vec<reduce::Message> {
            let mut sum = 0;
            while let Some(rr) = input.recv().await {
                sum += std::str::from_utf8(&rr.value)
                    .unwrap()
                    .parse::<i32>()
                    .unwrap();
            }
            vec![reduce::Message::new(sum.to_string().into_bytes())]
        }
    }

    struct SumCreator;

    impl reduce::ReducerCreator for SumCreator {
        type R = Sum;
        fn create(&self) -> Sum {
            Sum {}
        }
    }

    async fn setup_server<C: reduce::ReducerCreator + Send + Sync + 'static>(
        creator: C,
    ) -> Result<(reduce::Server<C>, PathBuf, PathBuf), Box<dyn Error>> {
        let tmp_dir = TempDir::new()?;
        let sock_file = tmp_dir.path().join("reduce.sock");
        let server_info_file = tmp_dir.path().join("reducer-server-info");

        let server = reduce::Server::new(creator)
            .with_server_info_file(&server_info_file)
            .with_socket_file(&sock_file)
            .with_max_message_size(10240);

        Ok((server, sock_file, server_info_file))
    }

    async fn setup_client(
        sock_file: PathBuf,
    ) -> Result<ReduceClient<tonic::transport::Channel>, Box<dyn Error>> {
        // https://github.com/hyperium/tonic/blob/master/examples/src/uds/client.rs
        let channel = tonic::transport::Endpoint::try_from("http://[::]:50051")?
            .connect_with_connector(service_fn(move |_: Uri| {
                // https://rust-lang.github.io/async-book/03_async_await/01_chapter.html#async-lifetimes
                let sock_file = sock_file.clone();
                async move {
                    Ok::<_, std::io::Error>(hyper_util::rt::TokioIo::new(
                        UnixStream::connect(sock_file).await?,
                    ))
                }
            }))
            .await?;

        let client = ReduceClient::new(channel);

        Ok(client)
    }

    #[tokio::test]
    async fn test_server_start() -> Result<(), Box<dyn Error>> {
        let (server, sock_file, server_info_file) = setup_server(SumCreator).await?;

        assert_eq!(server.max_message_size(), 10240);
        assert_eq!(server.server_info_file(), server_info_file);
        assert_eq!(server.socket_file(), sock_file);

        let (shutdown_tx, shutdown_rx) = oneshot::channel();

        let task = tokio::spawn(async move { server.start_with_shutdown(shutdown_rx).await });

        tokio::time::sleep(Duration::from_millis(50)).await;

        // Check if the server has started
        assert!(!task.is_finished(), "gRPC server should be running");

        // Send shutdown signal
        shutdown_tx
            .send(())
            .expect("Sending shutdown signal to gRPC server");

        // Check if the server has stopped within 100 ms
        for _ in 0..10 {
            tokio::time::sleep(Duration::from_millis(10)).await;
            if task.is_finished() {
                break;
            }
        }
        assert!(task.is_finished(), "gRPC server is still running");

        Ok(())
    }

    #[tokio::test]
    async fn valid_input() -> Result<(), Box<dyn Error>> {
        let (server, sock_file, _) = setup_server(SumCreator).await?;

        let (shutdown_tx, shutdown_rx) = oneshot::channel();

        let task = tokio::spawn(async move { server.start_with_shutdown(shutdown_rx).await });

        tokio::time::sleep(Duration::from_millis(50)).await;

        let mut client = setup_client(sock_file).await?;

        let (tx, rx) = mpsc::channel(1);

        // Spawn a task to send ReduceRequests to the channel
        tokio::spawn(async move {
            let data = vec![("key1".to_string(), 1..=10), ("key2".to_string(), 1..=9)];

            for (key, range) in data {
                for i in range {
                    let rr = reduce::proto::ReduceRequest {
                        payload: Some(reduce::proto::reduce_request::Payload {
                            keys: vec![key.clone()],
                            value: i.to_string().as_bytes().to_vec(),
                            watermark: None,
                            event_time: None,
                            headers: Default::default(),
                        }),
                        operation: Some(reduce::proto::reduce_request::WindowOperation {
                            event: 0,
                            windows: vec![reduce::proto::Window {
                                start: Some(Timestamp {
                                    seconds: 60000,
                                    nanos: 0,
                                }),
                                end: Some(Timestamp {
                                    seconds: 120000,
                                    nanos: 0,
                                }),
                                slot: "slot-0".to_string(),
                            }],
                        }),
                    };

                    tx.send(rr).await.unwrap();
                }
            }
        });

        // Convert the receiver end of the channel into a stream
        let stream = ReceiverStream::new(rx);

        // Create a tonic::Request from the stream
        let request = Request::new(stream);

        // Send the request to the server
        let resp = client.reduce_fn(request).await?;

        let mut response_stream = resp.into_inner();
        let mut responses = Vec::new();

        while let Some(response) = response_stream.message().await? {
            responses.push(response);
        }

        // since we are sending two different keys, we should get two responses + 1 EOF
        assert_eq!(responses.len(), 3);

        for (i, response) in responses.iter().enumerate() {
            if let Some(window) = response.window.as_ref() {
                if let Some(start) = window.start.as_ref() {
                    assert_eq!(start.seconds, 60000);
                }
                if let Some(end) = window.end.as_ref() {
                    assert_eq!(end.seconds, 120000);
                }
            }

            if let Some(result) = response.result.as_ref() {
                if result.keys == vec!["key1".to_string()] {
                    assert_eq!(result.value, 55.to_string().into_bytes());
                } else if result.keys == vec!["key2".to_string()] {
                    assert_eq!(result.value, 45.to_string().into_bytes());
                }
            }

            // Check if this is the last message in the stream
            // The last message should have eof set to true
            if i == responses.len() - 1 {
                assert!(response.eof);
            } else {
                assert!(!response.eof);
            }
        }

        shutdown_tx
            .send(())
            .expect("Sending shutdown signal to gRPC server");

        for _ in 0..10 {
            tokio::time::sleep(Duration::from_millis(10)).await;
            if task.is_finished() {
                break;
            }
        }
        assert!(task.is_finished(), "gRPC server is still running");

        Ok(())
    }

    #[tokio::test]
    async fn invalid_input() -> Result<(), Box<dyn Error>> {
        let (server, sock_file, _) = setup_server(SumCreator).await?;

        let (_shutdown_tx, shutdown_rx) = oneshot::channel();

        let task = tokio::spawn(async move { server.start_with_shutdown(shutdown_rx).await });

        tokio::time::sleep(Duration::from_millis(50)).await;

        let mut client = setup_client(sock_file).await?;

        let (tx, rx) = mpsc::unbounded_channel();

        // Spawn a task to send ReduceRequests to the channel
        let _sender_task = tokio::spawn(async move {
            for _ in 0..10 {
                let rr = reduce::proto::ReduceRequest {
                    payload: Some(reduce::proto::reduce_request::Payload {
                        keys: vec!["key1".to_string()],
                        value: vec![],
                        watermark: None,
                        event_time: None,
                        headers: Default::default(),
                    }),
                    operation: Some(reduce::proto::reduce_request::WindowOperation {
                        event: 0,
                        windows: vec![
                            reduce::proto::Window {
                                start: Some(Timestamp {
                                    seconds: 60000,
                                    nanos: 0,
                                }),
                                end: Some(Timestamp {
                                    seconds: 120000,
                                    nanos: 0,
                                }),
                                slot: "slot-0".to_string(),
                            },
                            reduce::proto::Window {
                                start: Some(Timestamp {
                                    seconds: 60000,
                                    nanos: 0,
                                }),
                                end: Some(Timestamp {
                                    seconds: 120000,
                                    nanos: 0,
                                }),
                                slot: "slot-0".to_string(),
                            },
                        ],
                    }),
                };

                tx.send(rr).unwrap();
                sleep(Duration::from_millis(10)).await;
            }
        });

        // Send the request to the server
        let resp = client
            .reduce_fn(Request::new(
                tokio_stream::wrappers::UnboundedReceiverStream::new(rx),
            ))
            .await;

        let mut response_stream = resp.unwrap().into_inner();

        if let Err(e) = response_stream.message().await {
            assert_eq!(e.code(), tonic::Code::Internal);
            assert!(e.message().contains("Exactly one window is required"));
        }

        for _ in 0..10 {
            tokio::time::sleep(Duration::from_millis(10)).await;
            if task.is_finished() {
                break;
            }
        }

        assert!(task.is_finished(), "gRPC server is still running");
        Ok(())
    }

    #[cfg(feature = "test-panic")]
    mod panic_tests {
        use super::*;

        struct SimplePanicReducer;

        #[tonic::async_trait]
        impl reduce::Reducer for SimplePanicReducer {
            async fn reduce(
                &self,
                _keys: Vec<String>,
                _input: mpsc::Receiver<reduce::ReduceRequest>,
                _md: &reduce::Metadata,
            ) -> Vec<reduce::Message> {
                panic!("Panic in reduce method");
            }
        }

        struct SimplePanicReducerCreator;

        impl reduce::ReducerCreator for SimplePanicReducerCreator {
            type R = SimplePanicReducer;
            fn create(&self) -> SimplePanicReducer {
                SimplePanicReducer {}
            }
        }

        struct ConditionalPanicReducer;

        #[tonic::async_trait]
        impl reduce::Reducer for ConditionalPanicReducer {
            async fn reduce(
                &self,
                keys: Vec<String>,
                mut input: mpsc::Receiver<reduce::ReduceRequest>,
                _md: &reduce::Metadata,
            ) -> Vec<reduce::Message> {
                let mut count = 0;
                while input.recv().await.is_some() {
                    count += 1;
                    if count == 10 && keys[0] == "key2" {
                        panic!("Panic in reduce method");
                    }
                }
                vec![]
            }
        }

        struct ConditionalPanicReducerCreator;

        impl reduce::ReducerCreator for ConditionalPanicReducerCreator {
            type R = ConditionalPanicReducer;
            fn create(&self) -> ConditionalPanicReducer {
                ConditionalPanicReducer {}
            }
        }

        #[tokio::test]
        async fn panic_in_reduce() -> Result<(), Box<dyn Error>> {
            let (server, sock_file, _) = setup_server(SimplePanicReducerCreator).await?;

            let (_shutdown_tx, shutdown_rx) = oneshot::channel();

            let task = tokio::spawn(async move { server.start_with_shutdown(shutdown_rx).await });

            tokio::time::sleep(Duration::from_millis(50)).await;

            let mut client = setup_client(sock_file.clone()).await?;

            let (tx, rx) = mpsc::channel(1);

            // Spawn a task to send ReduceRequests to the channel
            tokio::spawn(async move {
                let rr = reduce::proto::ReduceRequest {
                    payload: Some(reduce::proto::reduce_request::Payload {
                        keys: vec!["key1".to_string()],
                        value: vec![],
                        watermark: None,
                        event_time: None,
                        headers: Default::default(),
                    }),
                    operation: Some(reduce::proto::reduce_request::WindowOperation {
                        event: 0,
                        windows: vec![reduce::proto::Window {
                            start: Some(Timestamp {
                                seconds: 60000,
                                nanos: 0,
                            }),
                            end: Some(Timestamp {
                                seconds: 120000,
                                nanos: 0,
                            }),
                            slot: "slot-0".to_string(),
                        }],
                    }),
                };

                for _ in 0..10 {
                    tx.send(rr.clone()).await.unwrap();
                    sleep(Duration::from_millis(10)).await;
                }
            });

            // Convert the receiver end of the channel into a stream
            let stream = ReceiverStream::new(rx);

            // Create a tonic::Request from the stream
            let request = Request::new(stream);

            // Send the request to the server
            let resp = client.reduce_fn(request).await?;

            let mut response_stream = resp.into_inner();

            if let Err(e) = response_stream.message().await {
                assert_eq!(e.code(), tonic::Code::Internal);
                assert!(e.message().contains("UDF_EXECUTION_ERROR"))
            }

            for _ in 0..10 {
                tokio::time::sleep(Duration::from_millis(10)).await;
                if task.is_finished() {
                    break;
                }
            }
            assert!(task.is_finished(), "gRPC server is still running");

            Ok(())
        }

        // test panic in reduce method when there are multiple inflight requests
        // panic only happens for one of the requests, the other request should be
        // processed successfully since we do graceful shutdown of the server.
        #[tokio::test]
        async fn panic_with_multiple_keys() -> Result<(), Box<dyn Error>> {
            let (server, sock_file, _) = setup_server(ConditionalPanicReducerCreator).await?;

            let (_shutdown_tx, shutdown_rx) = oneshot::channel();

            let task = tokio::spawn(async move { server.start_with_shutdown(shutdown_rx).await });

            tokio::time::sleep(Duration::from_millis(50)).await;

            let client = setup_client(sock_file.clone()).await?;

            let (tx1, rx1) = mpsc::channel(1);

            let (tx2, rx2) = mpsc::channel(1);

            // Spawn a task to send ReduceRequests to the channel
            tokio::spawn(async move {
                let rr = reduce::proto::ReduceRequest {
                    payload: Some(reduce::proto::reduce_request::Payload {
                        keys: vec!["key1".to_string()],
                        value: vec![],
                        watermark: None,
                        event_time: None,
                        headers: Default::default(),
                    }),
                    operation: Some(reduce::proto::reduce_request::WindowOperation {
                        event: 0,
                        windows: vec![reduce::proto::Window {
                            start: Some(Timestamp {
                                seconds: 60000,
                                nanos: 0,
                            }),
                            end: Some(Timestamp {
                                seconds: 120000,
                                nanos: 0,
                            }),
                            slot: "slot-0".to_string(),
                        }],
                    }),
                };

                for _ in 0..20 {
                    tx1.send(rr.clone()).await.unwrap();
                    sleep(Duration::from_millis(10)).await;
                }
            });

            tokio::spawn(async move {
                let rr = reduce::proto::ReduceRequest {
                    payload: Some(reduce::proto::reduce_request::Payload {
                        keys: vec!["key2".to_string()],
                        value: vec![],
                        watermark: None,
                        event_time: None,
                        headers: Default::default(),
                    }),
                    operation: Some(reduce::proto::reduce_request::WindowOperation {
                        event: 0,
                        windows: vec![reduce::proto::Window {
                            start: Some(Timestamp {
                                seconds: 60000,
                                nanos: 0,
                            }),
                            end: Some(Timestamp {
                                seconds: 120000,
                                nanos: 0,
                            }),
                            slot: "slot-0".to_string(),
                        }],
                    }),
                };

                for _ in 0..10 {
                    tx2.send(rr.clone()).await.unwrap();
                    sleep(Duration::from_millis(10)).await;
                }
            });

            // Convert the receiver end of the channel into a stream
            let stream1 = ReceiverStream::new(rx1);

            let stream2 = ReceiverStream::new(rx2);

            // Create a tonic::Request from the stream
            let request1 = Request::new(stream1);

            let request2 = Request::new(stream2);

            let mut first_client = client.clone();
            tokio::spawn(async move {
                let mut response_stream =
                    first_client.reduce_fn(request1).await.unwrap().into_inner();
                assert!(response_stream.message().await.is_ok());
            });

            let mut second_client = client.clone();
            tokio::spawn(async move {
                let mut response_stream = second_client
                    .reduce_fn(request2)
                    .await
                    .unwrap()
                    .into_inner();

                if let Err(e) = response_stream.message().await {
                    assert_eq!(e.code(), tonic::Code::Internal);
                    assert!(e.message().contains("UDF_EXECUTION_ERROR"))
                }
            });

            for _ in 0..10 {
                tokio::time::sleep(Duration::from_millis(100)).await;
                if task.is_finished() {
                    break;
                }
            }
            assert!(task.is_finished(), "gRPC server is still running");

            Ok(())
        }
    }
}