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
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
use std::{
    collections::{HashMap, HashSet},
    io::Cursor,
    pin::Pin,
    sync::{
        mpsc::{sync_channel, SyncSender},
        Arc, OnceLock, Weak,
    },
    time::Duration,
};

use async_stream::stream;
use bytes::Bytes;
use chrono::{DateTime, TimeZone, Utc};
use futures::{Stream, StreamExt};
use futures_util::future::OptionFuture;
use rdkafka::{
    consumer::{
        stream_consumer::StreamPartitionQueue, CommitMode, Consumer, ConsumerContext, Rebalance,
        StreamConsumer,
    },
    error::KafkaError,
    message::{BorrowedMessage, Headers as _, Message},
    types::RDKafkaErrorCode,
    ClientConfig, ClientContext, Statistics, TopicPartitionList,
};
use serde_with::serde_as;
use snafu::{ResultExt, Snafu};
use tokio::{
    runtime::Handle,
    sync::{
        mpsc::{self, UnboundedReceiver, UnboundedSender},
        oneshot,
    },
    task::JoinSet,
    time::Sleep,
};
use tokio_util::codec::FramedRead;
use tracing::{Instrument, Span};
use vector_lib::codecs::{
    decoding::{DeserializerConfig, FramingConfig},
    StreamDecodingError,
};
use vector_lib::lookup::{lookup_v2::OptionalValuePath, owned_value_path, path, OwnedValuePath};

use vector_lib::configurable::configurable_component;
use vector_lib::finalizer::OrderedFinalizer;
use vector_lib::{
    config::{LegacyKey, LogNamespace},
    EstimatedJsonEncodedSizeOf,
};
use vrl::value::{kind::Collection, Kind, ObjectMap};

use crate::{
    codecs::{Decoder, DecodingConfig},
    config::{
        log_schema, LogSchema, SourceAcknowledgementsConfig, SourceConfig, SourceContext,
        SourceOutput,
    },
    event::{BatchNotifier, BatchStatus, Event, Value},
    internal_events::{
        KafkaBytesReceived, KafkaEventsReceived, KafkaOffsetUpdateError, KafkaReadError,
        StreamClosedError,
    },
    kafka,
    serde::{bool_or_struct, default_decoding, default_framing_message_based},
    shutdown::ShutdownSignal,
    SourceSender,
};

#[derive(Debug, Snafu)]
enum BuildError {
    #[snafu(display("The drain_timeout_ms ({}) must be less than session_timeout_ms ({})", value, session_timeout_ms.as_millis()))]
    InvalidDrainTimeout {
        value: u64,
        session_timeout_ms: Duration,
    },
    #[snafu(display("Could not create Kafka consumer: {}", source))]
    CreateError { source: rdkafka::error::KafkaError },
    #[snafu(display("Could not subscribe to Kafka topics: {}", source))]
    SubscribeError { source: rdkafka::error::KafkaError },
}

/// Metrics (beta) configuration.
#[configurable_component]
#[derive(Clone, Debug, Default)]
struct Metrics {
    /// Expose topic lag metrics for all topics and partitions. Metric names are `kafka_consumer_lag`.
    pub topic_lag_metric: bool,
}

/// Configuration for the `kafka` source.
#[serde_as]
#[configurable_component(source("kafka", "Collect logs from Apache Kafka."))]
#[derive(Clone, Debug, Derivative)]
#[derivative(Default)]
#[serde(deny_unknown_fields)]
pub struct KafkaSourceConfig {
    /// A comma-separated list of Kafka bootstrap servers.
    ///
    /// These are the servers in a Kafka cluster that a client should use to bootstrap its connection to the cluster,
    /// allowing discovery of all the other hosts in the cluster.
    ///
    /// Must be in the form of `host:port`, and comma-separated.
    #[configurable(metadata(docs::examples = "10.14.22.123:9092,10.14.23.332:9092"))]
    bootstrap_servers: String,

    /// The Kafka topics names to read events from.
    ///
    /// Regular expression syntax is supported if the topic begins with `^`.
    #[configurable(metadata(
        docs::examples = "^(prefix1|prefix2)-.+",
        docs::examples = "topic-1",
        docs::examples = "topic-2"
    ))]
    topics: Vec<String>,

    /// The consumer group name to be used to consume events from Kafka.
    #[configurable(metadata(docs::examples = "consumer-group-name"))]
    group_id: String,

    /// If offsets for consumer group do not exist, set them using this strategy.
    ///
    /// See the [librdkafka documentation](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) for the `auto.offset.reset` option for further clarification.
    #[serde(default = "default_auto_offset_reset")]
    #[configurable(metadata(docs::examples = "example_auto_offset_reset_values()"))]
    auto_offset_reset: String,

    /// The Kafka session timeout.
    #[serde_as(as = "serde_with::DurationMilliSeconds<u64>")]
    #[configurable(metadata(docs::examples = 5000, docs::examples = 10000))]
    #[configurable(metadata(docs::advanced))]
    #[serde(default = "default_session_timeout_ms")]
    #[configurable(metadata(docs::human_name = "Session Timeout"))]
    session_timeout_ms: Duration,

    /// Timeout to drain pending acknowledgements during shutdown or a Kafka
    /// consumer group rebalance.
    ///
    /// When Vector shuts down or the Kafka consumer group revokes partitions from this
    /// consumer, wait a maximum of `drain_timeout_ms` for the source to
    /// process pending acknowledgements. Must be less than `session_timeout_ms`
    /// to ensure the consumer is not excluded from the group during a rebalance.
    ///
    /// Default value is half of `session_timeout_ms`.
    #[serde(skip_serializing_if = "Option::is_none")]
    #[configurable(metadata(docs::examples = 2500, docs::examples = 5000))]
    #[configurable(metadata(docs::advanced))]
    #[configurable(metadata(docs::human_name = "Drain Timeout"))]
    drain_timeout_ms: Option<u64>,

    /// Timeout for network requests.
    #[serde_as(as = "serde_with::DurationMilliSeconds<u64>")]
    #[configurable(metadata(docs::examples = 30000, docs::examples = 60000))]
    #[configurable(metadata(docs::advanced))]
    #[serde(default = "default_socket_timeout_ms")]
    #[configurable(metadata(docs::human_name = "Socket Timeout"))]
    socket_timeout_ms: Duration,

    /// Maximum time the broker may wait to fill the response.
    #[serde_as(as = "serde_with::DurationMilliSeconds<u64>")]
    #[configurable(metadata(docs::examples = 50, docs::examples = 100))]
    #[configurable(metadata(docs::advanced))]
    #[serde(default = "default_fetch_wait_max_ms")]
    #[configurable(metadata(docs::human_name = "Max Fetch Wait Time"))]
    fetch_wait_max_ms: Duration,

    /// The frequency that the consumer offsets are committed (written) to offset storage.
    #[serde_as(as = "serde_with::DurationMilliSeconds<u64>")]
    #[serde(default = "default_commit_interval_ms")]
    #[configurable(metadata(docs::examples = 5000, docs::examples = 10000))]
    #[configurable(metadata(docs::human_name = "Commit Interval"))]
    commit_interval_ms: Duration,

    /// Overrides the name of the log field used to add the message key to each event.
    ///
    /// The value is the message key of the Kafka message itself.
    ///
    /// By default, `"message_key"` is used.
    #[serde(default = "default_key_field")]
    #[configurable(metadata(docs::examples = "message_key"))]
    key_field: OptionalValuePath,

    /// Overrides the name of the log field used to add the topic to each event.
    ///
    /// The value is the topic from which the Kafka message was consumed from.
    ///
    /// By default, `"topic"` is used.
    #[serde(default = "default_topic_key")]
    #[configurable(metadata(docs::examples = "topic"))]
    topic_key: OptionalValuePath,

    /// Overrides the name of the log field used to add the partition to each event.
    ///
    /// The value is the partition from which the Kafka message was consumed from.
    ///
    /// By default, `"partition"` is used.
    #[serde(default = "default_partition_key")]
    #[configurable(metadata(docs::examples = "partition"))]
    partition_key: OptionalValuePath,

    /// Overrides the name of the log field used to add the offset to each event.
    ///
    /// The value is the offset of the Kafka message itself.
    ///
    /// By default, `"offset"` is used.
    #[serde(default = "default_offset_key")]
    #[configurable(metadata(docs::examples = "offset"))]
    offset_key: OptionalValuePath,

    /// Overrides the name of the log field used to add the headers to each event.
    ///
    /// The value is the headers of the Kafka message itself.
    ///
    /// By default, `"headers"` is used.
    #[serde(default = "default_headers_key")]
    #[configurable(metadata(docs::examples = "headers"))]
    headers_key: OptionalValuePath,

    /// Advanced options set directly on the underlying `librdkafka` client.
    ///
    /// See the [librdkafka documentation](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) for details.
    #[configurable(metadata(docs::examples = "example_librdkafka_options()"))]
    #[configurable(metadata(docs::advanced))]
    #[configurable(metadata(
        docs::additional_props_description = "A librdkafka configuration option."
    ))]
    librdkafka_options: Option<HashMap<String, String>>,

    #[serde(flatten)]
    auth: kafka::KafkaAuthConfig,

    #[configurable(derived)]
    #[configurable(metadata(docs::advanced))]
    #[serde(default = "default_framing_message_based")]
    #[derivative(Default(value = "default_framing_message_based()"))]
    framing: FramingConfig,

    #[configurable(derived)]
    #[serde(default = "default_decoding")]
    #[derivative(Default(value = "default_decoding()"))]
    decoding: DeserializerConfig,

    #[configurable(derived)]
    #[serde(default, deserialize_with = "bool_or_struct")]
    acknowledgements: SourceAcknowledgementsConfig,

    /// The namespace to use for logs. This overrides the global setting.
    #[configurable(metadata(docs::hidden))]
    #[serde(default)]
    log_namespace: Option<bool>,

    #[configurable(derived)]
    #[serde(default)]
    metrics: Metrics,
}

