rskafka 0.6.0

A minimal Rust client for Apache Kafka
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
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
use std::{
    collections::HashMap,
    future::Future,
    io::Cursor,
    ops::DerefMut,
    sync::{
        Arc,
        atomic::{AtomicI32, Ordering},
    },
    task::Poll,
};

use futures::future::BoxFuture;
use parking_lot::Mutex;
use rsasl::{
    mechname::MechanismNameError,
    prelude::{Mechname, SASLError, SessionError},
};
use thiserror::Error;
use tokio::{
    io::{AsyncRead, AsyncWrite, AsyncWriteExt, WriteHalf},
    sync::{
        Mutex as AsyncMutex,
        oneshot::{Sender, channel},
    },
    task::JoinHandle,
};
use tracing::{debug, info, warn};

use crate::protocol::{messages::ApiVersionsRequest, traits::ReadType};
use crate::{
    backoff::ErrorOrThrottle,
    protocol::{
        api_key::ApiKey,
        api_version::ApiVersion,
        error::Error as ApiError,
        frame::{AsyncMessageRead, AsyncMessageWrite},
        messages::{
            ReadVersionedError, ReadVersionedType, RequestBody, RequestHeader, ResponseHeader,
            SaslAuthenticateRequest, SaslAuthenticateResponse, SaslHandshakeRequest,
            SaslHandshakeResponse, WriteVersionedError, WriteVersionedType,
        },
        primitives::{Int16, Int32, NullableString, TaggedFields},
    },
    throttle::maybe_throttle,
};
use crate::{
    client::SaslConfig,
    protocol::{api_version::ApiVersionRange, primitives::CompactString},
};

#[derive(Debug)]
struct Response {
    #[allow(dead_code)]
    header: ResponseHeader,
    data: Cursor<Vec<u8>>,
}

#[derive(Debug)]
struct ActiveRequest {
    channel: Sender<Result<Response, RequestError>>,
    use_tagged_fields_in_response: bool,
}

#[derive(Debug)]
enum MessengerState {
    /// Currently active requests by correlation ID.
    ///
    /// An active request is one that got prepared or send but the response wasn't received yet.
    RequestMap(HashMap<i32, ActiveRequest>),

    /// One or our streams died and we are unable to process any more requests.
    Poison(Arc<RequestError>),
}

impl MessengerState {
    fn poison(&mut self, err: RequestError) -> Arc<RequestError> {
        match self {
            Self::RequestMap(map) => {
                let err = Arc::new(err);

                // inform all active requests
                for (_correlation_id, active_request) in map.drain() {
                    // it's OK if the other side is gone
                    active_request
                        .channel
                        .send(Err(RequestError::Poisoned(Arc::clone(&err))))
                        .ok();
                }

                *self = Self::Poison(Arc::clone(&err));
                err
            }
            Self::Poison(e) => {
                // already poisoned, used existing error
                Arc::clone(e)
            }
        }
    }
}

/// A connection to a single broker
///
/// Note: Requests to the same [`Messenger`] will be pipelined by Kafka
///
#[derive(Debug)]
pub struct Messenger<RW> {
    /// The half of the stream that we use to send data TO the broker.
    ///
    /// This will be used by [`request`](Self::request) to queue up messages.
    stream_write: Arc<AsyncMutex<WriteHalf<RW>>>,

    /// Client ID.
    client_id: Arc<str>,

    /// The next correlation ID.
    ///
    /// This is used to map responses to active requests.
    correlation_id: AtomicI32,

    /// Version ranges that we think are supported by the broker.
    ///
    /// This needs to be bootstrapped by [`sync_versions`](Self::sync_versions).
    version_ranges: HashMap<ApiKey, ApiVersionRange>,

    /// Current stream state.
    ///
    /// Note that this and `stream_write` are separate struct to allow sending and receiving data concurrently.
    state: Arc<Mutex<MessengerState>>,

    /// Join handle for the background worker that fetches responses.
    join_handle: JoinHandle<()>,
}

#[derive(Error, Debug)]
#[non_exhaustive]
pub enum RequestError {
    #[error("Cannot find matching version for: {api_key:?}")]
    NoVersionMatch { api_key: ApiKey },

