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
use chrono::{DateTime, Utc};
use std::collections::HashMap;
use std::env;
use std::sync::Arc;
use tokio::sync::{mpsc, oneshot};
use tokio::task::JoinHandle;
use tokio_stream::wrappers::ReceiverStream;
use tokio_util::sync::CancellationToken;
use tonic::{Request, Status, Streaming};

use crate::error::{Error, ErrorKind};
use tracing::{debug, error, info};

use crate::proto::metadata as metadata_pb;
use crate::proto::sink::{self as sink_pb, SinkResponse};
use crate::shared;
use shared::{ContainerType, ENV_CONTAINER_TYPE, build_panic_status, get_panic_info};

/// Default socket address for sink service
pub const SOCK_ADDR: &str = "/var/run/numaflow/sink.sock";

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

/// Default socket address for fallback sink
pub const FB_SOCK_ADDR: &str = "/var/run/numaflow/fb-sink.sock";

/// Default server info file for fallback sink
pub const FB_SERVER_INFO_FILE: &str = "/var/run/numaflow/fb-sinker-server-info";

/// Container identifier for fallback sink
const FB_CONTAINER_TYPE: &str = "fb-udsink";

/// Default socket address for onSuccess ud sink
pub const ONS_SOCK_ADDR: &str = "/var/run/numaflow/ons-sink.sock";

/// Default server info file for onSuccess ud sink
pub const ONS_SERVER_INFO_FILE: &str = "/var/run/numaflow/ons-sinker-server-info";

/// Container identifier for onSuccess ud sink
const ONS_CONTAINER_TYPE: &str = "ons-udsink";

/// Default channel size for sink service
const CHANNEL_SIZE: usize = 1000;

struct SinkService<T: Sinker> {
    handler: Arc<T>,
    shutdown_tx: mpsc::Sender<()>,
    cancellation_token: CancellationToken,
}

/// Sinker trait for implementing user defined sinks.
///
/// Types implementing this trait can be passed as user-defined sink handle.
#[tonic::async_trait]
pub trait Sinker {
    /// The sink handle is given a stream of [`SinkRequest`] and the result is [`Response`].
    ///
    /// # Example
    ///
    /// A simple log sink.
    ///
    /// ```no_run
    /// use numaflow::sink::{self, Response, SinkRequest};
    /// use std::error::Error;
    ///
    /// struct Logger;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), Box<dyn Error + Send + Sync>> {
    ///     sink::Server::new(Logger).start().await
    /// }
    ///
    /// #[tonic::async_trait]
    /// impl sink::Sinker for Logger {
    ///     async fn sink(&self, mut input: tokio::sync::mpsc::Receiver<SinkRequest>) -> Vec<Response> {
    ///         let mut responses: Vec<Response> = Vec::new();
    ///
    ///         while let Some(datum) = input.recv().await {
    ///             // do something better, but for now let's just log it.
    ///             // please note that `from_utf8` is working because the input in this
    ///             // example uses utf-8 data.
    ///             let response = match std::str::from_utf8(&datum.value) {
    ///                 Ok(v) => {
    ///                     println!("{}", v);
    ///                     // record the response
    ///                     Response::ok(datum.id)
    ///                 }
    ///                 Err(e) => Response::failure(datum.id, format!("Invalid UTF-8 sequence: {}", e)),
    ///             };
    ///
    ///             // return the responses
    ///             responses.push(response);
    ///         }
    ///
    ///         responses
    ///     }
    /// }
    /// ```
    async fn sink(&self, input: mpsc::Receiver<SinkRequest>) -> Vec<Response>;
}

/// Incoming request into the  handler of [`Sinker`].
pub struct SinkRequest {
    /// 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.
    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.
    pub watermark: DateTime<Utc>,
    /// Time of the element as seen at source or aligned after a reduce operation.
    pub event_time: DateTime<Utc>,
    /// ID is the unique id of the message to be sent to the Sink.
    pub id: String,
    /// Headers for the message.
    pub headers: HashMap<String, String>,
    /// User Metadata for the message.
    pub user_metadata: UserMetadata,
    /// System Metadata for the message.
    pub system_metadata: SystemMetadata,
}