impl KafkaSourceConfig {
    fn keys(&self) -> Keys {
        Keys::from(log_schema(), self)
    }
}

const fn default_session_timeout_ms() -> Duration {
    Duration::from_millis(10000) // default in librdkafka
}

const fn default_socket_timeout_ms() -> Duration {
    Duration::from_millis(60000) // default in librdkafka
}

const fn default_fetch_wait_max_ms() -> Duration {
    Duration::from_millis(100) // default in librdkafka
}

const fn default_commit_interval_ms() -> Duration {
    Duration::from_millis(5000)
}

fn default_auto_offset_reset() -> String {
    "largest".into() // default in librdkafka
}

fn default_key_field() -> OptionalValuePath {
    OptionalValuePath::from(owned_value_path!("message_key"))
}

fn default_topic_key() -> OptionalValuePath {
    OptionalValuePath::from(owned_value_path!("topic"))
}

fn default_partition_key() -> OptionalValuePath {
    OptionalValuePath::from(owned_value_path!("partition"))
}

fn default_offset_key() -> OptionalValuePath {
    OptionalValuePath::from(owned_value_path!("offset"))
}

fn default_headers_key() -> OptionalValuePath {
    OptionalValuePath::from(owned_value_path!("headers"))
}

const fn example_auto_offset_reset_values() -> [&'static str; 7] {
    [
        "smallest",
        "earliest",
        "beginning",
        "largest",
        "latest",
        "end",
        "error",
    ]
}

fn example_librdkafka_options() -> HashMap<String, String> {
    HashMap::<_, _>::from_iter([
        ("client.id".to_string(), "${ENV_VAR}".to_string()),
        ("fetch.error.backoff.ms".to_string(), "1000".to_string()),
        ("socket.send.buffer.bytes".to_string(), "100".to_string()),
    ])
}

impl_generate_config_from_default!(KafkaSourceConfig);

#[async_trait::async_trait]
#[typetag::serde(name = "kafka")]
impl SourceConfig for KafkaSourceConfig {
    async fn build(&self, cx: SourceContext) -> crate::Result<super::Source> {
        let log_namespace = cx.log_namespace(self.log_namespace);

        let decoder =
            DecodingConfig::new(self.framing.clone(), self.decoding.clone(), log_namespace)
                .build()?;
        let acknowledgements = cx.do_acknowledgements(self.acknowledgements);

        if let Some(d) = self.drain_timeout_ms {
            snafu::ensure!(
                Duration::from_millis(d) <= self.session_timeout_ms,
                InvalidDrainTimeoutSnafu {
                    value: d,
                    session_timeout_ms: self.session_timeout_ms
                }
            );
        }

        let (consumer, callback_rx) = create_consumer(self, acknowledgements)?;

        Ok(Box::pin(kafka_source(
            self.clone(),
            consumer,
            callback_rx,
            decoder,
            cx.out,
            cx.shutdown,
            false,
            log_namespace,
        )))
    }

    fn outputs(&self, global_log_namespace: LogNamespace) -> Vec<SourceOutput> {
        let log_namespace = global_log_namespace.merge(self.log_namespace);
        let keys = self.keys();

        let schema_definition = self
            .decoding
            .schema_definition(log_namespace)
            .with_standard_vector_source_metadata()
            .with_source_metadata(
                Self::NAME,
                keys.timestamp.map(LegacyKey::Overwrite),
                &owned_value_path!("timestamp"),
                Kind::timestamp(),
                Some("timestamp"),
            )
            .with_source_metadata(
                Self::NAME,
                keys.topic.clone().map(LegacyKey::Overwrite),
                &owned_value_path!("topic"),
                Kind::bytes(),
                None,
            )
            .with_source_metadata(
                Self::NAME,
                keys.partition.clone().map(LegacyKey::Overwrite),
                &owned_value_path!("partition"),
                Kind::bytes(),
                None,
            )
            .with_source_metadata(
                Self::NAME,
                keys.offset.clone().map(LegacyKey::Overwrite),
                &owned_value_path!("offset"),
                Kind::bytes(),
                None,
            )
            .with_source_metadata(
                Self::NAME,
                keys.headers.clone().map(LegacyKey::Overwrite),
                &owned_value_path!("headers"),
                Kind::object(Collection::empty().with_unknown(Kind::bytes())),
                None,
            )
            .with_source_metadata(
                Self::NAME,
                keys.key_field.clone().map(LegacyKey::Overwrite),
                &owned_value_path!("message_key"),
                Kind::bytes(),
                None,
            );

        vec![SourceOutput::new_maybe_logs(
            self.decoding.output_type(),
            schema_definition,
        )]
    }

    fn can_acknowledge(&self) -> bool {
        true
    }
}

#[allow(clippy::too_many_arguments)]
async fn kafka_source(
    config: KafkaSourceConfig,
    consumer: StreamConsumer<KafkaSourceContext>,
    callback_rx: UnboundedReceiver<KafkaCallback>,
    decoder: Decoder,
    out: SourceSender,
    shutdown: ShutdownSignal,
    eof: bool,
    log_namespace: LogNamespace,
) -> Result<(), ()> {
    let span = info_span!("kafka_source");
    let consumer = Arc::new(consumer);

    consumer
        .context()
        .consumer
        .set(Arc::downgrade(&consumer))
        .expect("Error setting up consumer context.");

    // EOF signal allowing the coordination task to tell the kafka client task when all partitions have reached EOF
    let (eof_tx, eof_rx) = eof.then(oneshot::channel::<()>).unzip();

    let topics: Vec<&str> = config.topics.iter().map(|s| s.as_str()).collect();
    if let Err(e) = consumer.subscribe(&topics).context(SubscribeSnafu) {
        error!("{}", e);
        return Err(());
    }

    let coordination_task = {
        let span = span.clone();
        let consumer = Arc::clone(&consumer);
        let drain_timeout_ms = config
            .drain_timeout_ms
            .map_or(config.session_timeout_ms / 2, Duration::from_millis);
        let consumer_state =
            ConsumerStateInner::<Consuming>::new(config, decoder, out, log_namespace, span);
        tokio::spawn(async move {
            coordinate_kafka_callbacks(
                consumer,
                callback_rx,
                consumer_state,
                drain_timeout_ms,
                eof_tx,
            )
            .await;
        })
    };

    let client_task = {
        let consumer = Arc::clone(&consumer);
        tokio::task::spawn_blocking(move || {
            let _enter = span.enter();
            drive_kafka_consumer(consumer, shutdown, eof_rx);
        })
    };

    _ = tokio::join!(client_task, coordination_task);
    consumer.context().commit_consumer_state();

    Ok(())
}

/// ConsumerStateInner implements a small struct/enum-based state machine.
///
/// With a ConsumerStateInner<Consuming>, the client is able to spawn new tasks
/// when partitions are assigned. When a shutdown signal is received, or
/// partitions are being revoked, the Consuming state is traded for a Draining
/// state (and associated drain deadline future) via the `begin_drain` method
///
/// A ConsumerStateInner<Draining> keeps track of partitions that are expected
/// to complete, and also owns the signal that, when dropped, indicates to the
/// client driver task that it is safe to proceed with the rebalance or shutdown.
/// When draining is complete, or the deadline is reached, Draining is traded in for
/// either a Consuming (after a revoke) or Complete (in the case of shutdown) state,
/// via the `finish_drain` method.
///
/// A ConsumerStateInner<Complete> is the final state, reached after a shutdown
/// signal is received. This can not be traded for another state, and the
/// coordination task should exit when this state is reached.
struct ConsumerStateInner<S> {
    config: KafkaSourceConfig,
    decoder: Decoder,
    out: SourceSender,
    log_namespace: LogNamespace,
    consumer_state: S,
}
struct Consuming {
    /// The source's tracing Span used to instrument metrics emitted by consumer tasks
    span: Span,
}
struct Draining {
    /// The rendezvous channel sender from the revoke or shutdown callback. Sending on this channel
    /// indicates to the kafka client task that one or more partitions have been drained, while
    /// closing this channel indicates that all expected partitions have drained, or the drain
    /// timeout has been reached.
    signal: SyncSender<()>,

    /// The set of topic-partition tasks that are required to complete during
    /// the draining phase, populated at the beginning of a rebalance or shutdown.
    /// Partitions that are being revoked, but not being actively consumed
    /// (e.g. due to the consumer task exiting early) should not be included.
    /// The draining phase is considered complete when this set is empty.
    expect_drain: HashSet<TopicPartition>,

    /// Whether the client is shutting down after draining. If set to true,
    /// the `finish_drain` method will return a Complete state, otherwise
    /// a Consuming state.
    shutdown: bool,