    #[error("Cannot write data: {0}")]
    WriteError(#[from] WriteVersionedError),

    #[error("Cannot write versioned data: {0}")]
    WriteMessageError(#[from] crate::protocol::frame::WriteError),

    #[error("Cannot read data: {0}")]
    ReadError(#[from] crate::protocol::traits::ReadError),

    #[error("Cannot read versioned data: {0}")]
    ReadVersionedError(#[from] ReadVersionedError),

    #[error("Cannot read/write data: {0}")]
    IO(#[from] std::io::Error),

    #[error(
        "Data left at the end of the message. Got {message_size} bytes but only read {read} bytes. api_key={api_key:?} api_version={api_version}"
    )]
    TooMuchData {
        message_size: u64,
        read: u64,
        api_key: ApiKey,
        api_version: ApiVersion,
    },

    #[error("Cannot read framed message: {0}")]
    ReadFramedMessageError(#[from] crate::protocol::frame::ReadError),

    #[error("Connection is poisoned: {0}")]
    Poisoned(Arc<RequestError>),
}

#[derive(Error, Debug)]
#[non_exhaustive]
pub enum SyncVersionsError {
    #[error("Did not found a version for ApiVersion that works with that broker")]
    NoWorkingVersion,

    #[error("Request error: {0}")]
    RequestError(#[from] RequestError),

    #[error("Got flipped version from server for API key {api_key:?}: min={min:?} max={max:?}")]
    FlippedVersionRange {
        api_key: ApiKey,
        min: ApiVersion,
        max: ApiVersion,
    },
}

#[derive(Error, Debug)]
pub enum SaslError {
    #[error("Request error: {0}")]
    RequestError(#[from] RequestError),

    #[error("API error: {0}")]
    ApiError(#[from] ApiError),

    #[error("Invalid sasl mechanism: {0}")]
    InvalidSaslMechanism(#[from] MechanismNameError),

    #[error("Sasl session error: {0}")]
    SaslSessionError(#[from] SessionError),

    #[error("Invalid SASL config: {0}")]
    InvalidConfig(#[from] SASLError),

    #[error("Error in user defined callback: {0}")]
    Callback(Box<dyn std::error::Error + Send + Sync>),

    #[error("unsupported sasl mechanism")]
    UnsupportedSaslMechanism,
}

impl<RW> Messenger<RW>
where
    RW: AsyncRead + AsyncWrite + Send + 'static,
{
    pub fn new(stream: RW, max_message_size: usize, client_id: Arc<str>) -> Self {
        let (stream_read, stream_write) = tokio::io::split(stream);
        let state = Arc::new(Mutex::new(MessengerState::RequestMap(HashMap::default())));
        let state_captured = Arc::clone(&state);

        let join_handle = tokio::spawn(async move {
            let mut stream_read = stream_read;

            loop {
                match stream_read.read_message(max_message_size).await {
                    Ok(msg) => {
                        // message was read, so all subsequent errors should not poison the whole stream
                        let mut cursor = Cursor::new(msg);

                        // read header as version 0 (w/o tagged fields) first since this is a strict prefix or the more advanced
                        // header version
                        let mut header =
                            match ResponseHeader::read_versioned(&mut cursor, ApiVersion(Int16(0)))
                            {
                                Ok(header) => header,
                                Err(e) => {
                                    warn!(%e, "Cannot read message header, ignoring message");
                                    continue;
                                }
                            };

                        let active_request = match state_captured.lock().deref_mut() {
                            MessengerState::RequestMap(map) => {
                                match map.remove(&header.correlation_id.0) {
                                    Some(active_request) => active_request,
                                    _ => {
                                        warn!(
                                            correlation_id = header.correlation_id.0,
                                            "Got response for unknown request",
                                        );
                                        continue;
                                    }
                                }
                            }
                            MessengerState::Poison(_) => {
                                // stream is poisoned, no need to anything
                                return;
                            }
                        };

                        // optionally read tagged fields from the header as well
                        if active_request.use_tagged_fields_in_response {
                            header.tagged_fields = match TaggedFields::read(&mut cursor) {
                                Ok(fields) => Some(fields),
                                Err(e) => {
                                    // we don't care if the other side is gone
                                    active_request
                                        .channel
                                        .send(Err(RequestError::ReadError(e)))
                                        .ok();
                                    continue;
                                }
                            };
                        }

                        // we don't care if the other side is gone
                        active_request
                            .channel
                            .send(Ok(Response {
                                header,
                                data: cursor,
                            }))
                            .ok();
                    }
                    Err(e) => {
                        state_captured
                            .lock()
                            .poison(RequestError::ReadFramedMessageError(e));
                        return;
                    }
                }
            }
        });

        Self {
            stream_write: Arc::new(AsyncMutex::new(stream_write)),
            client_id,
            correlation_id: AtomicI32::new(0),
            version_ranges: HashMap::new(),
            state,
            join_handle,
        }
    }

    #[cfg(feature = "unstable-fuzzing")]
    pub fn override_version_ranges(&mut self, ranges: HashMap<ApiKey, ApiVersionRange>) {
        self.set_version_ranges(ranges);
    }

    /// Set supported version range.
    fn set_version_ranges(&mut self, ranges: HashMap<ApiKey, ApiVersionRange>) {
        self.version_ranges = ranges;
    }

    pub async fn request<R>(&self, msg: R) -> Result<R::ResponseBody, RequestError>
    where
        R: RequestBody + Send + WriteVersionedType<Vec<u8>>,
        R::ResponseBody: ReadVersionedType<Cursor<Vec<u8>>>,
    {
        self.request_with_version_ranges(msg, &self.version_ranges)
            .await
    }

    async fn request_with_version_ranges<R>(
        &self,
        msg: R,
        version_ranges: &HashMap<ApiKey, ApiVersionRange>,
    ) -> Result<R::ResponseBody, RequestError>
    where
        R: RequestBody + Send + WriteVersionedType<Vec<u8>>,
        R::ResponseBody: ReadVersionedType<Cursor<Vec<u8>>>,
    {
        let body_api_version = version_ranges
            .get(&R::API_KEY)
            .and_then(|range_server| match_versions(*range_server, R::API_VERSION_RANGE))
            .ok_or(RequestError::NoVersionMatch {
                api_key: R::API_KEY,
            })?;

        // determine if our request and response headers shall contain tagged fields. This system is borrowed from
        // rdkafka ("flexver"), see:
        // - https://github.com/edenhill/librdkafka/blob/2b76b65212e5efda213961d5f84e565038036270/src/rdkafka_request.c#L973
        // - https://github.com/edenhill/librdkafka/blob/2b76b65212e5efda213961d5f84e565038036270/src/rdkafka_buf.c#L167-L174
        let use_tagged_fields_in_request =
            body_api_version >= R::FIRST_TAGGED_FIELD_IN_REQUEST_VERSION;
        let use_tagged_fields_in_response =
            body_api_version >= R::FIRST_TAGGED_FIELD_IN_RESPONSE_VERSION;

        // Correlation ID so that we can de-multiplex the responses.
        let correlation_id = self.correlation_id.fetch_add(1, Ordering::SeqCst);

        let header = RequestHeader {
            request_api_key: R::API_KEY,
            request_api_version: body_api_version,
            correlation_id: Int32(correlation_id),
            // Technically we don't need to send a client_id, but newer redpanda version fail to parse the message
            // without it. See https://github.com/influxdata/rskafka/issues/169 .
            client_id: Some(NullableString(Some(String::from(self.client_id.as_ref())))),
            tagged_fields: Some(TaggedFields::default()),
        };
        let header_version = if use_tagged_fields_in_request {
            ApiVersion(Int16(2))
        } else {
            ApiVersion(Int16(1))
        };

        let mut buf = Vec::new();
        header
            .write_versioned(&mut buf, header_version)
            .expect("Writing header to buffer should always work");
        msg.write_versioned(&mut buf, body_api_version)?;

        let (tx, rx) = channel();

        // to prevent stale data in inner state, ensure that we would remove the request again if we are cancelled while
        // sending the request
        let cleanup_on_cancel =
            CleanupRequestStateOnCancel::new(Arc::clone(&self.state), correlation_id);

        match self.state.lock().deref_mut() {
            MessengerState::RequestMap(map) => {
                map.insert(
                    correlation_id,
                    ActiveRequest {
                        channel: tx,
                        use_tagged_fields_in_response,
                    },
                );
            }
            MessengerState::Poison(e) => {
                return Err(RequestError::Poisoned(Arc::clone(e)));
            }
        }

        self.send_message(buf).await?;
        cleanup_on_cancel.message_sent();

        let mut response = rx.await.expect("Who closed this channel?!")?;
        let body = R::ResponseBody::read_versioned(&mut response.data, body_api_version)?;

        // check if we fully consumed the message, otherwise there might be a bug in our protocol code
        let read_bytes = response.data.position();
        let message_bytes = response.data.into_inner().len() as u64;
        if read_bytes != message_bytes {
            return Err(RequestError::TooMuchData {
                message_size: message_bytes,
                read: read_bytes,
                api_key: R::API_KEY,
                api_version: body_api_version,
            });
        }

        Ok(body)
    }

    async fn send_message(&self, msg: Vec<u8>) -> Result<(), RequestError> {
        match self.send_message_inner(msg).await {
            Ok(()) => Ok(()),
            Err(e) => {
                // need to poison the stream because message framing might be out-of-sync
                let mut state = self.state.lock();
                Err(RequestError::Poisoned(state.poison(e)))
            }
        }
    }

    async fn send_message_inner(&self, msg: Vec<u8>) -> Result<(), RequestError> {
        let mut stream_write = Arc::clone(&self.stream_write).lock_owned().await;

        // use a wrapper so that cancelation doesn't cancel the send operation and leaves half-send messages on the wire
        let fut = CancellationSafeFuture::new(async move {
            stream_write.write_message(&msg).await?;
            stream_write.flush().await?;
            Ok(())
        });

        fut.await
    }

    /// Sync supported version range.
    ///
    /// Takes `&self mut` to ensure exclusive access.
    pub async fn sync_versions(&mut self) -> Result<(), SyncVersionsError> {
        'iter_upper_bound: for upper_bound in (ApiVersionsRequest::API_VERSION_RANGE.min().0.0
            ..=ApiVersionsRequest::API_VERSION_RANGE.max().0.0)
            .rev()
        {
            let version_ranges = HashMap::from([(
                ApiKey::ApiVersions,
                ApiVersionRange::new(
                    ApiVersionsRequest::API_VERSION_RANGE.min(),
                    ApiVersion(Int16(upper_bound)),
                ),
            )]);

            let body = ApiVersionsRequest {
                client_software_name: Some(CompactString(String::from(env!("CARGO_PKG_NAME")))),
                client_software_version: Some(CompactString(String::from(env!(
                    "CARGO_PKG_VERSION"
                )))),
                tagged_fields: Some(TaggedFields::default()),
            };

            'throttle: loop {
                match self
                    .request_with_version_ranges(&body, &version_ranges)
                    .await
                {
                    Ok(response) => {
                        if let Err(ErrorOrThrottle::Throttle(throttle)) =
                            maybe_throttle::<SyncVersionsError>(response.throttle_time_ms)
                        {
                            info!(
                                ?throttle,
                                request_name = "version sync",
                                "broker asked us to throttle"
                            );
                            tokio::time::sleep(throttle).await;
                            continue 'throttle;
                        }

                        if let Some(e) = response.error_code {
                            debug!(
                                %e,
                                version=upper_bound,
                                "Got error during version sync, cannot use version for ApiVersionRequest",
                            );
                            continue 'iter_upper_bound;
                        }

                        // check range sanity
                        for api_key in &response.api_keys {
                            if api_key.min_version.0 > api_key.max_version.0 {
                                return Err(SyncVersionsError::FlippedVersionRange {
                                    api_key: api_key.api_key,
                                    min: api_key.min_version,
                                    max: api_key.max_version,
                                });
                            }
                        }

                        let ranges = response
                            .api_keys
                            .into_iter()
                            .map(|x| {
                                (
                                    x.api_key,
                                    ApiVersionRange::new(x.min_version, x.max_version),
                                )
                            })
                            .collect();
                        debug!(
                            versions=%sorted_ranges_repr(&ranges),
                            "Detected supported broker versions",
                        );
                        self.set_version_ranges(ranges);
                        return Ok(());
                    }
                    Err(RequestError::NoVersionMatch { .. }) => {
                        unreachable!("Just set to version range to a non-empty range")
                    }
                    Err(RequestError::ReadVersionedError(e)) => {
                        debug!(
                            %e,
                            version=upper_bound,
                            "Cannot read ApiVersionResponse for version",
                        );
                        continue 'iter_upper_bound;
                    }
                    Err(RequestError::ReadError(e)) => {
                        debug!(
                            %e,
                            version=upper_bound,
                            "Cannot read ApiVersionResponse for version",
                        );
                        continue 'iter_upper_bound;
                    }
                    Err(e @ RequestError::TooMuchData { .. }) => {
                        debug!(
                            %e,
                            version=upper_bound,
                            "Cannot read ApiVersionResponse for version",
                        );
                        continue 'iter_upper_bound;
                    }
                    Err(e) => {
                        return Err(SyncVersionsError::RequestError(e));
                    }
                }
            }
        }

        Err(SyncVersionsError::NoWorkingVersion)
    }

    async fn sasl_authentication(
        &self,
        auth_bytes: Vec<u8>,
    ) -> Result<SaslAuthenticateResponse, SaslError> {
        let req = SaslAuthenticateRequest::new(auth_bytes);
        let resp = self.request(req).await?;
        if let Some(err) = resp.error_code {
            if let Some(s) = resp.error_message.0 {
                debug!("Sasl auth error message: {s}");
            }
            return Err(SaslError::ApiError(err));
        }

        Ok(resp)
    }

    async fn sasl_handshake(&self, mechanism: &str) -> Result<SaslHandshakeResponse, SaslError> {
        let req = SaslHandshakeRequest::new(mechanism);
        let resp = self.request(req).await?;
        if let Some(err) = resp.error_code {
            return Err(SaslError::ApiError(err));
        }
        Ok(resp)
    }

    pub async fn do_sasl(&self, config: SaslConfig) -> Result<(), SaslError> {
        let mechanism = config.mechanism();
        let resp = self.sasl_handshake(mechanism).await?;

        let config = config.get_sasl_config().await?;
        let sasl = rsasl::prelude::SASLClient::new(config);
        let raw_mechanisms = resp.mechanisms.0.unwrap_or_default();
        let mechanisms = raw_mechanisms
            .iter()
            .map(|mech| Mechname::parse(mech.0.as_bytes()).map_err(SaslError::InvalidSaslMechanism))
            .collect::<Result<Vec<_>, SaslError>>()?;
        debug!(?mechanisms, "Supported SASL mechanisms");
        let prefer_mechanism =
            Mechname::parse(mechanism.as_bytes()).map_err(SaslError::InvalidSaslMechanism)?;
        if !mechanisms.contains(&prefer_mechanism) {
            return Err(SaslError::UnsupportedSaslMechanism);
        }
        let mut session = sasl
            .start_suggested(&[prefer_mechanism])
            .map_err(|_| SaslError::UnsupportedSaslMechanism)?;
        debug!(?mechanism, "Using SASL Mechanism");
        // we step through the auth process, starting on our side with NO data received so far
        let mut data_received: Option<Vec<u8>> = None;
        loop {
            let mut to_sent = Cursor::new(Vec::new());
            let state = session.step(data_received.as_deref(), &mut to_sent)?;

            if state.has_sent_message() {
                let authentication_response =
                    self.sasl_authentication(to_sent.into_inner()).await?;
                data_received = Some(authentication_response.auth_bytes.0);
            }

            if state.is_finished() {
                break;
            }
        }

        Ok(())
    }
}

impl<RW> Drop for Messenger<RW> {
    fn drop(&mut self) {
        self.join_handle.abort();
    }
}

fn sorted_ranges_repr(ranges: &HashMap<ApiKey, ApiVersionRange>) -> String {
    let mut ranges: Vec<_> = ranges.iter().map(|(key, range)| (*key, *range)).collect();
    ranges.sort_by_key(|(key, _range)| *key);
    let ranges: Vec<_> = ranges
        .into_iter()
        .map(|(key, range)| format!("{:?}: {}", key, range))
        .collect();
    ranges.join(", ")
}

fn match_versions(range_a: ApiVersionRange, range_b: ApiVersionRange) -> Option<ApiVersion> {
    if range_a.min() <= range_b.max() && range_b.min() <= range_a.max() {
        Some(range_a.max().min(range_b.max()))
    } else {
        None
    }
}

/// Helper that ensures that a request is removed when a request is cancelled before it was actually sent out.
struct CleanupRequestStateOnCancel {
    state: Arc<Mutex<MessengerState>>,
    correlation_id: i32,
    message_sent: bool,
}

impl CleanupRequestStateOnCancel {
    /// Create new helper.
    ///
    /// You must call [`message_sent`](Self::message_sent) when the request was sent.
    fn new(state: Arc<Mutex<MessengerState>>, correlation_id: i32) -> Self {
        Self {
            state,
            correlation_id,
            message_sent: false,
        }
    }

    /// Request was sent. Do NOT clean the state any longer.
    fn message_sent(mut self) {
        self.message_sent = true;
    }
}

impl Drop for CleanupRequestStateOnCancel {
    fn drop(&mut self) {
        if !self.message_sent {
            if let MessengerState::RequestMap(map) = self.state.lock().deref_mut() {
                map.remove(&self.correlation_id);
            }
        }
    }
}

/// Wrapper around a future that cannot be cancelled.
///
/// When the future is dropped/cancelled, we'll spawn a tokio task to _rescue_ it.
struct CancellationSafeFuture<F>
where
    F: Future + Send + 'static,
{
    /// Mark if the inner future finished. If not, we must spawn a helper task on drop.
    done: bool,

    /// Inner future.
    ///
    /// Wrapped in an `Option` so we can extract it during drop. Inside that option however we also need a pinned
    /// box because once this wrapper is polled, it will be pinned in memory -- even during drop. Now the inner
    /// future does not necessarily implement `Unpin`, so we need a heap allocation to pin it in memory even when we
    /// move it out of this option.
    inner: Option<BoxFuture<'static, F::Output>>,
}

impl<F> Drop for CancellationSafeFuture<F>
where
    F: Future + Send + 'static,
{
    fn drop(&mut self) {
        if !self.done {
            let inner = self.inner.take().expect("Double-drop?");
            tokio::task::spawn(async move {
                inner.await;
            });
        }
    }
}

impl<F> CancellationSafeFuture<F>
where
    F: Future + Send,
{
    fn new(fut: F) -> Self {
        Self {
            done: false,
            inner: Some(Box::pin(fut)),
        }
    }
}

impl<F> Future for CancellationSafeFuture<F>
where
    F: Future + Send,
{
    type Output = F::Output;

    fn poll(
        mut self: std::pin::Pin<&mut Self>,
        cx: &mut std::task::Context<'_>,
    ) -> Poll<Self::Output> {
        match self.inner.as_mut().expect("no dropped").as_mut().poll(cx) {
            Poll::Ready(res) => {
                self.done = true;
                Poll::Ready(res)
            }
            Poll::Pending => Poll::Pending,
        }
    }
}

#[cfg(test)]
mod tests {
    use std::time::Duration;

    use assert_matches::assert_matches;
    use futures::{FutureExt, pin_mut};
    use tokio::{
        io::{AsyncReadExt, DuplexStream},
        sync::{Barrier, mpsc::UnboundedSender},
    };

    use super::*;

    use crate::{
        build_info::DEFAULT_CLIENT_ID,
        protocol::{
            error::Error as ApiError,
            messages::{
                ApiVersionsResponse, ApiVersionsResponseApiKey, ListOffsetsRequest, NORMAL_CONSUMER,
            },
            traits::WriteType,
        },
    };

    #[test]
    fn test_match_versions() {
        assert_eq!(
            match_versions(
                ApiVersionRange::new(ApiVersion(Int16(10)), ApiVersion(Int16(20))),
                ApiVersionRange::new(ApiVersion(Int16(10)), ApiVersion(Int16(20))),
            ),
            Some(ApiVersion(Int16(20))),
        );

        assert_eq!(
            match_versions(
                ApiVersionRange::new(ApiVersion(Int16(10)), ApiVersion(Int16(15))),
                ApiVersionRange::new(ApiVersion(Int16(13)), ApiVersion(Int16(20))),
            ),
            Some(ApiVersion(Int16(15))),
        );

        assert_eq!(
            match_versions(
                ApiVersionRange::new(ApiVersion(Int16(10)), ApiVersion(Int16(15))),
                ApiVersionRange::new(ApiVersion(Int16(15)), ApiVersion(Int16(20))),
            ),
            Some(ApiVersion(Int16(15))),
        );

        assert_eq!(
            match_versions(
                ApiVersionRange::new(ApiVersion(Int16(10)), ApiVersion(Int16(14))),
                ApiVersionRange::new(ApiVersion(Int16(15)), ApiVersion(Int16(20))),
            ),
            None,
        );
    }

    #[tokio::test]
    async fn test_sync_versions_ok() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));

        // construct response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(0),
            tagged_fields: Default::default(), // NOT serialized for ApiVersion!
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        ApiVersionsResponse {
            error_code: None,
            api_keys: vec![ApiVersionsResponseApiKey {
                api_key: ApiKey::Produce,
                min_version: ApiVersion(Int16(1)),
                max_version: ApiVersion(Int16(5)),
                tagged_fields: Default::default(),
            }],
            throttle_time_ms: None,
            tagged_fields: None,
        }
        .write_versioned(&mut msg, ApiVersionsRequest::API_VERSION_RANGE.max())
        .unwrap();
        sim.push(msg);

        // sync versions
        messenger.sync_versions().await.unwrap();
        let expected = HashMap::from([(
            (ApiKey::Produce),
            ApiVersionRange::new(ApiVersion(Int16(1)), ApiVersion(Int16(5))),
        )]);
        assert_eq!(messenger.version_ranges, expected);
    }

    #[tokio::test]
    async fn test_sync_versions_ignores_error_code() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));

        // construct error response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(0),
            tagged_fields: Default::default(), // NOT serialized for ApiVersion!
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        ApiVersionsResponse {
            error_code: Some(ApiError::CorruptMessage),
            api_keys: vec![ApiVersionsResponseApiKey {
                api_key: ApiKey::Produce,
                min_version: ApiVersion(Int16(2)),
                max_version: ApiVersion(Int16(3)),
                tagged_fields: Default::default(),
            }],
            throttle_time_ms: None,
            tagged_fields: None,
        }
        .write_versioned(&mut msg, ApiVersionsRequest::API_VERSION_RANGE.max())
        .unwrap();
        sim.push(msg);

        // construct good response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(1),
            tagged_fields: Default::default(),
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        ApiVersionsResponse {
            error_code: None,
            api_keys: vec![ApiVersionsResponseApiKey {
                api_key: ApiKey::Produce,
                min_version: ApiVersion(Int16(1)),
                max_version: ApiVersion(Int16(5)),
                tagged_fields: Default::default(),
            }],
            throttle_time_ms: None,
            tagged_fields: None,
        }
        .write_versioned(
            &mut msg,
            ApiVersion(Int16(ApiVersionsRequest::API_VERSION_RANGE.max().0.0 - 1)),
        )
        .unwrap();
        sim.push(msg);

        // sync versions
        messenger.sync_versions().await.unwrap();
        let expected = HashMap::from([(
            (ApiKey::Produce),
            ApiVersionRange::new(ApiVersion(Int16(1)), ApiVersion(Int16(5))),
        )]);
        assert_eq!(messenger.version_ranges, expected);
    }

    #[tokio::test]
    async fn test_sync_versions_ignores_read_code() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));