/// Since sink is the last vertex in the pipeline, only GET methods
/// are available on SystemMetadata and UserMetadata.
/// UserMetadata is the user metadata of the message
#[derive(Debug, Clone, Default)]
pub struct UserMetadata {
    data: HashMap<String, HashMap<String, Vec<u8>>>,
}

impl UserMetadata {
    /// Create a new UserMetadata instance
    pub fn new() -> Self {
        Self {
            data: Default::default(),
        }
    }

    /// groups returns the groups of the user metadata.
    /// If there are no groups, it returns an empty vector.
    ///
    /// # Example
    ///
    /// ```no_run
    /// # use numaflow::sink::SinkRequest;
    /// # let datum: SinkRequest = unimplemented!();
    /// let umd = datum.user_metadata;
    /// let groups = umd.groups();
    /// println!("User metadata groups: {:?}", groups);
    /// ```
    pub fn groups(&self) -> Vec<String> {
        self.data.keys().cloned().collect()
    }

    /// keys returns the keys of the user metadata for the given group.
    /// If there are no keys or the group is not present, it returns an empty vector.
    ///
    /// # Example
    ///
    /// ```no_run
    /// # use numaflow::sink::SinkRequest;
    /// # let datum: SinkRequest = unimplemented!();
    /// let umd = datum.user_metadata;
    /// let keys = umd.keys("my-group");
    /// println!("Keys in my-group: {:?}", keys);
    /// ```
    pub fn keys(&self, group: &str) -> Vec<String> {
        self.data
            .get(group)
            .unwrap_or(&HashMap::new())
            .keys()
            .cloned()
            .collect()
    }

    /// value returns the value of the user metadata for the given group and key.
    /// If there is no value or the group or key is not present, it returns an empty vector.
    ///
    /// # Example
    ///
    /// ```no_run
    /// # use numaflow::sink::SinkRequest;
    /// # let datum: SinkRequest = unimplemented!();
    /// let umd = datum.user_metadata;
    /// let value = umd.value("my-group", "my-key");
    /// println!("Value: {:?}", value);
    /// ```
    pub fn value(&self, group: &str, key: &str) -> Vec<u8> {
        self.data
            .get(group)
            .unwrap_or(&HashMap::new())
            .get(key)
            .unwrap_or(&Vec::new())
            .clone()
    }
}

/// SystemMetadata is the system metadata of the message
#[derive(Debug, Clone, Default)]
pub struct SystemMetadata {
    data: HashMap<String, HashMap<String, Vec<u8>>>,
}

impl SystemMetadata {
    /// Create a new SystemMetadata instance
    pub fn new() -> Self {
        Self {
            data: Default::default(),
        }
    }

    /// groups returns the groups of the system metadata.
    /// If there are no groups, it returns an empty vector.
    ///
    /// # Example
    ///
    /// ```no_run
    /// # use numaflow::sink::SinkRequest;
    /// # let datum: SinkRequest = unimplemented!();
    /// let smd = datum.system_metadata;
    /// let groups = smd.groups();
    /// println!("System metadata groups: {:?}", groups);
    /// ```
    pub fn groups(&self) -> Vec<String> {
        self.data.keys().cloned().collect()
    }

    /// keys returns the keys of the system metadata for the given group.
    /// If there are no keys or the group is not present, it returns an empty vector.
    ///
    /// # Example
    ///
    /// ```no_run
    /// # use numaflow::sink::SinkRequest;
    /// # let datum: SinkRequest = unimplemented!();
    /// let smd = datum.system_metadata;
    /// let keys = smd.keys("system-group");
    /// println!("Keys in system-group: {:?}", keys);
    /// ```
    pub fn keys(&self, group: &str) -> Vec<String> {
        self.data
            .get(group)
            .unwrap_or(&HashMap::new())
            .keys()
            .cloned()
            .collect()
    }

    /// value returns the value of the system metadata for the given group and key.
    /// If there is no value or the group or key is not present, it returns an empty vector.
    ///
    /// # Example
    ///
    /// ```no_run
    /// # use numaflow::sink::SinkRequest;
    /// # let datum: SinkRequest = unimplemented!();
    /// let smd = datum.system_metadata;
    /// let value = smd.value("system-group", "system-key");
    /// println!("Value: {:?}", value);
    /// ```
    pub fn value(&self, group: &str, key: &str) -> Vec<u8> {
        self.data
            .get(group)
            .unwrap_or(&HashMap::new())
            .get(key)
            .unwrap_or(&Vec::new())
            .clone()
    }
}