    /// The source's tracing Span used to instrument metrics emitted by consumer tasks
    span: Span,
}
type OptionDeadline = OptionFuture<Pin<Box<Sleep>>>;
enum ConsumerState {
    Consuming(ConsumerStateInner<Consuming>),
    Draining(ConsumerStateInner<Draining>),
    Complete,
}
impl Draining {
    fn new(signal: SyncSender<()>, shutdown: bool, span: Span) -> Self {
        Self {
            signal,
            shutdown,
            expect_drain: HashSet::new(),
            span,
        }
    }

    fn is_complete(&self) -> bool {
        self.expect_drain.is_empty()
    }
}

impl<C> ConsumerStateInner<C> {
    fn complete(self, _deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) {
        (None.into(), ConsumerState::Complete)
    }
}

impl ConsumerStateInner<Consuming> {
    const fn new(
        config: KafkaSourceConfig,
        decoder: Decoder,
        out: SourceSender,
        log_namespace: LogNamespace,
        span: Span,
    ) -> Self {
        Self {
            config,
            decoder,
            out,
            log_namespace,
            consumer_state: Consuming { span },
        }
    }

    /// Spawn a task on the provided JoinSet to consume the kafka StreamPartitionQueue, and handle
    /// acknowledgements for the messages consumed Returns a channel sender that can be used to
    /// signal that the consumer should stop and drain pending acknowledgements, and an AbortHandle
    /// that can be used to forcefully end the task.
    fn consume_partition(
        &self,
        join_set: &mut JoinSet<(TopicPartition, PartitionConsumerStatus)>,
        tp: TopicPartition,
        consumer: Arc<StreamConsumer<KafkaSourceContext>>,
        p: StreamPartitionQueue<KafkaSourceContext>,
        acknowledgements: bool,
        exit_eof: bool,
    ) -> (oneshot::Sender<()>, tokio::task::AbortHandle) {
        let keys = self.config.keys();
        let decoder = self.decoder.clone();
        let log_namespace = self.log_namespace;
        let mut out = self.out.clone();

        let (end_tx, mut end_signal) = oneshot::channel::<()>();

        let handle = join_set.spawn(async move {
            let mut messages = p.stream();
            let (finalizer, mut ack_stream) = OrderedFinalizer::<FinalizerEntry>::new(None);

            // finalizer is the entry point for new pending acknowledgements;
            // when it is dropped, no new messages will be consumed, and the
            // task will end when it reaches the end of ack_stream
            let mut finalizer = Some(finalizer);

            let mut status = PartitionConsumerStatus::NormalExit;

            loop {
                tokio::select!(
                    // Make sure to handle the acknowledgement stream before new messages to prevent
                    // unbounded memory growth caused by those acks being handled slower than
                    // incoming messages when the load is high.
                    biased;

                    // is_some() checks prevent polling end_signal after it completes
                    _ = &mut end_signal, if finalizer.is_some() => {
                        finalizer.take();
                    },

                    ack = ack_stream.next() => match ack {
                        Some((status, entry)) => {
                            if status == BatchStatus::Delivered {
                                if let Err(error) =  consumer.store_offset(&entry.topic, entry.partition, entry.offset) {
                                    emit!(KafkaOffsetUpdateError { error });
                                }
                            }
                        }
                        None if finalizer.is_none() => {
                            debug!("Acknowledgement stream complete for partition {}:{}.", &tp.0, tp.1);
                            break
                        }
                        None => {
                            debug!("Acknowledgement stream empty for {}:{}", &tp.0, tp.1);
                        }
                    },

                    message = messages.next(), if finalizer.is_some() => match message {
                        None => unreachable!("MessageStream never calls Ready(None)"),
                        Some(Err(error)) => match error {
                            rdkafka::error::KafkaError::PartitionEOF(partition) if exit_eof => {
                                debug!("EOF for partition {}.", partition);
                                status = PartitionConsumerStatus::PartitionEOF;
                                finalizer.take();
                            },
                            _ => emit!(KafkaReadError { error }),
                        },
                        Some(Ok(msg)) => {
                            emit!(KafkaBytesReceived {
                                byte_size: msg.payload_len(),
                                protocol: "tcp",
                                topic: msg.topic(),
                                partition: msg.partition(),
                            });
                            parse_message(msg, decoder.clone(), &keys, &mut out, acknowledgements, &finalizer, log_namespace).await;
                        }
                    },
                )
            }
            (tp, status)
        }.instrument(self.consumer_state.span.clone()));
        (end_tx, handle)
    }

    /// Consume self, and return a "Draining" ConsumerState, along with a Future
    /// representing a drain deadline, based on max_drain_ms
    fn begin_drain(
        self,
        max_drain_ms: Duration,
        sig: SyncSender<()>,
        shutdown: bool,
    ) -> (OptionDeadline, ConsumerStateInner<Draining>) {
        let deadline = Box::pin(tokio::time::sleep(max_drain_ms));

        let draining = ConsumerStateInner {
            config: self.config,
            decoder: self.decoder,
            out: self.out,
            log_namespace: self.log_namespace,
            consumer_state: Draining::new(sig, shutdown, self.consumer_state.span),
        };

        (Some(deadline).into(), draining)
    }

    pub const fn keep_consuming(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) {
        (deadline, ConsumerState::Consuming(self))
    }
}

impl ConsumerStateInner<Draining> {
    /// Mark the given TopicPartition as being revoked, adding it to the set of
    /// partitions expected to drain
    fn revoke_partition(&mut self, tp: TopicPartition, end_signal: oneshot::Sender<()>) {
        // Note that if this send() returns Err, it means the task has already
        // ended, but the completion has not been processed yet (otherwise we wouldn't have access to the end_signal),
        // so we should still add it to the "expect to drain" set
        _ = end_signal.send(());
        self.consumer_state.expect_drain.insert(tp);
    }

    /// Add the given TopicPartition to the set of known "drained" partitions,
    /// i.e. the consumer has drained the acknowledgement channel. A signal is
    /// sent on the signal channel, indicating to the client that offsets may be committed
    fn partition_drained(&mut self, tp: TopicPartition) {
        // This send() will only return Err if the receiver has already been disconnected (i.e. the
        // kafka client task is no longer running)
        _ = self.consumer_state.signal.send(());
        self.consumer_state.expect_drain.remove(&tp);
    }

    /// Return true if all expected partitions have drained
    fn is_drain_complete(&self) -> bool {
        self.consumer_state.is_complete()
    }

    /// Finish partition drain mode. Consumes self and the drain deadline
    /// future, and returns a "Consuming" or "Complete" ConsumerState
    fn finish_drain(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) {
        if self.consumer_state.shutdown {
            self.complete(deadline)
        } else {
            (
                None.into(),
                ConsumerState::Consuming(ConsumerStateInner {
                    config: self.config,
                    decoder: self.decoder,
                    out: self.out,
                    log_namespace: self.log_namespace,
                    consumer_state: Consuming {
                        span: self.consumer_state.span,
                    },
                }),
            )
        }
    }

    pub const fn keep_draining(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) {
        (deadline, ConsumerState::Draining(self))
    }
}

