forked from vectordotdev/vector
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathkafka.rs
2173 lines (1938 loc) · 81.6 KB
/
kafka.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
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
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);