/// Get UserMetadata from proto Metadata
fn user_metadata_from_proto(proto: Option<&metadata_pb::Metadata>) -> UserMetadata {
    let proto = match proto {
        Some(p) => p,
        None => return UserMetadata::new(),
    };

    let mut user_map = HashMap::new();
    for (group, kv_group) in &proto.user_metadata {
        // Note: In proto3 with prost, kv_group is always a valid KeyValueGroup.
        // If empty, kv_group.key_value will be an empty HashMap.
        user_map.insert(group.clone(), kv_group.key_value.clone());
    }

    UserMetadata { data: user_map }
}

/// Get SystemMetadata from proto Metadata
fn system_metadata_from_proto(proto: Option<&metadata_pb::Metadata>) -> SystemMetadata {
    let proto = match proto {
        Some(p) => p,
        None => return SystemMetadata::new(),
    };

    let mut sys_map = HashMap::new();
    for (group, kv_group) in &proto.sys_metadata {
        // Note: In proto3 with prost, kv_group is always a valid KeyValueGroup.
        // If empty, kv_group.key_value will be an empty HashMap.
        sys_map.insert(group.clone(), kv_group.key_value.clone());
    }

    SystemMetadata { data: sys_map }
}

impl From<sink_pb::sink_request::Request> for SinkRequest {
    fn from(sr: sink_pb::sink_request::Request) -> Self {
        let user_metadata = user_metadata_from_proto(sr.metadata.as_ref());
        let system_metadata = system_metadata_from_proto(sr.metadata.as_ref());

        Self {
            keys: sr.keys,
            value: sr.value,
            watermark: shared::utc_from_timestamp(sr.watermark),
            event_time: shared::utc_from_timestamp(sr.event_time),
            id: sr.id,
            headers: sr.headers,
            user_metadata,
            system_metadata,
        }
    }
}

/// Type of response from the sink handler.
pub enum ResponseType {
    /// write to the sink was successful.
    Success,
    /// write to the sink failed.
    Failure,
    /// message should be forwarded to the fallback sink.
    FallBack,
    /// message should be written to the serving store.
    Serve,
    /// message should be forwarded to the onSuccess store.
    OnSuccess,
}

#[derive(Default)]
pub struct KeyValueGroup {
    pub key_value: HashMap<String, Vec<u8>>,
}

impl From<KeyValueGroup> for metadata_pb::KeyValueGroup {
    fn from(kv: KeyValueGroup) -> Self {
        Self {
            key_value: kv.key_value,
        }
    }
}

impl From<HashMap<String, Vec<u8>>> for KeyValueGroup {
    fn from(hm: HashMap<String, Vec<u8>>) -> Self {
        Self { key_value: hm }
    }
}

impl From<HashMap<String, String>> for KeyValueGroup {
    fn from(hm: HashMap<String, String>) -> Self {
        Self {
            key_value: hm.into_iter().map(|(k, v)| (k, v.into())).collect(),
        }
    }
}

#[derive(Default)]
/// OnSuccess message contains information that needs to be sent to the OnSuccess sink.
/// The message can be different from the original message. In case same information that was
/// written to the primary sink needs to be written to the onSuccess sink, do not build/send this
/// message and rather simply send `None` to the onSuccess sink.
///
/// At least a `value` is required to build Message for OnSuccess sink.
/// # Example
///
/// ```
/// use numaflow::sink::Message;
/// use numaflow::sink::Response;
///
/// let message = Message::new(vec![1, 2, 3]);
/// let response = Response::on_success("id".to_string(), Some(message));
/// ```
pub struct Message {
    pub keys: Option<Vec<String>>,
    pub value: Vec<u8>,
    pub user_metadata: Option<HashMap<String, KeyValueGroup>>,
}

impl Message {
    /// Creates a new message for OnSuccess sink with the specified value.
    ///
    /// # Arguments
    ///
    /// * `value` - A vector of bytes representing the message's payload.
    ///
    /// # Examples
    ///
    /// ```
    /// use numaflow::sink::Message;
    ///
    /// let onSuccessMessage = Message::new(vec![1, 2, 3]).build();
    /// ```
    pub fn new(value: Vec<u8>) -> Self {
        Self {
            value,
            keys: None,
            user_metadata: None,
        }
    }

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