async fn coordinate_kafka_callbacks(
    consumer: Arc<StreamConsumer<KafkaSourceContext>>,
    mut callbacks: UnboundedReceiver<KafkaCallback>,
    consumer_state: ConsumerStateInner<Consuming>,
    max_drain_ms: Duration,
    mut eof: Option<oneshot::Sender<()>>,
) {
    let mut drain_deadline: OptionFuture<_> = None.into();
    let mut consumer_state = ConsumerState::Consuming(consumer_state);

    // A oneshot channel is used for each consumed partition, so that we can
    // signal to that task to stop consuming, drain pending acks, and exit
    let mut end_signals: HashMap<TopicPartition, oneshot::Sender<()>> = HashMap::new();

    // The set of consumer tasks, each consuming a specific partition. The task
    // is both consuming the messages (passing them to the output stream) _and_
    // processing the corresponding acknowledgement stream. A consumer task
    // should completely drain its acknowledgement stream after receiving an end signal
    let mut partition_consumers: JoinSet<(TopicPartition, PartitionConsumerStatus)> =
        Default::default();

    // Handles that will let us end any consumer task that exceeds a drain deadline
    let mut abort_handles: HashMap<TopicPartition, tokio::task::AbortHandle> = HashMap::new();

    let exit_eof = eof.is_some();

    while let ConsumerState::Consuming(_) | ConsumerState::Draining(_) = consumer_state {
        tokio::select! {
            Some(Ok((finished_partition, status))) = partition_consumers.join_next(), if !partition_consumers.is_empty() => {
                debug!("Partition consumer finished for {}:{}", &finished_partition.0, finished_partition.1);
                // If this task ended on its own, the end_signal for it will still be in here.
                end_signals.remove(&finished_partition);
                abort_handles.remove(&finished_partition);

                (drain_deadline, consumer_state) = match consumer_state {
                    ConsumerState::Complete => unreachable!("Partition consumer finished after completion."),
                    ConsumerState::Draining(mut state) => {
                        state.partition_drained(finished_partition);

                        if state.is_drain_complete() {
                            debug!("All expected partitions have drained.");
                            state.finish_drain(drain_deadline)
                        } else {
                            state.keep_draining(drain_deadline)
                        }
                    },
                    ConsumerState::Consuming(state) => {
                        // If we are here, it is likely because the consumer
                        // tasks are set up to exit upon reaching the end of the
                        // partition.
                        if !exit_eof {
                            debug!("Partition consumer task finished, while not in draining mode.");
                        }
                        state.keep_consuming(drain_deadline)
                    },
                };

                // PartitionConsumerStatus differentiates between a task that exited after
                // being signaled to end, and one that reached the end of its partition and
                // was configured to exit. After the last such task ends, we signal the kafka
                // driver task to shut down the main consumer too. Note this is only used in tests.
                if exit_eof && status == PartitionConsumerStatus::PartitionEOF && partition_consumers.is_empty() {
                    debug!("All partitions have exited or reached EOF.");
                    let _ = eof.take().map(|e| e.send(()));
                }
            },
            Some(callback) = callbacks.recv() => match callback {
                KafkaCallback::PartitionsAssigned(mut assigned_partitions, done) => match consumer_state {
                    ConsumerState::Complete => unreachable!("Partition assignment received after completion."),
                    ConsumerState::Draining(_) => error!("Partition assignment received while draining revoked partitions, maybe an invalid assignment."),
                    ConsumerState::Consuming(ref consumer_state) => {
                        let acks = consumer.context().acknowledgements;
                        for tp in assigned_partitions.drain(0..) {
                            let topic = tp.0.as_str();
                            let partition = tp.1;
                            if let Some(pq) = consumer.split_partition_queue(topic, partition) {
                                debug!("Consuming partition {}:{}.", &tp.0, tp.1);
                                let (end_tx, handle) = consumer_state.consume_partition(&mut partition_consumers, tp.clone(), Arc::clone(&consumer), pq, acks, exit_eof);
                                abort_handles.insert(tp.clone(), handle);
                                end_signals.insert(tp, end_tx);
                            } else {
                                warn!("Failed to get queue for assigned partition {}:{}.", &tp.0, tp.1);
                            }
                        }
                        // ensure this is retained until all individual queues are set up
                        drop(done);
                    }
                },
                KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => (drain_deadline, consumer_state) = match consumer_state {
                    ConsumerState::Complete => unreachable!("Partitions revoked after completion."),
                    ConsumerState::Draining(d) => {
                        // NB: This would only happen if the task driving the kafka client (i.e. rebalance handlers)
                        // is not handling shutdown signals, and a revoke happens during a shutdown drain; otherwise
                        // this is unreachable code.
                        warn!("Kafka client is already draining revoked partitions.");
                        d.keep_draining(drain_deadline)
                    },
                    ConsumerState::Consuming(state) => {
                        let (deadline, mut state) = state.begin_drain(max_drain_ms, drain, false);

                        for tp in revoked_partitions.drain(0..) {
                            if let Some(end) = end_signals.remove(&tp) {
                                debug!("Revoking partition {}:{}", &tp.0, tp.1);
                                state.revoke_partition(tp, end);
                            } else {
                                debug!("Consumer task for partition {}:{} already finished.", &tp.0, tp.1);
                            }
                        }

                        state.keep_draining(deadline)
                    }
                },
                KafkaCallback::ShuttingDown(drain) => (drain_deadline, consumer_state) = match consumer_state {
                    ConsumerState::Complete => unreachable!("Shutdown received after completion."),
                    // Shutting down is just like a full assignment revoke, but we also close the
                    // callback channels, since we don't expect additional assignments or rebalances
                    ConsumerState::Draining(state) => {
                        // NB: This would only happen if the task driving the kafka client is
                        // not handling shutdown signals; otherwise this is unreachable code
                        error!("Kafka client handled a shutdown signal while a rebalance was in progress.");
                        callbacks.close();
                        state.keep_draining(drain_deadline)
                    },
                    ConsumerState::Consuming(state) => {
                        callbacks.close();
                        let (deadline, mut state) = state.begin_drain(max_drain_ms, drain, true);
                        if let Ok(tpl) = consumer.assignment() {
                            // TODO  workaround for https://github.com/fede1024/rust-rdkafka/issues/681
                            if tpl.capacity() == 0 {
                                return;
                            }
                            tpl.elements()
                                .iter()
                                .for_each(|el| {

                                let tp: TopicPartition = (el.topic().into(), el.partition());
                                if let Some(end) = end_signals.remove(&tp) {
                                    debug!("Shutting down and revoking partition {}:{}", &tp.0, tp.1);
                                    state.revoke_partition(tp, end);
                                } else {
                                    debug!("Consumer task for partition {}:{} already finished.", &tp.0, tp.1);
                                }
                            });
                        }
                        // If shutdown was initiated by partition EOF mode, the drain phase
                        // will already be complete and would time out if not accounted for here
                        if state.is_drain_complete() {
                            state.finish_drain(deadline)
                        } else {
                            state.keep_draining(deadline)
                        }
                    }
                },
            },

            Some(_) = &mut drain_deadline => (drain_deadline, consumer_state) = match consumer_state {
                ConsumerState::Complete => unreachable!("Drain deadline received after completion."),
                ConsumerState::Consuming(state) => {
                    warn!("A drain deadline fired outside of draining mode.");
                    state.keep_consuming(None.into())
                },
                ConsumerState::Draining(mut draining) => {
                    debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions.");
                    for tp in draining.consumer_state.expect_drain.drain() {
                        if let Some(handle) = abort_handles.remove(&tp) {
                            handle.abort();
                        }
                    }
                    draining.finish_drain(drain_deadline)
                }
            },
        }
    }
}

fn drive_kafka_consumer(
    consumer: Arc<StreamConsumer<KafkaSourceContext>>,
    mut shutdown: ShutdownSignal,
    eof: Option<oneshot::Receiver<()>>,
) {
    Handle::current().block_on(async move {
        let mut eof: OptionFuture<_> = eof.into();
        let mut stream = consumer.stream();
        loop {
            tokio::select! {
                _ = &mut shutdown => {
                    consumer.context().shutdown();
                    break
                },

                Some(_) = &mut eof => {
                    consumer.context().shutdown();
                    break
                },

                // NB: messages are not received on this thread, however we poll
                // the consumer to serve client callbacks, such as rebalance notifications
                message = stream.next() => match message {
                    None => unreachable!("MessageStream never returns Ready(None)"),
                    Some(Err(error)) => emit!(KafkaReadError { error }),
                    Some(Ok(_msg)) => {
                        unreachable!("Messages are consumed in dedicated tasks for each partition.")
                    }
                },
            }
        }
    });
}

async fn parse_message(
    msg: BorrowedMessage<'_>,
    decoder: Decoder,
    keys: &'_ Keys,
    out: &mut SourceSender,
    acknowledgements: bool,
    finalizer: &Option<OrderedFinalizer<FinalizerEntry>>,
    log_namespace: LogNamespace,
) {
    if let Some((count, stream)) = parse_stream(&msg, decoder, keys, log_namespace) {
        let (batch, receiver) = BatchNotifier::new_with_receiver();
        let mut stream = stream.map(|event| {
            // All acknowledgements flow through the normal Finalizer stream so
            // that they can be handled in one place, but are only tied to the
            // batch when acknowledgements are enabled
            if acknowledgements {
                event.with_batch_notifier(&batch)
            } else {
                event
            }
        });
        match out.send_event_stream(&mut stream).await {
            Err(_) => {
                emit!(StreamClosedError { count });
            }
            Ok(_) => {
                // Drop stream to avoid borrowing `msg`: "[...] borrow might be used
                // here, when `stream` is dropped and runs the destructor [...]".
                drop(stream);
                if let Some(f) = finalizer.as_ref() {
                    f.add(msg.into(), receiver)
                }
            }
        }
    }
}

// Turn the received message into a stream of parsed events.
fn parse_stream<'a>(
    msg: &BorrowedMessage<'a>,
    decoder: Decoder,
    keys: &'a Keys,
    log_namespace: LogNamespace,
) -> Option<(usize, impl Stream<Item = Event> + 'a)> {
    let payload = msg.payload()?; // skip messages with empty payload

    let rmsg = ReceivedMessage::from(msg);

    let payload = Cursor::new(Bytes::copy_from_slice(payload));

    let mut stream = FramedRead::with_capacity(payload, decoder, msg.payload_len());
    let (count, _) = stream.size_hint();
    let stream = stream! {
        while let Some(result) = stream.next().await {
            match result {
                Ok((events, _byte_size)) => {
                    emit!(KafkaEventsReceived {
                        count: events.len(),
                        byte_size: events.estimated_json_encoded_size_of(),
                        topic: &rmsg.topic,
                        partition: rmsg.partition,
                    });
                    for mut event in events {
                        rmsg.apply(keys, &mut event, log_namespace);
                        yield event;
                    }
                },
                Err(error) => {
                    // Error is logged by `codecs::Decoder`, no further handling
                    // is needed here.
                    if !error.can_continue() {
                        break;
                    }
                }
            }
        }
    }
    .boxed();
    Some((count, stream))
}

#[derive(Clone, Debug)]
struct Keys {
    timestamp: Option<OwnedValuePath>,
    key_field: Option<OwnedValuePath>,
    topic: Option<OwnedValuePath>,
    partition: Option<OwnedValuePath>,
    offset: Option<OwnedValuePath>,
    headers: Option<OwnedValuePath>,
}