        // construct error response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(0),
            tagged_fields: Default::default(), // NOT serialized for ApiVersion!
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        msg.push(b'\0'); // malformed message body which can happen if the server doesn't really support this version
        sim.push(msg);

        // construct good response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(1),
            tagged_fields: Default::default(),
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        ApiVersionsResponse {
            error_code: None,
            api_keys: vec![ApiVersionsResponseApiKey {
                api_key: ApiKey::Produce,
                min_version: ApiVersion(Int16(1)),
                max_version: ApiVersion(Int16(5)),
                tagged_fields: Default::default(),
            }],
            throttle_time_ms: None,
            tagged_fields: None,
        }
        .write_versioned(
            &mut msg,
            ApiVersion(Int16(ApiVersionsRequest::API_VERSION_RANGE.max().0.0 - 1)),
        )
        .unwrap();
        sim.push(msg);

        // sync versions
        messenger.sync_versions().await.unwrap();
        let expected = HashMap::from([(
            (ApiKey::Produce),
            ApiVersionRange::new(ApiVersion(Int16(1)), ApiVersion(Int16(5))),
        )]);
        assert_eq!(messenger.version_ranges, expected);
    }

    #[tokio::test]
    async fn test_sync_versions_err_flipped_range() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));

        // construct response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(0),
            tagged_fields: Default::default(), // NOT serialized for ApiVersion!
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        ApiVersionsResponse {
            error_code: None,
            api_keys: vec![ApiVersionsResponseApiKey {
                api_key: ApiKey::Produce,
                min_version: ApiVersion(Int16(2)),
                max_version: ApiVersion(Int16(1)),
                tagged_fields: Default::default(),
            }],
            throttle_time_ms: None,
            tagged_fields: None,
        }
        .write_versioned(&mut msg, ApiVersionsRequest::API_VERSION_RANGE.max())
        .unwrap();
        sim.push(msg);

        // sync versions
        let err = messenger.sync_versions().await.unwrap_err();
        assert_matches!(err, SyncVersionsError::FlippedVersionRange { .. });
    }

    #[tokio::test]
    async fn test_sync_versions_ignores_garbage() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));

        // construct response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(0),
            tagged_fields: Default::default(), // NOT serialized for ApiVersion!
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        ApiVersionsResponse {
            error_code: None,
            api_keys: vec![ApiVersionsResponseApiKey {
                api_key: ApiKey::Produce,
                min_version: ApiVersion(Int16(1)),
                max_version: ApiVersion(Int16(2)),
                tagged_fields: Default::default(),
            }],
            throttle_time_ms: None,
            tagged_fields: None,
        }
        .write_versioned(&mut msg, ApiVersionsRequest::API_VERSION_RANGE.max())
        .unwrap();
        msg.push(b'\0'); // add junk to the end of the message to trigger `TooMuchData`
        sim.push(msg);

        // construct good response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(1),
            tagged_fields: Default::default(),
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        ApiVersionsResponse {
            error_code: None,
            api_keys: vec![ApiVersionsResponseApiKey {
                api_key: ApiKey::Produce,
                min_version: ApiVersion(Int16(1)),
                max_version: ApiVersion(Int16(5)),
                tagged_fields: Default::default(),
            }],
            throttle_time_ms: None,
            tagged_fields: None,
        }
        .write_versioned(
            &mut msg,
            ApiVersion(Int16(ApiVersionsRequest::API_VERSION_RANGE.max().0.0 - 1)),
        )
        .unwrap();
        sim.push(msg);

        // sync versions
        messenger.sync_versions().await.unwrap();
        let expected = HashMap::from([(
            (ApiKey::Produce),
            ApiVersionRange::new(ApiVersion(Int16(1)), ApiVersion(Int16(5))),
        )]);
        assert_eq!(messenger.version_ranges, expected);
    }

    #[tokio::test]
    async fn test_sync_versions_err_no_working_version() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));

        // construct error response
        for (i, v) in ((ApiVersionsRequest::API_VERSION_RANGE.min().0.0)
            ..=(ApiVersionsRequest::API_VERSION_RANGE.max().0.0))
            .rev()
            .enumerate()
        {
            let mut msg = vec![];
            ResponseHeader {
                correlation_id: Int32(i as i32),
                tagged_fields: Default::default(),
            }
            .write_versioned(&mut msg, ApiVersion(Int16(0)))
            .unwrap();
            ApiVersionsResponse {
                error_code: Some(ApiError::CorruptMessage),
                api_keys: vec![ApiVersionsResponseApiKey {
                    api_key: ApiKey::Produce,
                    min_version: ApiVersion(Int16(1)),
                    max_version: ApiVersion(Int16(5)),
                    tagged_fields: Default::default(),
                }],
                throttle_time_ms: None,
                tagged_fields: None,
            }
            .write_versioned(&mut msg, ApiVersion(Int16(v)))
            .unwrap();
            sim.push(msg);
        }

        // sync versions
        let err = messenger.sync_versions().await.unwrap_err();
        assert_matches!(err, SyncVersionsError::NoWorkingVersion);
    }

    #[tokio::test]
    async fn test_poison_hangup() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));
        messenger.set_version_ranges(HashMap::from([(
            ApiKey::ListOffsets,
            ListOffsetsRequest::API_VERSION_RANGE,
        )]));

        sim.hang_up();

        let err = messenger
            .request(ListOffsetsRequest {
                replica_id: NORMAL_CONSUMER,
                isolation_level: None,
                topics: vec![],
            })
            .await
            .unwrap_err();
        assert_matches!(err, RequestError::Poisoned(_));
    }

    #[tokio::test]
    async fn test_poison_negative_message_size() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));
        messenger.set_version_ranges(HashMap::from([(
            ApiKey::ListOffsets,
            ListOffsetsRequest::API_VERSION_RANGE,
        )]));

        sim.negative_message_size();

        let err = messenger
            .request(ListOffsetsRequest {
                replica_id: NORMAL_CONSUMER,
                isolation_level: None,
                topics: vec![],
            })
            .await
            .unwrap_err();
        assert_matches!(err, RequestError::Poisoned(_));

        // follow-up message is broken as well
        let err = messenger
            .request(ListOffsetsRequest {
                replica_id: NORMAL_CONSUMER,
                isolation_level: None,
                topics: vec![],
            })
            .await
            .unwrap_err();
        assert_matches!(err, RequestError::Poisoned(_));
    }

    #[tokio::test]
    async fn test_broken_msg_header_does_not_poison() {
        let (sim, rx) = MessageSimulator::new();
        let mut messenger = Messenger::new(rx, 1_000, Arc::from(DEFAULT_CLIENT_ID));
        messenger.set_version_ranges(HashMap::from([(
            ApiKey::ApiVersions,
            ApiVersionsRequest::API_VERSION_RANGE,
        )]));

        // garbage
        sim.send(b"foo".to_vec());

        // construct good response
        let mut msg = vec![];
        ResponseHeader {
            correlation_id: Int32(0),
            tagged_fields: Default::default(), // NOT serialized for ApiVersion!
        }
        .write_versioned(&mut msg, ApiVersion(Int16(0)))
        .unwrap();
        let resp = ApiVersionsResponse {
            error_code: Some(ApiError::CorruptMessage),
            api_keys: vec![],
            throttle_time_ms: Some(Int32(1)),
            tagged_fields: Some(TaggedFields::default()),
        };
        resp.write_versioned(&mut msg, ApiVersionsRequest::API_VERSION_RANGE.max())
            .unwrap();
        sim.push(msg);

        let actual = messenger
            .request(ApiVersionsRequest {
                client_software_name: Some(CompactString(String::new())),
                client_software_version: Some(CompactString(String::new())),
                tagged_fields: Some(TaggedFields::default()),
            })
            .await
            .unwrap();
        assert_eq!(actual, resp);
    }

    #[tokio::test]
    async fn test_cancel_request() {
        // Use a "virtual" network between a simulated broker and a client. The network is intercepted in the middle to
        // pause it after 3 bytes are sent by the client.
        let (tx_front, rx_middle) = tokio::io::duplex(1);
        let (tx_middle, mut rx_back) = tokio::io::duplex(1);

        let mut messenger = Messenger::new(tx_front, 1_000, Arc::from(DEFAULT_CLIENT_ID));

        // create two barriers:
        // - pause: will be passed after 3 bytes were sent by the client
        // - continue: will be passed to continue client->broker traffic
        //
        // The barriers do NOT affect broker->client traffic.
        //
        // The sizes of the barriers is 2: one for the network simulation task and one for the main/control thread.
        let network_pause = Arc::new(Barrier::new(2));
        let network_pause_captured = Arc::clone(&network_pause);
        let network_continue = Arc::new(Barrier::new(2));
        let network_continue_captured = Arc::clone(&network_continue);
        let handle_network = tokio::spawn(async move {
            // Need to split both directions into read and write halfs so we can run full-duplex in two loops. Otherwise
            // the test might deadlock even though the code is just fine (TCP is full-duplex).
            let (mut rx_middle_read, mut rx_middle_write) = tokio::io::split(rx_middle);
            let (mut tx_middle_read, mut tx_middle_write) = tokio::io::split(tx_middle);

            let direction_client_broker = async {
                for i in 0.. {
                    let mut buf = [0; 1];
                    rx_middle_read.read_exact(&mut buf).await.unwrap();
                    tx_middle_write.write_all(&buf).await.unwrap();

                    if i == 3 {
                        network_pause_captured.wait().await;
                        network_continue_captured.wait().await;
                    }
                }
            };

            let direction_broker_client = async {
                loop {
                    let mut buf = [0; 1];
                    tx_middle_read.read_exact(&mut buf).await.unwrap();
                    rx_middle_write.write_all(&buf).await.unwrap();
                }
            };

            tokio::select! {
                _ = direction_client_broker => {}
                _ = direction_broker_client => {}
            }
        });

        // simulated broker, just reads messages and answers w/ "api versions" responses
        let handle_broker = tokio::spawn(async move {
            for correlation_id in 0.. {
                let data = rx_back.read_message(1_000).await.unwrap();
                let mut data = Cursor::new(data);
                let header =
                    RequestHeader::read_versioned(&mut data, ApiVersion(Int16(1))).unwrap();
                assert_eq!(
                    header,
                    RequestHeader {
                        request_api_key: ApiKey::ApiVersions,
                        request_api_version: ApiVersion(Int16(0)),
                        correlation_id: Int32(correlation_id),
                        client_id: Some(NullableString(Some(String::from(env!("CARGO_PKG_NAME"))))),
                        tagged_fields: None,
                    }
                );
                let body =
                    ApiVersionsRequest::read_versioned(&mut data, ApiVersion(Int16(0))).unwrap();
                assert_eq!(
                    body,
                    ApiVersionsRequest {
                        client_software_name: None,
                        client_software_version: None,
                        tagged_fields: None,
                    }
                );
                assert_eq!(data.position() as usize, data.get_ref().len());

                let mut msg = vec![];
                ResponseHeader {
                    correlation_id: Int32(correlation_id),
                    tagged_fields: Default::default(), // NOT serialized for ApiVersion!
                }
                .write_versioned(&mut msg, ApiVersion(Int16(0)))
                .unwrap();
                let resp = ApiVersionsResponse {
                    error_code: Some(ApiError::CorruptMessage),
                    api_keys: vec![],
                    throttle_time_ms: Some(Int32(1)),
                    tagged_fields: Some(TaggedFields::default()),
                };
                resp.write_versioned(&mut msg, ApiVersionsRequest::API_VERSION_RANGE.min())
                    .unwrap();
                rx_back.write_message(&msg).await.unwrap();
            }
        });

        messenger.set_version_ranges(HashMap::from([(
            ApiKey::ApiVersions,
            ApiVersionRange::new(ApiVersion(Int16(0)), ApiVersion(Int16(0))),
        )]));

        // send first message, this task will be canceled after 3 bytes got sent.
        let task_to_cancel = (async {
            messenger
                .request(ApiVersionsRequest {
                    client_software_name: Some(CompactString(String::from("foo"))),
                    client_software_version: Some(CompactString(String::from("bar"))),
                    tagged_fields: Some(TaggedFields::default()),
                })
                .await
                .unwrap();
        })
        .fuse();

        {
            // cancel when we exit this block
            pin_mut!(task_to_cancel);

            // write exactly 3 bytes via the client and then cancel the task.
            futures::select_biased! {
                _ = &mut task_to_cancel => panic!("should not have finished"),
                _ = network_pause.wait().fuse() => {},
            }
        }

        // continue client->broker traffic
        network_continue.wait().await;

        // IF the original bug in https://github.com/influxdata/rskafka/issues/103 exists, then the following statement
        // will timeout because the broker got garbage and will wait forever to read the message.
        tokio::time::timeout(Duration::from_millis(100), async {
            messenger
                .request(ApiVersionsRequest {
                    client_software_name: Some(CompactString(String::from("foo"))),
                    client_software_version: Some(CompactString(String::from("bar"))),
                    tagged_fields: Some(TaggedFields::default()),
                })
                .await
                .unwrap();
        })
        .await
        .unwrap();

        // clean up helper tasks
        handle_broker.abort();
        handle_network.abort();
    }

    #[derive(Debug)]
    enum Message {
        Send(Vec<u8>),
        Consume,
        NegativeMessageSize,
        HangUp,
    }

    struct MessageSimulator {
        messages: UnboundedSender<Message>,
        join_handle: JoinHandle<()>,
    }

    impl MessageSimulator {
        fn new() -> (Self, DuplexStream) {
            let (mut tx, rx) = tokio::io::duplex(1_000);
            let (msg_tx, mut msg_rx) = tokio::sync::mpsc::unbounded_channel();

            let join_handle = tokio::task::spawn(async move {
                loop {
                    let message = match msg_rx.recv().await {
                        Some(msg) => msg,
                        None => return,
                    };

                    match message {
                        Message::Consume => {
                            tx.read_message(1_000).await.unwrap();
                        }
                        Message::Send(data) => {
                            tx.write_message(&data).await.unwrap();
                        }
                        Message::NegativeMessageSize => {
                            let mut buf = vec![];
                            Int32(-1).write(&mut buf).unwrap();
                            tx.write_all(&buf).await.unwrap()
                        }
                        Message::HangUp => {
                            return;
                        }
                    }
                }
            });

            let this = Self {
                messages: msg_tx,
                join_handle,
            };
            (this, rx)
        }

        fn push(&self, msg: Vec<u8>) {
            // Must wait for the request message before reading the response, otherwise `Messenger` might read
            // our response that doesn't have a correlated request yet and throws it away. This is because
            // servers never send data without being asked to do so.
            self.consume();
            self.send(msg);
        }

        fn consume(&self) {
            self.messages.send(Message::Consume).unwrap();
        }

        fn send(&self, msg: Vec<u8>) {
            self.messages.send(Message::Send(msg)).unwrap();
        }

        fn negative_message_size(&self) {
            self.messages.send(Message::NegativeMessageSize).unwrap();
        }

        fn hang_up(&self) {
            self.messages.send(Message::HangUp).unwrap();
        }
    }

    impl Drop for MessageSimulator {
        fn drop(&mut self) {
            // this will drop the future and therefore tx which will close th streams
            self.join_handle.abort();
        }
    }
}