    /// Sets or replaces the user metadata associated with the message for OnSuccess sink.
    ///
    /// # Arguments
    ///
    /// * `user_metadata` - A hash map of strings to `KeyValueGroup` representing the user metadata.
    ///
    /// # Examples
    ///
    /// ```
    /// use numaflow::sink::{Message, KeyValueGroup};
    /// use std::collections::HashMap;
    ///
    /// let user_metadata = HashMap::from([(
    ///         "key1".to_string(),
    ///         KeyValueGroup::from(
    ///             HashMap::from([(
    ///                 "key2".to_string(),
    ///                 Vec::<u8>::from("SomeValue")
    ///             )])
    ///         ),
    ///     )]);
    /// let onSuccessMessage = Message::new(vec![1, 2, 3]).with_user_metadata(user_metadata).build();
    /// ```
    pub fn with_user_metadata(mut self, user_metadata: HashMap<String, KeyValueGroup>) -> Self {
        self.user_metadata = Some(user_metadata);
        self
    }

    /// Builds the message to be sent to the OnSuccess sink.
    ///
    /// # Examples
    ///
    /// ```
    /// use numaflow::sink::Message;
    /// use numaflow::sink::Response;
    ///
    /// let onSuccessMessage = Message::new(vec![1, 2, 3]).build();
    /// let onSuccessResponse = Response::on_success("id".to_string(), onSuccessMessage);
    /// ```
    pub fn build(self) -> Option<Self> {
        Some(self)
    }
}

/// Converts a [`Message`] into a [`sink_pb::sink_response::result::Message`].
impl From<Message> for sink_pb::sink_response::result::Message {
    fn from(msg: Message) -> Self {
        Self {
            keys: msg.keys.map_or(vec![], |keys| keys),
            value: msg.value,
            metadata: msg
                .user_metadata
                .map(|user_metadata| metadata_pb::Metadata {
                    user_metadata: user_metadata
                        .into_iter()
                        .map(|(k, v)| (k, metadata_pb::KeyValueGroup::from(v)))
                        .collect(),
                    ..Default::default()
                }),
        }
    }
}

/// The result of the call to [`Sinker::sink`] method.
pub struct Response {
    /// id is the unique ID of the message.
    pub id: String,
    /// response_type indicates the type of the response.
    pub response_type: ResponseType,
    /// err string is used to describe the error if [`ResponseType::Failure`]  is set.
    pub err: Option<String>,
    pub serve_response: Option<Vec<u8>>,
    /// Optional payload to be sent to on_success sink. Send original message to sink in case
    /// `None` is provided.
    pub on_success_msg: Option<Message>,
}

impl Response {
    /// Creates a new `Response` instance indicating a successful operation.
    pub fn ok(id: String) -> Self {
        Self {
            id,
            response_type: ResponseType::Success,
            err: None,
            serve_response: None,
            on_success_msg: None,
        }
    }

    /// Creates a new `Response` instance indicating a failed operation.
    pub fn failure(id: String, err: String) -> Self {
        Self {
            id,
            response_type: ResponseType::Failure,
            err: Some(err),
            serve_response: None,
            on_success_msg: None,
        }
    }

    /// Creates a new `Response` instance indicating a failed operation with a fallback
    /// set to 'true'. So that the message will be forwarded to the fallback sink.
    pub fn fallback(id: String) -> Self {
        Self {
            id,
            response_type: ResponseType::FallBack,
            err: None,
            serve_response: None,
            on_success_msg: None,
        }
    }

    pub fn serve(id: String, payload: Vec<u8>) -> Self {
        Self {
            id,
            response_type: ResponseType::Serve,
            err: None,
            serve_response: Some(payload),
            on_success_msg: None,
        }
    }

    /// Optional payload to be sent to on_success sink. Send original message to sink in case
    /// `None` is provided.
    pub fn on_success(id: String, payload: Option<Message>) -> Self {
        Self {
            id,
            response_type: ResponseType::OnSuccess,
            err: None,
            serve_response: None,
            on_success_msg: payload,
        }
    }
}