impl Keys {
    fn from(schema: &LogSchema, config: &KafkaSourceConfig) -> Self {
        Self {
            timestamp: schema.timestamp_key().cloned(),
            key_field: config.key_field.path.clone(),
            topic: config.topic_key.path.clone(),
            partition: config.partition_key.path.clone(),
            offset: config.offset_key.path.clone(),
            headers: config.headers_key.path.clone(),
        }
    }
}

struct ReceivedMessage {
    timestamp: Option<DateTime<Utc>>,
    key: Value,
    headers: ObjectMap,
    topic: String,
    partition: i32,
    offset: i64,
}

impl ReceivedMessage {
    fn from(msg: &BorrowedMessage<'_>) -> Self {
        // Extract timestamp from kafka message
        let timestamp = msg
            .timestamp()
            .to_millis()
            .and_then(|millis| Utc.timestamp_millis_opt(millis).latest());

        let key = msg
            .key()
            .map(|key| Value::from(Bytes::from(key.to_owned())))
            .unwrap_or(Value::Null);

        let mut headers_map = ObjectMap::new();
        if let Some(headers) = msg.headers() {
            for header in headers.iter() {
                if let Some(value) = header.value {
                    headers_map.insert(
                        header.key.into(),
                        Value::from(Bytes::from(value.to_owned())),
                    );
                }
            }
        }

        Self {
            timestamp,
            key,
            headers: headers_map,
            topic: msg.topic().to_string(),
            partition: msg.partition(),
            offset: msg.offset(),
        }
    }

    fn apply(&self, keys: &Keys, event: &mut Event, log_namespace: LogNamespace) {
        if let Event::Log(ref mut log) = event {
            match log_namespace {
                LogNamespace::Vector => {
                    // We'll only use this function in Vector namespaces because we don't want
                    // "timestamp" to be set automatically in legacy namespaces. In legacy
                    // namespaces, the "timestamp" field corresponds to the Kafka message, not the
                    // timestamp when the event was processed.
                    log_namespace.insert_standard_vector_source_metadata(
                        log,
                        KafkaSourceConfig::NAME,
                        Utc::now(),
                    );
                }
                LogNamespace::Legacy => {
                    if let Some(source_type_key) = log_schema().source_type_key_target_path() {
                        log.insert(source_type_key, KafkaSourceConfig::NAME);
                    }
                }
            }

            log_namespace.insert_source_metadata(
                KafkaSourceConfig::NAME,
                log,
                keys.key_field.as_ref().map(LegacyKey::Overwrite),
                path!("message_key"),
                self.key.clone(),
            );

            log_namespace.insert_source_metadata(
                KafkaSourceConfig::NAME,
                log,
                keys.timestamp.as_ref().map(LegacyKey::Overwrite),
                path!("timestamp"),
                self.timestamp,
            );

            log_namespace.insert_source_metadata(
                KafkaSourceConfig::NAME,
                log,
                keys.topic.as_ref().map(LegacyKey::Overwrite),
                path!("topic"),
                self.topic.clone(),
            );

            log_namespace.insert_source_metadata(
                KafkaSourceConfig::NAME,
                log,
                keys.partition.as_ref().map(LegacyKey::Overwrite),
                path!("partition"),
                self.partition,
            );

            log_namespace.insert_source_metadata(
                KafkaSourceConfig::NAME,
                log,
                keys.offset.as_ref().map(LegacyKey::Overwrite),
                path!("offset"),
                self.offset,
            );

            log_namespace.insert_source_metadata(
                KafkaSourceConfig::NAME,
                log,
                keys.headers.as_ref().map(LegacyKey::Overwrite),
                path!("headers"),
                self.headers.clone(),
            );
        }
    }
}

#[derive(Debug, Eq, PartialEq, Hash)]
struct FinalizerEntry {
    topic: String,
    partition: i32,
    offset: i64,
}

impl<'a> From<BorrowedMessage<'a>> for FinalizerEntry {
    fn from(msg: BorrowedMessage<'a>) -> Self {
        Self {
            topic: msg.topic().into(),
            partition: msg.partition(),
            offset: msg.offset(),
        }
    }
}

fn create_consumer(
    config: &KafkaSourceConfig,
    acknowledgements: bool,
) -> crate::Result<(
    StreamConsumer<KafkaSourceContext>,
    UnboundedReceiver<KafkaCallback>,
)> {
    let mut client_config = ClientConfig::new();
    client_config
        .set("group.id", &config.group_id)
        .set("bootstrap.servers", &config.bootstrap_servers)
        .set("auto.offset.reset", &config.auto_offset_reset)
        .set(
            "session.timeout.ms",
            config.session_timeout_ms.as_millis().to_string(),
        )
        .set(
            "socket.timeout.ms",
            config.socket_timeout_ms.as_millis().to_string(),
        )
        .set(
            "fetch.wait.max.ms",
            config.fetch_wait_max_ms.as_millis().to_string(),
        )
        .set("enable.partition.eof", "false")
        .set("enable.auto.commit", "true")
        .set(
            "auto.commit.interval.ms",
            config.commit_interval_ms.as_millis().to_string(),
        )
        .set("enable.auto.offset.store", "false")
        .set("statistics.interval.ms", "1000")
        .set("client.id", "vector");

    config.auth.apply(&mut client_config)?;

    if let Some(librdkafka_options) = &config.librdkafka_options {
        for (key, value) in librdkafka_options {
            client_config.set(key.as_str(), value.as_str());
        }
    }

    let (callbacks, callback_rx) = mpsc::unbounded_channel();
    let consumer = client_config
        .create_with_context::<_, StreamConsumer<_>>(KafkaSourceContext::new(
            config.metrics.topic_lag_metric,
            acknowledgements,
            callbacks,
            Span::current(),
        ))
        .context(CreateSnafu)?;

    Ok((consumer, callback_rx))
}

type TopicPartition = (String, i32);

/// Status returned by partition consumer tasks, allowing the coordination task
/// to differentiate between a consumer exiting normally (after receiving an end
/// signal) and exiting when it reaches the end of a partition
#[derive(PartialEq)]
enum PartitionConsumerStatus {
    NormalExit,
    PartitionEOF,
}

enum KafkaCallback {
    PartitionsAssigned(Vec<TopicPartition>, SyncSender<()>),
    PartitionsRevoked(Vec<TopicPartition>, SyncSender<()>),
    ShuttingDown(SyncSender<()>),
}

struct KafkaSourceContext {
    acknowledgements: bool,
    stats: kafka::KafkaStatisticsContext,

    /// A callback channel used to coordinate between the main consumer task and the acknowledgement task
    callbacks: UnboundedSender<KafkaCallback>,

    /// A weak reference to the consumer, so that we can commit offsets during a rebalance operation
    consumer: OnceLock<Weak<StreamConsumer<KafkaSourceContext>>>,
}

impl KafkaSourceContext {
    fn new(
        expose_lag_metrics: bool,
        acknowledgements: bool,
        callbacks: UnboundedSender<KafkaCallback>,
        span: Span,
    ) -> Self {
        Self {
            stats: kafka::KafkaStatisticsContext {
                expose_lag_metrics,
                span,
            },
            acknowledgements,
            consumer: OnceLock::default(),
            callbacks,
        }
    }

    fn shutdown(&self) {
        let (send, rendezvous) = sync_channel(0);
        if self
            .callbacks
            .send(KafkaCallback::ShuttingDown(send))
            .is_ok()
        {
            while rendezvous.recv().is_ok() {
                self.commit_consumer_state();
            }
        }
    }

    /// Emit a PartitionsAssigned callback with the topic-partitions to be consumed,
    /// and block until confirmation is received that a stream and consumer for
    /// each topic-partition has been set up. This function blocks until the
    /// rendezvous channel sender is dropped by the callback handler.
    fn consume_partitions(&self, tpl: &TopicPartitionList) {
        // TODO  workaround for https://github.com/fede1024/rust-rdkafka/issues/681
        if tpl.capacity() == 0 {
            return;
        }
        let (send, rendezvous) = sync_channel(0);
        let _ = self.callbacks.send(KafkaCallback::PartitionsAssigned(
            tpl.elements()
                .iter()
                .map(|tp| (tp.topic().into(), tp.partition()))
                .collect(),
            send,
        ));

        while rendezvous.recv().is_ok() {
            // no-op: wait for partition assignment handler to complete
        }
    }

    /// Emit a PartitionsRevoked callback and block until confirmation is
    /// received that acknowledgements have been processed for each of them.
    /// The rendezvous channel used in the callback can send multiple times to
    /// signal individual partitions completing. This function blocks until the
    /// sender is dropped by the callback handler.
    fn revoke_partitions(&self, tpl: &TopicPartitionList) {
        let (send, rendezvous) = sync_channel(0);
        let _ = self.callbacks.send(KafkaCallback::PartitionsRevoked(
            tpl.elements()
                .iter()
                .map(|tp| (tp.topic().into(), tp.partition()))
                .collect(),
            send,
        ));

        while rendezvous.recv().is_ok() {
            self.commit_consumer_state();
        }
    }

    fn commit_consumer_state(&self) {
        if let Some(consumer) = self
            .consumer
            .get()
            .expect("Consumer reference was not initialized.")
            .upgrade()
        {
            match consumer.commit_consumer_state(CommitMode::Sync) {
                Ok(_) | Err(KafkaError::ConsumerCommit(RDKafkaErrorCode::NoOffset)) => {
                    /* Success, or nothing to do - yay \0/ */
                }
                Err(error) => emit!(KafkaOffsetUpdateError { error }),
            }
        }
    }
}

impl ClientContext for KafkaSourceContext {
    fn stats(&self, statistics: Statistics) {
        self.stats.stats(statistics)
    }
}

impl ConsumerContext for KafkaSourceContext {
    fn pre_rebalance(&self, rebalance: &Rebalance) {
        match rebalance {
            Rebalance::Assign(tpl) => self.consume_partitions(tpl),

            Rebalance::Revoke(tpl) => {
                // TODO  workaround for https://github.com/fede1024/rust-rdkafka/issues/681
                if tpl.capacity() == 0 {
                    return;
                }
                self.revoke_partitions(tpl);
                self.commit_consumer_state();
            }

            Rebalance::Error(message) => {
                error!("Error during Kafka consumer group rebalance: {}.", message);
            }
        }
    }
}

#[cfg(test)]
mod test {
    use vector_lib::lookup::OwnedTargetPath;
    use vector_lib::schema::Definition;

    use super::*;

    pub fn kafka_host() -> String {
        std::env::var("KAFKA_HOST").unwrap_or_else(|_| "localhost".into())
    }
    pub fn kafka_port() -> u16 {
        let port = std::env::var("KAFKA_PORT").unwrap_or_else(|_| "9091".into());
        port.parse().expect("Invalid port number")
    }

    pub fn kafka_address() -> String {
        format!("{}:{}", kafka_host(), kafka_port())
    }

    #[test]
    fn generate_config() {
        crate::test_util::test_generate_config::<KafkaSourceConfig>();
    }

    pub(super) fn make_config(
        topic: &str,
        group: &str,
        log_namespace: LogNamespace,
        librdkafka_options: Option<HashMap<String, String>>,
    ) -> KafkaSourceConfig {
        KafkaSourceConfig {
            bootstrap_servers: kafka_address(),
            topics: vec![topic.into()],
            group_id: group.into(),
            auto_offset_reset: "beginning".into(),
            session_timeout_ms: Duration::from_millis(6000),
            commit_interval_ms: Duration::from_millis(1),
            librdkafka_options,
            key_field: default_key_field(),
            topic_key: default_topic_key(),
            partition_key: default_partition_key(),
            offset_key: default_offset_key(),
            headers_key: default_headers_key(),
            socket_timeout_ms: Duration::from_millis(60000),
            fetch_wait_max_ms: Duration::from_millis(100),
            log_namespace: Some(log_namespace == LogNamespace::Vector),
            ..Default::default()
        }
    }

    #[test]
    fn test_output_schema_definition_vector_namespace() {
        let definitions = make_config("topic", "group", LogNamespace::Vector, None)
            .outputs(LogNamespace::Vector)
            .remove(0)
            .schema_definition(true);

        assert_eq!(
            definitions,
            Some(
                Definition::new_with_default_metadata(Kind::bytes(), [LogNamespace::Vector])
                    .with_meaning(OwnedTargetPath::event_root(), "message")
                    .with_metadata_field(
                        &owned_value_path!("kafka", "timestamp"),
                        Kind::timestamp(),
                        Some("timestamp")
                    )
                    .with_metadata_field(
                        &owned_value_path!("kafka", "message_key"),
                        Kind::bytes(),
                        None
                    )
                    .with_metadata_field(&owned_value_path!("kafka", "topic"), Kind::bytes(), None)
                    .with_metadata_field(
                        &owned_value_path!("kafka", "partition"),
                        Kind::bytes(),
                        None
                    )
                    .with_metadata_field(&owned_value_path!("kafka", "offset"), Kind::bytes(), None)
                    .with_metadata_field(
                        &owned_value_path!("kafka", "headers"),
                        Kind::object(Collection::empty().with_unknown(Kind::bytes())),
                        None
                    )
                    .with_metadata_field(
                        &owned_value_path!("vector", "ingest_timestamp"),
                        Kind::timestamp(),
                        None
                    )
                    .with_metadata_field(
                        &owned_value_path!("vector", "source_type"),
                        Kind::bytes(),
                        None
                    )
            )
        )
    }

    #[test]
    fn test_output_schema_definition_legacy_namespace() {
        let definitions = make_config("topic", "group", LogNamespace::Legacy, None)
            .outputs(LogNamespace::Legacy)
            .remove(0)
            .schema_definition(true);

        assert_eq!(
            definitions,
            Some(
                Definition::new_with_default_metadata(Kind::json(), [LogNamespace::Legacy])
                    .unknown_fields(Kind::undefined())
                    .with_event_field(
                        &owned_value_path!("message"),
                        Kind::bytes(),
                        Some("message")
                    )
                    .with_event_field(
                        &owned_value_path!("timestamp"),
                        Kind::timestamp(),
                        Some("timestamp")
                    )
                    .with_event_field(&owned_value_path!("message_key"), Kind::bytes(), None)
                    .with_event_field(&owned_value_path!("topic"), Kind::bytes(), None)
                    .with_event_field(&owned_value_path!("partition"), Kind::bytes(), None)
                    .with_event_field(&owned_value_path!("offset"), Kind::bytes(), None)
                    .with_event_field(
                        &owned_value_path!("headers"),
                        Kind::object(Collection::empty().with_unknown(Kind::bytes())),
                        None
                    )
                    .with_event_field(&owned_value_path!("source_type"), Kind::bytes(), None)
            )
        )
    }

    #[tokio::test]
    async fn consumer_create_ok() {
        let config = make_config("topic", "group", LogNamespace::Legacy, None);
        assert!(create_consumer(&config, true).is_ok());
    }

    #[tokio::test]
    async fn consumer_create_incorrect_auto_offset_reset() {
        let config = KafkaSourceConfig {
            auto_offset_reset: "incorrect-auto-offset-reset".to_string(),
            ..make_config("topic", "group", LogNamespace::Legacy, None)
        };
        assert!(create_consumer(&config, true).is_err());
    }
}

#[cfg(feature = "kafka-integration-tests")]
#[cfg(test)]
mod integration_test {
    use std::time::Duration;

    use chrono::{DateTime, SubsecRound, Utc};
    use futures::Stream;
    use futures_util::stream::FuturesUnordered;
    use rdkafka::{
        admin::{AdminClient, AdminOptions, NewTopic, TopicReplication},
        client::DefaultClientContext,
        config::{ClientConfig, FromClientConfig},
        consumer::BaseConsumer,
        message::{Header, OwnedHeaders},
        producer::{FutureProducer, FutureRecord},
        util::Timeout,
        Offset, TopicPartitionList,
    };
    use stream_cancel::{Trigger, Tripwire};
    use tokio::time::sleep;
    use vector_lib::event::EventStatus;
    use vrl::{event_path, value};

    use super::{test::*, *};
    use crate::{
        event::{EventArray, EventContainer},
        shutdown::ShutdownSignal,
        test_util::{collect_n, components::assert_source_compliance, random_string},
        SourceSender,
    };

    const KEY: &str = "my key";
    const TEXT: &str = "my message";
    const HEADER_KEY: &str = "my header";
    const HEADER_VALUE: &str = "my header value";

    fn kafka_test_topic() -> String {
        std::env::var("KAFKA_TEST_TOPIC")
            .unwrap_or_else(|_| format!("test-topic-{}", random_string(10)))
    }
    fn kafka_max_bytes() -> String {
        std::env::var("KAFKA_MAX_BYTES").unwrap_or_else(|_| "1024".into())
    }

    fn client_config<T: FromClientConfig>(group: Option<&str>) -> T {
        let mut client = ClientConfig::new();
        client.set("bootstrap.servers", kafka_address());
        client.set("produce.offset.report", "true");
        client.set("message.timeout.ms", "5000");
        client.set("auto.commit.interval.ms", "1");
        if let Some(group) = group {
            client.set("group.id", group);
        }
        client.create().expect("Producer creation error")
    }

    async fn send_events(topic: String, partitions: i32, count: usize) -> DateTime<Utc> {
        let now = Utc::now();
        let timestamp = now.timestamp_millis();

        let producer: &FutureProducer = &client_config(None);
        let topic_name = topic.as_ref();

        create_topic(topic_name, partitions).await;

        (0..count)
            .map(|i| async move {
                let text = format!("{} {:03}", TEXT, i);
                let key = format!("{} {}", KEY, i);
                let record = FutureRecord::to(topic_name)
                    .payload(&text)
                    .key(&key)
                    .timestamp(timestamp)
                    .headers(OwnedHeaders::new().insert(Header {
                        key: HEADER_KEY,
                        value: Some(HEADER_VALUE),
                    }));
                if let Err(error) = producer.send(record, Timeout::Never).await {
                    panic!("Cannot send event to Kafka: {:?}", error);
                }
            })
            .collect::<FuturesUnordered<_>>()
            .collect::<Vec<_>>()
            .await;

        now
    }

    async fn send_to_test_topic(partitions: i32, count: usize) -> (String, String, DateTime<Utc>) {
        let topic = kafka_test_topic();
        let group_id = format!("test-group-{}", random_string(10));

        let sent_at = send_events(topic.clone(), partitions, count).await;

        (topic, group_id, sent_at)
    }