impl From<Response> for sink_pb::sink_response::Result {
    fn from(r: Response) -> Self {
        Self {
            id: r.id,
            status: match r.response_type {
                ResponseType::Success => sink_pb::Status::Success as i32,
                ResponseType::Failure => sink_pb::Status::Failure as i32,
                ResponseType::FallBack => sink_pb::Status::Fallback as i32,
                ResponseType::Serve => sink_pb::Status::Serve as i32,
                ResponseType::OnSuccess => sink_pb::Status::OnSuccess as i32,
            },
            err_msg: r.err.unwrap_or_default(),
            serve_response: r.serve_response,
            on_success_msg: r.on_success_msg.map(|msg| msg.into()),
        }
    }
}

#[tonic::async_trait]
impl<T> sink_pb::sink_server::Sink for SinkService<T>
where
    T: Sinker + Send + Sync + 'static,
{
    type SinkFnStream = ReceiverStream<Result<SinkResponse, Status>>;

    async fn sink_fn(
        &self,
        request: Request<Streaming<sink_pb::SinkRequest>>,
    ) -> Result<tonic::Response<Self::SinkFnStream>, Status> {
        let mut sink_stream = request.into_inner();
        let sink_handle = self.handler.clone();
        let shutdown_tx = self.shutdown_tx.clone();
        let cln_token = self.cancellation_token.clone();
        let (resp_tx, resp_rx) = mpsc::channel::<Result<SinkResponse, Status>>(CHANNEL_SIZE);

        self.perform_handshake(&mut sink_stream, &resp_tx).await?;

        let grpc_resp_tx = resp_tx.clone();
        let handle: JoinHandle<Result<(), Error>> = tokio::spawn(async move {
            Self::process_sink_stream(sink_handle, sink_stream, grpc_resp_tx).await
        });

        tokio::spawn(Self::handle_sink_errors(
            handle,
            resp_tx,
            shutdown_tx,
            cln_token,
        ));

        Ok(tonic::Response::new(ReceiverStream::new(resp_rx)))
    }

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

impl<T> SinkService<T>
where
    T: Sinker + Send + Sync + 'static,
{
    /// processes the stream of requests from the client
    async fn process_sink_stream(
        sink_handle: Arc<T>,
        mut sink_stream: Streaming<sink_pb::SinkRequest>,
        grpc_resp_tx: mpsc::Sender<Result<SinkResponse, Status>>,
    ) -> Result<(), Error> {
        // loop until the global stream has been shutdown.
        let mut global_stream_ended = false;
        while !global_stream_ended {
            // for every batch, we need to read from the stream. The end-of-batch is
            // encoded in the request.
            global_stream_ended = Self::process_sink_batch(
                sink_handle.clone(),
                &mut sink_stream,
                grpc_resp_tx.clone(),
            )
            .await?;
        }
        Ok(())
    }

    /// processes a batch of messages from the client, sends them to the sink handler and sends the
    /// responses back to the client batches are separated by an EOT message.
    /// Returns true if the global bidi-stream has ended, otherwise false.
    async fn process_sink_batch(
        sink_handle: Arc<T>,
        sink_stream: &mut Streaming<sink_pb::SinkRequest>,
        grpc_resp_tx: mpsc::Sender<Result<SinkResponse, Status>>,
    ) -> Result<bool, Error> {
        let (tx, rx) = mpsc::channel::<SinkRequest>(CHANNEL_SIZE);
        let resp_tx = grpc_resp_tx.clone();
        let sink_handle = sink_handle.clone();

        // spawn the UDF
        let sinker_handle = tokio::spawn(async move {
            let responses = sink_handle.sink(rx).await;
            if resp_tx
                .send(Ok(SinkResponse {
                    results: responses.into_iter().map(|r| r.into()).collect(),
                    handshake: None,
                    status: None,
                }))
                .await
                .is_err()
            {
                return;
            }

            // send an EOT message to the client to indicate the end of transmission for this batch
            if resp_tx
                .send(Ok(SinkResponse {
                    results: vec![],
                    handshake: None,
                    status: Some(sink_pb::TransmissionStatus { eot: true }),
                }))
                .await
                .is_err()
            {}
        });

        let mut global_stream_ended = false;

        // loop until eot happens or stream is closed.
        loop {
            let message = match sink_stream.message().await {
                Ok(Some(m)) => m,
                Ok(None) => {
                    info!("global bidi stream ended");
                    // NOTE: this will only happen during shutdown. We can be certain that there
                    // are no messages left hanging in the UDF.
                    global_stream_ended = true;
                    break; // bidi stream ended
                }
                Err(e) => {
                    error!("Error reading message from stream: {}", e);
                    global_stream_ended = true;
                    return Ok(global_stream_ended);
                }
            };

            // we are done with this batch because eot=true
            if message.status.is_some_and(|status| status.eot) {
                debug!("Batch Ended, received an EOT message");
                break;
            }

            // message.request cannot be none
            let request = message.request.ok_or_else(|| {
                Error::SinkError(ErrorKind::InternalError(
                    "Invalid argument, request can't be None".to_string(),
                ))
            })?;

            // write to the UDF's tx
            tx.send(request.into()).await.map_err(|e| {
                Error::SinkError(ErrorKind::InternalError(format!(
                    "Error sending message to sink handler: {}",
                    e
                )))
            })?;
        }

        // drop the sender to signal the sink handler that the batch has ended
        drop(tx);

        // Wait for UDF task to return with panic detection
        match sinker_handle.await {
            Ok(_) => {
                // UDF completed successfully
            }
            Err(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);

                    // Return detailed error to trigger shutdown
                    return Err(Error::GrpcStatus(status));
                } else {
                    // This is a non-panic error
                    return Err(Error::SinkError(ErrorKind::UserDefinedError(e.to_string())));
                }
            }
        }

        Ok(global_stream_ended)
    }

    /// handles errors from the sink handler and sends them to the client via the response channel
    async fn handle_sink_errors(
        handle: JoinHandle<Result<(), Error>>,
        resp_tx: mpsc::Sender<Result<SinkResponse, Status>>,
        shutdown_tx: mpsc::Sender<()>,
        cln_token: CancellationToken,
    ) {
        tokio::select! {
            resp = handle => {
                match resp {
                    Ok(Ok(_)) => {},
                    Ok(Err(e)) => {
                        resp_tx.send(Err(e.into_status())).await
                            .inspect_err(|send_err| error!("Failed to send error to response channel (receiver likely dropped): {}", send_err))
                            .ok();
                        shutdown_tx.send(()).await
                            .inspect_err(|send_err| error!("Failed to send shutdown signal: {}", send_err))
                            .ok();
                    }
                    Err(e) => {
                        resp_tx
                            .send(Err(Status::internal(format!(
                                "Sink handler aborted: {}",
                                e
                            ))))
                            .await
                            .inspect_err(|send_err| error!("Failed to send error to response channel (receiver likely dropped): {}", send_err))
                            .ok();
                        shutdown_tx.send(()).await
                            .inspect_err(|send_err| error!("Failed to send shutdown signal: {}", send_err))
                            .ok();
                    }
                }
            },
            _ = cln_token.cancelled() => {
                resp_tx
                    .send(Err(Status::cancelled("Sink handler cancelled")))
                    .await
                    .inspect_err(|send_err| error!("Token cancelled: Failed to send error to response channel: {}", send_err))
                    .ok();
            }
        }
    }

    /// performs handshake with the client
    async fn perform_handshake(
        &self,
        sink_stream: &mut Streaming<sink_pb::SinkRequest>,
        resp_tx: &mpsc::Sender<Result<SinkResponse, Status>>,
    ) -> Result<(), Status> {
        let handshake_request = sink_stream
            .message()
            .await
            .map_err(|e| Status::internal(format!("handshake failed {}", e)))?
            .ok_or_else(|| Status::internal("stream closed before handshake"))?;

        if let Some(handshake) = handshake_request.handshake {
            resp_tx
                .send(Ok(SinkResponse {
                    results: vec![],
                    handshake: Some(handshake),
                    status: None,
                }))
                .await
                .map_err(|e| {
                    Status::internal(format!("failed to send handshake response {}", e))
                })?;
            Ok(())
        } else {
            Err(Status::invalid_argument("Handshake not present"))
        }
    }
}

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

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

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