    #[tokio::test]
    async fn consumes_event_with_acknowledgements() {
        send_receive(true, |_| false, 10, LogNamespace::Legacy).await;
    }

    #[tokio::test]
    async fn consumes_event_with_acknowledgements_vector_namespace() {
        send_receive(true, |_| false, 10, LogNamespace::Vector).await;
    }

    #[tokio::test]
    async fn consumes_event_without_acknowledgements() {
        send_receive(false, |_| false, 10, LogNamespace::Legacy).await;
    }

    #[tokio::test]
    async fn consumes_event_without_acknowledgements_vector_namespace() {
        send_receive(false, |_| false, 10, LogNamespace::Vector).await;
    }

    #[tokio::test]
    async fn handles_one_negative_acknowledgement() {
        send_receive(true, |n| n == 2, 10, LogNamespace::Legacy).await;
    }

    #[tokio::test]
    async fn handles_one_negative_acknowledgement_vector_namespace() {
        send_receive(true, |n| n == 2, 10, LogNamespace::Vector).await;
    }

    #[tokio::test]
    async fn handles_permanent_negative_acknowledgement() {
        send_receive(true, |n| n >= 2, 2, LogNamespace::Legacy).await;
    }

    #[tokio::test]
    async fn handles_permanent_negative_acknowledgement_vector_namespace() {
        send_receive(true, |n| n >= 2, 2, LogNamespace::Vector).await;
    }

    async fn send_receive(
        acknowledgements: bool,
        error_at: impl Fn(usize) -> bool,
        receive_count: usize,
        log_namespace: LogNamespace,
    ) {
        const SEND_COUNT: usize = 10;

        let topic = format!("test-topic-{}", random_string(10));
        let group_id = format!("test-group-{}", random_string(10));
        let config = make_config(&topic, &group_id, log_namespace, None);

        let now = send_events(topic.clone(), 1, 10).await;

        let events = assert_source_compliance(&["protocol", "topic", "partition"], async move {
            let (tx, rx) = SourceSender::new_test_errors(error_at);
            let (trigger_shutdown, shutdown_done) =
                spawn_kafka(tx, config, acknowledgements, false, log_namespace);
            let events = collect_n(rx, SEND_COUNT).await;
            // Yield to the finalization task to let it collect the
            // batch status receivers before signalling the shutdown.
            tokio::task::yield_now().await;
            drop(trigger_shutdown);
            shutdown_done.await;

            events
        })
        .await;

        let offset = fetch_tpl_offset(&group_id, &topic, 0);
        assert_eq!(offset, Offset::from_raw(receive_count as i64));

        assert_eq!(events.len(), SEND_COUNT);
        for (i, event) in events.into_iter().enumerate() {
            if let LogNamespace::Legacy = log_namespace {
                assert_eq!(
                    event.as_log()[log_schema().message_key().unwrap().to_string()],
                    format!("{} {:03}", TEXT, i).into()
                );
                assert_eq!(
                    event.as_log()["message_key"],
                    format!("{} {}", KEY, i).into()
                );
                assert_eq!(
                    event.as_log()[log_schema().source_type_key().unwrap().to_string()],
                    "kafka".into()
                );
                assert_eq!(
                    event.as_log()[log_schema().timestamp_key().unwrap().to_string()],
                    now.trunc_subsecs(3).into()
                );
                assert_eq!(event.as_log()["topic"], topic.clone().into());
                assert!(event.as_log().contains("partition"));
                assert!(event.as_log().contains("offset"));
                let mut expected_headers = ObjectMap::new();
                expected_headers.insert(HEADER_KEY.into(), Value::from(HEADER_VALUE));
                assert_eq!(event.as_log()["headers"], Value::from(expected_headers));
            } else {
                let meta = event.as_log().metadata().value();

                assert_eq!(
                    meta.get(path!("vector", "source_type")).unwrap(),
                    &value!(KafkaSourceConfig::NAME)
                );
                assert!(meta
                    .get(path!("vector", "ingest_timestamp"))
                    .unwrap()
                    .is_timestamp());

                assert_eq!(
                    event.as_log().value(),
                    &value!(format!("{} {:03}", TEXT, i))
                );
                assert_eq!(
                    meta.get(path!("kafka", "message_key")).unwrap(),
                    &value!(format!("{} {}", KEY, i))
                );

                assert_eq!(
                    meta.get(path!("kafka", "timestamp")).unwrap(),
                    &value!(now.trunc_subsecs(3))
                );
                assert_eq!(
                    meta.get(path!("kafka", "topic")).unwrap(),
                    &value!(topic.clone())
                );
                assert!(meta.get(path!("kafka", "partition")).unwrap().is_integer(),);
                assert!(meta.get(path!("kafka", "offset")).unwrap().is_integer(),);

                let mut expected_headers = ObjectMap::new();
                expected_headers.insert(HEADER_KEY.into(), Value::from(HEADER_VALUE));
                assert_eq!(
                    meta.get(path!("kafka", "headers")).unwrap(),
                    &Value::from(expected_headers)
                );
            }
        }
    }

    fn make_rand_config() -> (String, String, KafkaSourceConfig) {
        let topic = format!("test-topic-{}", random_string(10));
        let group_id = format!("test-group-{}", random_string(10));
        let config = make_config(&topic, &group_id, LogNamespace::Legacy, None);
        (topic, group_id, config)
    }

    fn delay_pipeline(
        id: usize,
        delay: Duration,
        status: EventStatus,
    ) -> (SourceSender, impl Stream<Item = EventArray> + Unpin) {
        let (pipe, recv) = SourceSender::new_test_sender_with_buffer(100);
        let recv = recv.into_stream();
        let recv = recv.then(move |item| async move {
            let mut events = item.events;
            events.iter_logs_mut().for_each(|log| {
                log.insert(event_path!("pipeline_id"), id.to_string());
            });
            sleep(delay).await;
            events.iter_events_mut().for_each(|mut event| {
                let metadata = event.metadata_mut();
                metadata.update_status(status);
                metadata.update_sources();
            });
            events
        });
        (pipe, Box::pin(recv))
    }

    fn spawn_kafka(
        out: SourceSender,
        config: KafkaSourceConfig,
        acknowledgements: bool,
        eof: bool,
        log_namespace: LogNamespace,
    ) -> (Trigger, Tripwire) {
        let (trigger_shutdown, shutdown, shutdown_done) = ShutdownSignal::new_wired();

        let decoder = DecodingConfig::new(
            config.framing.clone(),
            config.decoding.clone(),
            log_namespace,
        )
        .build()
        .unwrap();

        let (consumer, callback_rx) = create_consumer(&config, acknowledgements).unwrap();

        tokio::spawn(kafka_source(
            config,
            consumer,
            callback_rx,
            decoder,
            out,
            shutdown,
            eof,
            log_namespace,
        ));
        (trigger_shutdown, shutdown_done)
    }

    fn fetch_tpl_offset(group_id: &str, topic: &str, partition: i32) -> Offset {
        let client: BaseConsumer = client_config(Some(group_id));
        client.subscribe(&[topic]).expect("Subscribing failed");

        let mut tpl = TopicPartitionList::new();
        tpl.add_partition(topic, partition);
        client
            .committed_offsets(tpl, Duration::from_secs(1))
            .expect("Getting committed offsets failed")
            .find_partition(topic, partition)
            .expect("Missing topic/partition")
            .offset()
    }

    async fn create_topic(topic: &str, partitions: i32) {
        let client: AdminClient<DefaultClientContext> = client_config(None);
        let topic_results = client
            .create_topics(
                [&NewTopic {
                    name: topic,
                    num_partitions: partitions,
                    replication: TopicReplication::Fixed(1),
                    config: vec![],
                }],
                &AdminOptions::default(),
            )
            .await
            .expect("create_topics failed");

        for result in topic_results {
            if let Err((topic, err)) = result {
                if err != rdkafka::types::RDKafkaErrorCode::TopicAlreadyExists {
                    panic!("Creating a topic failed: {:?}", (topic, err))
                }
            }
        }
    }

    // Failure timeline:
    // - Topic exists on multiple partitions
    // - Consumer A connects to topic, is assigned both partitions
    // - Consumer A receives some messages
    // - Consumer B connects to topic
    // - Consumer A has one partition revoked (rebalance)
    // - Consumer B is assigned a partition
    // - Consumer A stores an order on the revoked partition
    // - Consumer B skips receiving messages?
    #[ignore]
    #[tokio::test]
    async fn handles_rebalance() {
        // The test plan here is to:
        // - Set up one source instance, feeding into a pipeline that delays acks.
        // - Wait a bit, and set up a second source instance. This should cause a rebalance.
        // - Wait further until all events will have been pulled down.
        // - Verify that all events are captured by the two sources, and that offsets are set right, etc.

        // However this test, as written, does not actually cause the
        // conditions required to test this. We have had external
        // validation that the sink behaves properly on rebalance
        // events.  This test also requires the insertion of a small
        // delay into the source to guarantee the timing, which is not
        // suitable for production code.

        const NEVENTS: usize = 200;
        const DELAY: u64 = 100;

        let (topic, group_id, config) = make_rand_config();
        create_topic(&topic, 2).await;

        let _send_start = send_events(topic.clone(), 1, NEVENTS).await;

        let (tx, rx1) = delay_pipeline(1, Duration::from_millis(200), EventStatus::Delivered);
        let (trigger_shutdown1, shutdown_done1) =
            spawn_kafka(tx, config.clone(), true, false, LogNamespace::Legacy);
        let events1 = tokio::spawn(collect_n(rx1, NEVENTS));

        sleep(Duration::from_secs(1)).await;

        let (tx, rx2) = delay_pipeline(2, Duration::from_millis(DELAY), EventStatus::Delivered);
        let (trigger_shutdown2, shutdown_done2) =
            spawn_kafka(tx, config, true, false, LogNamespace::Legacy);
        let events2 = tokio::spawn(collect_n(rx2, NEVENTS));

        sleep(Duration::from_secs(5)).await;

        drop(trigger_shutdown1);
        let events1 = events1.await.unwrap();
        shutdown_done1.await;

        sleep(Duration::from_secs(5)).await;

        drop(trigger_shutdown2);
        let events2 = events2.await.unwrap();
        shutdown_done2.await;

        sleep(Duration::from_secs(1)).await;

        assert!(!events1.is_empty());
        assert!(!events2.is_empty());

        match fetch_tpl_offset(&group_id, &topic, 0) {
            Offset::Offset(offset) => {
                assert!((offset as isize - events1.len() as isize).abs() <= 1)
            }
            o => panic!("Invalid offset for partition 0 {:?}", o),
        }

        match fetch_tpl_offset(&group_id, &topic, 1) {
            Offset::Offset(offset) => {
                assert!((offset as isize - events2.len() as isize).abs() <= 1)
            }
            o => panic!("Invalid offset for partition 0 {:?}", o),
        }

        let mut all_events = events1
            .into_iter()
            .chain(events2.into_iter())
            .flat_map(map_logs)
            .collect::<Vec<String>>();
        all_events.sort();

        // Assert they are all in sequential order and no dupes, TODO
    }

    #[tokio::test]
    async fn drains_acknowledgements_at_shutdown() {
        // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count)
        let send_count: usize = std::env::var("KAFKA_SEND_COUNT")
            .unwrap_or_else(|_| "125000".into())
            .parse()
            .expect("Number of messages to send to kafka.");
        let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT")
            .unwrap_or_else(|_| format!("{}", send_count))
            .parse()
            .expect("Number of messages to expect consumers to process.");
        let delay_ms: u64 = std::env::var("KAFKA_SHUTDOWN_DELAY")
            .unwrap_or_else(|_| "2000".into())
            .parse()
            .expect("Number of milliseconds before shutting down first consumer.");

        let (topic, group_id, _) = send_to_test_topic(1, send_count).await;

        // 2. Run the kafka source to read some of the events
        // 3. Send a shutdown signal (at some point before all events are read)
        let mut opts = HashMap::new();
        // Set options to get partition EOF notifications, and fetch data in small/configurable size chunks
        opts.insert("enable.partition.eof".into(), "true".into());
        opts.insert("fetch.message.max.bytes".into(), kafka_max_bytes());
        let events1 = {
            let config = make_config(&topic, &group_id, LogNamespace::Legacy, Some(opts.clone()));
            let (tx, rx) = SourceSender::new_test_errors(|_| false);
            let (trigger_shutdown, shutdown_done) =
                spawn_kafka(tx, config, true, false, LogNamespace::Legacy);
            let (events, _) = tokio::join!(rx.collect::<Vec<Event>>(), async move {
                sleep(Duration::from_millis(delay_ms)).await;
                drop(trigger_shutdown);
            });
            shutdown_done.await;
            events
        };

        debug!("Consumer group.id: {}", &group_id);
        debug!(
            "First consumer read {} of {} messages.",
            events1.len(),
            expect_count
        );

        // 4. Run the kafka source again to finish reading the events
        let events2 = {
            let config = make_config(&topic, &group_id, LogNamespace::Legacy, Some(opts));
            let (tx, rx) = SourceSender::new_test_errors(|_| false);
            let (trigger_shutdown, shutdown_done) =
                spawn_kafka(tx, config, true, true, LogNamespace::Legacy);
            let events = rx.collect::<Vec<Event>>().await;
            drop(trigger_shutdown);
            shutdown_done.await;
            events
        };

        debug!(
            "Second consumer read {} of {} messages.",
            events2.len(),
            expect_count
        );

        // 5. Total number of events processed should equal the number sent
        let total = events1.len() + events2.len();
        assert_ne!(
            events1.len(),
            0,
            "First batch of events should be non-zero (increase KAFKA_SHUTDOWN_DELAY?)"
        );
        assert_ne!(events2.len(), 0, "Second batch of events should be non-zero (decrease KAFKA_SHUTDOWN_DELAY or increase KAFKA_SEND_COUNT?) ");
        assert_eq!(total, expect_count);
    }

    async fn consume_with_rebalance(rebalance_strategy: String) {
        // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count)
        let send_count: usize = std::env::var("KAFKA_SEND_COUNT")
            .unwrap_or_else(|_| "125000".into())
            .parse()
            .expect("Number of messages to send to kafka.");
        let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT")
            .unwrap_or_else(|_| format!("{}", send_count))
            .parse()
            .expect("Number of messages to expect consumers to process.");
        let delay_ms: u64 = std::env::var("KAFKA_CONSUMER_DELAY")
            .unwrap_or_else(|_| "2000".into())
            .parse()
            .expect("Number of milliseconds before shutting down first consumer.");

        let (topic, group_id, _) = send_to_test_topic(6, send_count).await;
        debug!("Topic: {}", &topic);
        debug!("Consumer group.id: {}", &group_id);

        // 2. Run the kafka source to read some of the events
        // 3. Start 2nd & 3rd consumers using the same group.id, triggering rebalance events
        let mut kafka_options = HashMap::new();
        kafka_options.insert("enable.partition.eof".into(), "true".into());
        kafka_options.insert("fetch.message.max.bytes".into(), kafka_max_bytes());
        kafka_options.insert("partition.assignment.strategy".into(), rebalance_strategy);
        let config1 = make_config(
            &topic,
            &group_id,
            LogNamespace::Legacy,
            Some(kafka_options.clone()),
        );
        let config2 = config1.clone();
        let config3 = config1.clone();
        let config4 = config1.clone();

        let (events1, events2, events3) = tokio::join!(
            async move {
                let (tx, rx) = SourceSender::new_test_errors(|_| false);
                let (_trigger_shutdown, _shutdown_done) =
                    spawn_kafka(tx, config1, true, true, LogNamespace::Legacy);

                rx.collect::<Vec<Event>>().await
            },
            async move {
                sleep(Duration::from_millis(delay_ms)).await;
                let (tx, rx) = SourceSender::new_test_errors(|_| false);
                let (_trigger_shutdown, _shutdown_done) =
                    spawn_kafka(tx, config2, true, true, LogNamespace::Legacy);

                rx.collect::<Vec<Event>>().await
            },
            async move {
                sleep(Duration::from_millis(delay_ms * 2)).await;
                let (tx, rx) = SourceSender::new_test_errors(|_| false);
                let (_trigger_shutdown, _shutdown_done) =
                    spawn_kafka(tx, config3, true, true, LogNamespace::Legacy);

                rx.collect::<Vec<Event>>().await
            }
        );

        let unconsumed = async move {
            let (tx, rx) = SourceSender::new_test_errors(|_| false);
            let (_trigger_shutdown, _shutdown_done) =
                spawn_kafka(tx, config4, true, true, LogNamespace::Legacy);

            rx.collect::<Vec<Event>>().await
        }
        .await;

        debug!(
            "First consumer read {} of {} messages.",
            events1.len(),
            expect_count
        );

        debug!(
            "Second consumer read {} of {} messages.",
            events2.len(),
            expect_count
        );
        debug!(
            "Third consumer read {} of {} messages.",
            events3.len(),
            expect_count
        );

        // 5. Total number of events processed should equal the number sent
        let total = events1.len() + events2.len() + events3.len();
        assert_ne!(
            events1.len(),
            0,
            "First batch of events should be non-zero (increase delay?)"
        );
        assert_ne!(
            events2.len(),
            0,
            "Second batch of events should be non-zero (decrease delay or increase KAFKA_SEND_COUNT?) "
        );
        assert_ne!(
            events3.len(),
            0,
            "Third batch of events should be non-zero (decrease delay or increase KAFKA_SEND_COUNT?) "
        );
        assert_eq!(
            unconsumed.len(),
            0,
            "The first set of consumers should consume and ack all messages."
        );
        assert_eq!(total, expect_count);
    }

    #[tokio::test]
    async fn drains_acknowledgements_during_rebalance_default_assignments() {
        // the default, eager rebalance strategies generally result in more revocations
        consume_with_rebalance("range,roundrobin".into()).await;
    }
    #[tokio::test]
    async fn drains_acknowledgements_during_rebalance_sticky_assignments() {
        // Cooperative rebalance strategies generally result in fewer revokes,
        // as only reassigned partitions are revoked
        consume_with_rebalance("cooperative-sticky".into()).await;
    }

    fn map_logs(events: EventArray) -> impl Iterator<Item = String> {
        events.into_events().map(|event| {
            let log = event.into_log();
            format!(
                "{} {} {} {}",
                log["message"].to_string_lossy(),
                log["topic"].to_string_lossy(),
                log["partition"].to_string_lossy(),
                log["offset"].to_string_lossy(),
            )
        })
    }
}