impl<T> Server<T> {
    pub fn new(svc: T) -> Self {
        let container_type = env::var(ENV_CONTAINER_TYPE).unwrap_or_default();
        let (sock_addr, server_info_file) = if container_type == FB_CONTAINER_TYPE {
            (FB_SOCK_ADDR, FB_SERVER_INFO_FILE)
        } else if container_type == ONS_CONTAINER_TYPE {
            (ONS_SOCK_ADDR, ONS_SERVER_INFO_FILE)
        } else {
            (SOCK_ADDR, SERVER_INFO_FILE)
        };

        Self {
            inner: shared::Server::new_with_custom_paths(
                svc,
                ContainerType::Sink,
                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,
        shutdown_rx: oneshot::Receiver<()>,
    ) -> Result<(), Box<dyn std::error::Error + Send + Sync>>
    where
        T: Sinker + Send + Sync + 'static,
    {
        self.inner
            .start_with_shutdown(
                shutdown_rx,
                |handler, max_message_size, shutdown_tx, cln_token| {
                    let svc = SinkService {
                        handler: Arc::new(handler),
                        shutdown_tx,
                        cancellation_token: cln_token,
                    };

                    let svc = sink_pb::sink_server::SinkServer::new(svc)
                        .max_encoding_message_size(max_message_size)
                        .max_decoding_message_size(max_message_size);

                    tonic::transport::Server::builder().add_service(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 singal arrives.
    pub async fn start(self) -> Result<(), Box<dyn std::error::Error + Send + Sync>>
    where
        T: Sinker + Send + Sync + 'static,
    {
        self.inner
            .start(|handler, max_message_size, shutdown_tx, cln_token| {
                let svc = SinkService {
                    handler: Arc::new(handler),
                    shutdown_tx,
                    cancellation_token: cln_token,
                };

                let svc = sink_pb::sink_server::SinkServer::new(svc)
                    .max_encoding_message_size(max_message_size)
                    .max_decoding_message_size(max_message_size);

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

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

    use tempfile::TempDir;
    use tokio::net::UnixStream;
    use tokio::sync::oneshot;
    use tonic::transport::Uri;
    use tower::service_fn;

    use crate::proto::sink::TransmissionStatus;
    use crate::proto::sink::sink_client::SinkClient;
    use crate::proto::sink::sink_request::Request;
    use crate::proto::sink::{Handshake, SinkRequest};
    use crate::sink;

    #[tokio::test]
    async fn sink_server() -> Result<(), Box<dyn Error>> {
        struct Logger;
        #[tonic::async_trait]
        impl sink::Sinker for Logger {
            async fn sink(
                &self,
                mut input: tokio::sync::mpsc::Receiver<sink::SinkRequest>,
            ) -> Vec<sink::Response> {
                let mut responses: Vec<sink::Response> = Vec::new();
                while let Some(datum) = input.recv().await {
                    let response = match std::str::from_utf8(&datum.value) {
                        Ok(_) => sink::Response::ok(datum.id),
                        Err(e) => sink::Response::failure(
                            datum.id,
                            format!("Invalid UTF-8 sequence: {}", e),
                        ),
                    };
                    responses.push(response);
                }
                responses
            }
        }

        let tmp_dir = TempDir::new()?;
        let sock_file = tmp_dir.path().join("sink.sock");
        let server_info_file = tmp_dir.path().join("sinker-server-info");

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

        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;

        // 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 mut client = SinkClient::new(channel);
        // Send handshake request
        let handshake_request = SinkRequest {
            request: None,
            status: None,
            handshake: Some(Handshake { sot: true }),
        };
        let request = SinkRequest {
            request: Some(Request {
                keys: vec!["first".into(), "second".into()],
                value: "hello".into(),
                watermark: Some(prost_types::Timestamp::default()),
                event_time: Some(prost_types::Timestamp::default()),
                id: "1".to_string(),
                headers: Default::default(),
                metadata: None,
            }),
            status: None,
            handshake: None,
        };

        let eot_request = SinkRequest {
            request: None,
            status: Some(TransmissionStatus { eot: true }),
            handshake: None,
        };

        let request_two = SinkRequest {
            request: Some(Request {
                keys: vec!["first".into(), "second".into()],
                value: "hello".into(),
                watermark: Some(prost_types::Timestamp::default()),
                event_time: Some(prost_types::Timestamp::default()),
                id: "2".to_string(),
                headers: Default::default(),
                metadata: None,
            }),
            status: None,
            handshake: None,
        };

        let resp = client
            .sink_fn(tokio_stream::iter(vec![
                handshake_request,
                request,
                eot_request.clone(),
                request_two,
                eot_request,
            ]))
            .await?;

        let mut resp_stream = resp.into_inner();
        // handshake response
        let resp = resp_stream.message().await.unwrap().unwrap();
        assert!(resp.handshake.is_some());

        let resp = resp_stream.message().await.unwrap().unwrap();
        assert!(!resp.results.is_empty());
        let msg = resp.results.first().unwrap();
        assert_eq!(msg.err_msg, "");
        assert_eq!(msg.id, "1");

        // eot for first request
        let resp = resp_stream.message().await.unwrap().unwrap();
        assert!(resp.results.is_empty());
        assert!(resp.handshake.is_none());
        let msg = &resp.status.unwrap();
        assert!(msg.eot);

        let resp = resp_stream.message().await.unwrap().unwrap();
        assert!(!resp.results.is_empty());
        assert!(resp.handshake.is_none());
        let msg = resp.results.first().unwrap();
        assert_eq!(msg.err_msg, "");
        assert_eq!(msg.id, "2");

        // eot for second request
        let resp = resp_stream.message().await.unwrap().unwrap();
        assert!(resp.results.is_empty());
        assert!(resp.handshake.is_none());
        let msg = &resp.status.unwrap();
        assert!(msg.eot);

        shutdown_tx
            .send(())
            .expect("Sending shutdown signal to gRPC server");
        tokio::time::sleep(Duration::from_millis(50)).await;
        assert!(task.is_finished(), "gRPC server is still running");
        Ok(())
    }

    #[cfg(feature = "test-panic")]
    #[tokio::test]
    async fn sink_panic() -> Result<(), Box<dyn Error>> {
        struct PanicSink;
        #[tonic::async_trait]
        impl sink::Sinker for PanicSink {
            async fn sink(
                &self,
                mut input: tokio::sync::mpsc::Receiver<sink::SinkRequest>,
            ) -> Vec<sink::Response> {
                let mut responses: Vec<sink::Response> = Vec::new();
                let mut count = 0;

                while let Some(datum) = input.recv().await {
                    if count > 5 {
                        panic!("Should not cross 5");
                    }
                    count += 1;
                    responses.push(sink::Response::ok(datum.id));
                }
                responses
            }
        }

        let tmp_dir = TempDir::new()?;
        let sock_file = tmp_dir.path().join("sink.sock");
        let server_info_file = tmp_dir.path().join("sinker-server-info");

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

        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;

        // 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 mut client = SinkClient::new(channel);
        // Send handshake request
        let handshake_request = SinkRequest {
            request: None,
            status: None,
            handshake: Some(Handshake { sot: true }),
        };

        let mut requests = vec![handshake_request];

        for i in 0..10 {
            let request = SinkRequest {
                request: Some(Request {
                    keys: vec!["first".into(), "second".into()],
                    value: format!("hello {}", i).into(),
                    watermark: Some(prost_types::Timestamp::default()),
                    event_time: Some(prost_types::Timestamp::default()),
                    id: i.to_string(),
                    headers: Default::default(),
                    metadata: None,
                }),
                status: None,
                handshake: None,
            };
            requests.push(request);
        }

        requests.push(SinkRequest {
            request: None,
            status: Some(TransmissionStatus { eot: true }),
            handshake: None,
        });

        let mut resp_stream = client
            .sink_fn(tokio_stream::iter(requests))
            .await
            .unwrap()
            .into_inner();

        // handshake response
        let resp = resp_stream.message().await.unwrap().unwrap();
        assert!(resp.results.is_empty());
        assert!(resp.handshake.is_some());

        let err_resp = resp_stream.message().await;
        assert!(err_resp.is_err());

        if let Err(e) = err_resp {
            assert_eq!(e.code(), tonic::Code::Internal);
            assert!(e.message().contains("UDF_EXECUTION_ERROR"));
            assert!(e.message().contains("Should not cross 5"));
        }

        // server should shut down gracefully because there was a panic in the handler.
        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(())
    }
}