-
Notifications
You must be signed in to change notification settings - Fork 65
/
Copy pathsession.rs
1445 lines (1292 loc) · 59.7 KB
/
session.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 super::{App, Collection, Read};
use crate::{
api_client::KafkaClientAuth, from_downstream_topic_name, from_upstream_topic_name,
logging::propagate_task_forwarder, read::BatchResult, to_downstream_topic_name,
to_upstream_topic_name, topology::PartitionOffset, KafkaApiClient, SessionAuthentication,
};
use anyhow::{bail, Context};
use bytes::{BufMut, Bytes, BytesMut};
use futures::TryFutureExt;
use kafka_protocol::{
error::{ParseResponseErrorCode, ResponseError},
messages::{
self,
metadata_response::{
MetadataResponseBroker, MetadataResponsePartition, MetadataResponseTopic,
},
ConsumerProtocolAssignment, ConsumerProtocolSubscription, ListGroupsResponse,
RequestHeader, TopicName,
},
protocol::{buf::ByteBuf, Decodable, Encodable, Message, StrBytes},
};
use rustls::crypto::hash::Hash;
use std::{cmp::max, sync::Arc};
use std::{
collections::{hash_map::Entry, HashMap},
time::SystemTime,
};
use tracing::instrument;
struct PendingRead {
offset: i64, // Journal offset to be completed by this PendingRead.
last_write_head: i64, // Most-recent observed journal write head.
handle: tokio_util::task::AbortOnDropHandle<anyhow::Result<(Read, BatchResult)>>,
}
#[derive(Clone, Debug)]
enum SessionDataPreviewState {
Unknown,
NotDataPreview,
DataPreview(HashMap<(TopicName, i32), PartitionOffset>),
}
pub struct Session {
app: Arc<App>,
client: Option<KafkaApiClient>,
reads: HashMap<(TopicName, i32), (PendingRead, std::time::Instant)>,
secret: String,
auth: Option<SessionAuthentication>,
data_preview_state: SessionDataPreviewState,
broker_urls: Vec<String>,
msk_region: String,
// ------ This can be cleaned up once everyone is migrated off of the legacy connection mode ------
legacy_mode_broker_urls: Vec<String>,
legacy_mode_broker_username: String,
legacy_mode_broker_password: String,
// ------------------------------------------------------------------------------------------------
}
impl Session {
pub fn new(
app: Arc<App>,
secret: String,
broker_urls: Vec<String>,
msk_region: String,
legacy_mode_broker_urls: Vec<String>,
legacy_mode_broker_username: String,
legacy_mode_broker_password: String,
) -> Self {
Self {
app,
client: None,
broker_urls,
msk_region,
legacy_mode_broker_urls,
legacy_mode_broker_username,
legacy_mode_broker_password,
reads: HashMap::new(),
auth: None,
secret,
data_preview_state: SessionDataPreviewState::Unknown,
}
}
async fn get_kafka_client(&mut self) -> anyhow::Result<&mut KafkaApiClient> {
if let Some(ref mut client) = self.client {
Ok(client)
} else {
let (auth, urls) = match self.auth {
Some(SessionAuthentication::Task(_)) => (
KafkaClientAuth::MSK {
aws_region: self.msk_region.clone(),
provider: aws_config::from_env()
.region(aws_types::region::Region::new(self.msk_region.clone()))
.load()
.await
.credentials_provider()
.unwrap(),
cached: None,
},
self.broker_urls.as_slice(),
),
Some(SessionAuthentication::User(_)) => (
KafkaClientAuth::NonRefreshing(rsasl::config::SASLConfig::with_credentials(
None,
self.legacy_mode_broker_username.clone(),
self.legacy_mode_broker_password.clone(),
)?),
self.legacy_mode_broker_urls.as_slice(),
),
None => anyhow::bail!("Must be authenticated"),
};
self.client.replace(
KafkaApiClient::connect(
urls,
auth,
).await.context(
"failed to connect or authenticate to upstream Kafka broker used for serving group management APIs",
)?
);
Ok(self.client.as_mut().expect("guaranteed to exist"))
}
}
/// SASL handshake responds with supported SASL mechanisms.
/// We support PLAIN user/password, because we expect the password to be a control-plane token.
pub async fn sasl_handshake(
&mut self,
request: messages::SaslHandshakeRequest,
) -> anyhow::Result<messages::SaslHandshakeResponse> {
if let Some(_) = self.auth {
anyhow::bail!("This session is already authenticated")
}
let mut response = messages::SaslHandshakeResponse::default();
response.mechanisms.push(StrBytes::from_static_str("PLAIN"));
if request.mechanism.ne("PLAIN") {
response.error_code = ResponseError::UnsupportedSaslMechanism.code();
}
Ok(response)
}
/// Parse a PLAIN user/password to extract a control-plane access token.
pub async fn sasl_authenticate(
&mut self,
request: messages::SaslAuthenticateRequest,
) -> anyhow::Result<messages::SaslAuthenticateResponse> {
if let Some(_) = self.auth {
anyhow::bail!("This session is already authenticated")
}
let mut it = request
.auth_bytes
.split(|b| *b == 0) // SASL uses NULL to separate components.
.map(std::str::from_utf8);
let _authzid = it.next().context("expected SASL authzid")??;
let authcid = it.next().context("expected SASL authcid")??;
let password = it.next().context("expected SASL passwd")??;
let response = match self.app.authenticate(authcid, password).await {
Ok(auth) => {
let mut response = messages::SaslAuthenticateResponse::default();
response.session_lifetime_ms = (auth
.valid_until()
.duration_since(SystemTime::now())?
.as_secs()
* 1000)
.try_into()?;
self.auth.replace(auth);
response
}
Err(err) => messages::SaslAuthenticateResponse::default()
.with_error_code(ResponseError::SaslAuthenticationFailed.code())
.with_error_message(Some(StrBytes::from_string(format!(
"SASL authentication error: Authentication failed: {err:#}",
)))),
};
Ok(response)
}
/// Serve metadata of topics and their partitions.
/// For efficiency, we do NOT enumerate partitions when we receive an unqualified metadata request.
/// Otherwise, if specific "topics" (collections) are listed, we fetch and map journals into partitions.
#[instrument(skip_all)]
pub async fn metadata(
&mut self,
mut request: messages::MetadataRequest,
) -> anyhow::Result<messages::MetadataResponse> {
let topics = match request.topics.take() {
Some(topics) if topics.len() > 0 => self.metadata_select_topics(topics).await,
_ => self.metadata_all_topics().await,
}?;
// We only ever advertise a single logical broker.
let brokers = vec![MetadataResponseBroker::default()
.with_node_id(messages::BrokerId(1))
.with_host(StrBytes::from_string(self.app.advertise_host.clone()))
.with_port(self.app.advertise_kafka_port as i32)];
Ok(messages::MetadataResponse::default()
.with_brokers(brokers)
.with_cluster_id(Some(StrBytes::from_static_str("estuary-dekaf")))
.with_controller_id(messages::BrokerId(1))
.with_topics(topics))
}
// Lists all read-able collections as Kafka topics. Omits partition metadata.
async fn metadata_all_topics(&mut self) -> anyhow::Result<Vec<MetadataResponseTopic>> {
let collections = self
.auth
.as_mut()
.ok_or(anyhow::anyhow!("Session not authenticated"))?
.fetch_all_collection_names()
.await?;
tracing::debug!(collections=?ops::DebugJson(&collections), "fetched all collections");
let topics = collections
.into_iter()
.map(|name| {
Ok(MetadataResponseTopic::default()
.with_name(Some(self.encode_topic_name(name)?))
.with_is_internal(false)
.with_partitions(vec![MetadataResponsePartition::default()
.with_partition_index(0)
.with_leader_id(0.into())]))
})
.collect::<anyhow::Result<_>>()?;
Ok(topics)
}
// Lists partitions of specific, requested collections.
async fn metadata_select_topics(
&mut self,
requests: Vec<messages::metadata_request::MetadataRequestTopic>,
) -> anyhow::Result<Vec<MetadataResponseTopic>> {
let auth = self
.auth
.as_mut()
.ok_or(anyhow::anyhow!("Session not authenticated"))?;
let app = &self.app;
let pg_client = &auth.flow_client(app).await?.pg_client();
// Re-declare here to drop mutable reference
let auth = self.auth.as_ref().unwrap();
// Concurrently fetch Collection instances for all requested topics.
let collections: anyhow::Result<Vec<(TopicName, Option<Collection>)>> =
futures::future::try_join_all(requests.into_iter().map(|topic| async move {
let maybe_collection = Collection::new(
app,
auth,
pg_client,
from_downstream_topic_name(topic.name.to_owned().unwrap_or_default()).as_str(),
)
.await?;
Ok((topic.name.unwrap_or_default(), maybe_collection))
}))
.await;
let mut topics = vec![];
for (name, maybe_collection) in collections? {
let Some(collection) = maybe_collection else {
topics.push(
MetadataResponseTopic::default()
.with_name(Some(self.encode_topic_name(name.to_string())?))
.with_error_code(ResponseError::UnknownTopicOrPartition.code()),
);
continue;
};
let partitions = collection
.partitions
.iter()
.enumerate()
.map(|(index, _)| {
messages::metadata_response::MetadataResponsePartition::default()
.with_partition_index(index as i32)
.with_leader_id(messages::BrokerId(1))
.with_replica_nodes(vec![messages::BrokerId(1)])
.with_isr_nodes(vec![messages::BrokerId(1)])
})
.collect();
topics.push(
MetadataResponseTopic::default()
.with_name(Some(name))
.with_is_internal(false)
.with_partitions(partitions),
);
}
Ok(topics)
}
/// FindCoordinator always responds with our single logical broker.
pub async fn find_coordinator(
&mut self,
request: messages::FindCoordinatorRequest,
) -> anyhow::Result<messages::FindCoordinatorResponse> {
let coordinators = request
.coordinator_keys
.iter()
.map(|_key| {
messages::find_coordinator_response::Coordinator::default()
.with_node_id(messages::BrokerId(1))
.with_host(StrBytes::from_string(self.app.advertise_host.clone()))
.with_port(self.app.advertise_kafka_port as i32)
})
.collect();
Ok(messages::FindCoordinatorResponse::default()
.with_node_id(messages::BrokerId(1))
.with_host(StrBytes::from_string(self.app.advertise_host.clone()))
.with_port(self.app.advertise_kafka_port as i32)
.with_coordinators(coordinators))
}
pub async fn list_offsets(
&mut self,
request: messages::ListOffsetsRequest,
) -> anyhow::Result<messages::ListOffsetsResponse> {
let auth = self
.auth
.as_mut()
.ok_or(anyhow::anyhow!("Session not authenticated"))?;
let app = &self.app;
let pg_client = &auth.flow_client(app).await?.pg_client();
// Re-declare here to drop mutable reference
let auth = self.auth.as_ref().unwrap();
// Concurrently fetch Collection instances and offsets for all requested topics and partitions.
// Map each "topic" into Vec<(Partition Index, Option<PartitionOffset>.
let collections: anyhow::Result<Vec<(TopicName, Vec<(i32, Option<PartitionOffset>)>)>> =
futures::future::try_join_all(request.topics.into_iter().map(|topic| async move {
let maybe_collection = Collection::new(
app,
auth,
pg_client,
from_downstream_topic_name(topic.name.clone()).as_str(),
)
.await?;
let Some(collection) = maybe_collection else {
return Ok((
topic.name,
topic
.partitions
.iter()
.map(|p| (p.partition_index, None))
.collect(),
));
};
let collection = &collection;
// Concurrently fetch requested offset for each named partition.
let offsets: anyhow::Result<_> = futures::future::try_join_all(
topic.partitions.into_iter().map(|partition| async move {
Ok((
partition.partition_index,
collection
.fetch_partition_offset(
partition.partition_index as usize,
partition.timestamp, // In millis.
)
.await?,
))
}),
)
.await;
Ok((topic.name, offsets?))
}))
.await;
let collections = collections?;
use messages::list_offsets_response::{
ListOffsetsPartitionResponse, ListOffsetsTopicResponse,
};
// Map topics, partition indices, and fetched offsets into a comprehensive response.
let response = collections
.into_iter()
.map(|(topic_name, offsets)| {
let partitions = offsets
.into_iter()
.map(|(partition_index, maybe_offset)| {
let Some(PartitionOffset {
offset,
mod_time: timestamp,
..
}) = maybe_offset
else {
return ListOffsetsPartitionResponse::default()
.with_partition_index(partition_index)
.with_error_code(ResponseError::UnknownTopicOrPartition.code());
};
ListOffsetsPartitionResponse::default()
.with_partition_index(partition_index)
.with_offset(offset)
.with_timestamp(timestamp)
})
.collect();
ListOffsetsTopicResponse::default()
.with_name(topic_name)
.with_partitions(partitions)
})
.collect();
Ok(messages::ListOffsetsResponse::default().with_topics(response))
}
/// Fetch records from select "partitions" (journals) and "topics" (collections).
#[tracing::instrument(
skip_all,
fields(
max_wait_ms=request.max_wait_ms
)
)]
pub async fn fetch(
&mut self,
request: messages::FetchRequest,
) -> anyhow::Result<messages::FetchResponse> {
use messages::fetch_response::{FetchableTopicResponse, PartitionData};
let messages::FetchRequest {
topics: topic_requests,
max_bytes: _, // Ignored.
max_wait_ms,
min_bytes: _, // Ignored.
session_id,
..
} = request;
let timeout = std::time::Duration::from_millis(max_wait_ms as u64);
// Start reads for all partitions which aren't already pending.
for topic_request in &topic_requests {
let mut key = (from_downstream_topic_name(topic_request.topic.clone()), 0);
for partition_request in &topic_request.partitions {
key.1 = partition_request.partition;
let fetch_offset = partition_request.fetch_offset;
let data_preview_params: Option<PartitionOffset> = match self
.data_preview_state
.to_owned()
{
// On the first Fetch call, check to see whether it is considered a data-preview
// fetch or not. If so, flag the whole session as being tainted, and also keep track
// of the neccesary offset data in order to serve the rewritten data preview responses.
SessionDataPreviewState::Unknown => {
if let Some(state) = self
.is_fetch_data_preview(key.0.to_string(), key.1, fetch_offset)
.await?
{
let mut data_preview_state = HashMap::new();
data_preview_state.insert(key.to_owned(), state);
self.data_preview_state =
SessionDataPreviewState::DataPreview(data_preview_state);
Some(state)
} else {
self.data_preview_state = SessionDataPreviewState::NotDataPreview;
None
}
}
// If the first Fetch request in a session was not considered for data preview,
// then skip all further checks in order to avoid slowing down fetches.
SessionDataPreviewState::NotDataPreview => None,
SessionDataPreviewState::DataPreview(mut state) => {
match state.entry(key.to_owned()) {
// If a session is marked as being used for data preview, and this Fetch request
// is for a topic/partition that we've already loaded the offsets for, re-use them
// so long as the request is still a data preview request. If not, bail out
Entry::Occupied(entry) => {
let data_preview_state = entry.get();
if fetch_offset >= data_preview_state.offset
|| data_preview_state.offset - fetch_offset > 12
{
bail!("Session was used for fetching preview data, cannot be used for fetching non-preview data.")
}
Some(data_preview_state.to_owned())
}
// Otherwise, load the offsets for this new topic/partition, and also ensure that this is
// still a data-preview request. If not, bail out.
Entry::Vacant(entry) => {
if let Some(state) = self
.is_fetch_data_preview(key.0.to_string(), key.1, fetch_offset)
.await?
{
entry.insert(state);
Some(state)
} else {
bail!("Session was used for fetching preview data, cannot be used for fetching non-preview data.")
}
}
}
}
};
match self.reads.get(&key) {
Some((_, started_at))
if started_at.elapsed() > std::time::Duration::from_secs(60 * 5) =>
{
metrics::counter!(
"dekaf_fetch_requests",
"topic_name" => key.0.to_string(),
"partition_index" => key.1.to_string(),
"state" => "read_expired"
)
.increment(1);
tracing::debug!(lifetime=?started_at.elapsed(), topic_name=?key.0,partition_index=?key.1, "Restarting expired Read");
self.reads.remove(&key);
let auth = self
.auth
.as_mut()
.ok_or(anyhow::anyhow!("Session not authenticated"))?;
auth.refresh_gazette_clients();
}
Some((pending, _)) if pending.offset == fetch_offset => {
metrics::counter!(
"dekaf_fetch_requests",
"topic_name" => key.0.to_string(),
"partition_index" => key.1.to_string(),
"state" => "read_pending"
)
.increment(1);
continue; // Common case: fetch is at the pending offset.
}
_ => {}
}
let auth = self.auth.as_mut().unwrap();
let pg_client = auth.flow_client(&self.app).await?.pg_client();
let Some(collection) =
Collection::new(&self.app, &auth, &pg_client, &key.0).await?
else {
metrics::counter!(
"dekaf_fetch_requests",
"topic_name" => key.0.to_string(),
"partition_index" => key.1.to_string(),
"state" => "collection_not_found"
)
.increment(1);
tracing::debug!(collection = ?&key.0, "Collection doesn't exist!");
continue; // Collection doesn't exist.
};
let Some(partition) = collection
.partitions
.get(partition_request.partition as usize)
else {
metrics::counter!(
"dekaf_fetch_requests",
"topic_name" => key.0.to_string(),
"partition_index" => key.1.to_string(),
"state" => "partition_not_found"
)
.increment(1);
tracing::debug!(collection = ?&key.0, partition=partition_request.partition, "Partition doesn't exist!");
continue; // Partition doesn't exist.
};
let (key_schema_id, value_schema_id) =
collection.registered_schema_ids(&pg_client).await?;
let pending = PendingRead {
offset: fetch_offset,
last_write_head: fetch_offset,
handle: tokio_util::task::AbortOnDropHandle::new(match data_preview_params {
// Startree: 0, Tinybird: 12
Some(PartitionOffset {
fragment_start,
offset: latest_offset,
..
}) if latest_offset - fetch_offset <= 12 => {
let diff = latest_offset - fetch_offset;
metrics::counter!(
"dekaf_fetch_requests",
"topic_name" => key.0.to_string(),
"partition_index" => key.1.to_string(),
"state" => "new_data_preview_read"
)
.increment(1);
tokio::spawn(propagate_task_forwarder(
Read::new(
collection.journal_client.clone(),
&collection,
partition,
fragment_start,
key_schema_id,
value_schema_id,
Some(partition_request.fetch_offset - 1),
&auth,
)?
.next_batch(
// Have to read at least 2 docs, as the very last doc
// will probably be a control document and will be
// ignored by the consumer, looking like 0 docs were read
crate::read::ReadTarget::Docs(max(diff as usize, 2)),
std::time::Instant::now() + timeout,
),
))
}
_ => {
metrics::counter!(
"dekaf_fetch_requests",
"topic_name" => key.0.to_string(),
"partition_index" => key.1.to_string(),
"state" => "new_regular_read"
)
.increment(1);
tokio::spawn(propagate_task_forwarder(
Read::new(
collection.journal_client.clone(),
&collection,
partition,
fetch_offset,
key_schema_id,
value_schema_id,
None,
&auth,
)?
.next_batch(
crate::read::ReadTarget::Bytes(
partition_request.partition_max_bytes as usize,
),
std::time::Instant::now() + timeout,
),
))
}
}),
};
tracing::info!(
journal = &partition.spec.name,
key_schema_id,
value_schema_id,
fetch_offset,
"started read",
);
if let Some((old, started_at)) = self
.reads
.insert(key.clone(), (pending, std::time::Instant::now()))
{
tracing::warn!(
topic = topic_request.topic.as_str(),
partition = partition_request.partition,
old_offset = old.offset,
new_offset = fetch_offset,
read_lifetime = ?started_at.elapsed(),
"discarding pending read due to offset jump",
);
}
}
}
// Poll pending reads across all requested topics.
let mut topic_responses = Vec::with_capacity(topic_requests.len());
for topic_request in &topic_requests {
let mut key = (from_downstream_topic_name(topic_request.topic.clone()), 0);
let mut partition_responses = Vec::with_capacity(topic_request.partitions.len());
for partition_request in &topic_request.partitions {
key.1 = partition_request.partition;
let Some((pending, _)) = self.reads.get_mut(&key) else {
partition_responses.push(
PartitionData::default()
.with_partition_index(partition_request.partition)
.with_error_code(ResponseError::UnknownTopicOrPartition.code()),
);
continue;
};
let (read, batch) = (&mut pending.handle).await??;
let batch = match batch {
BatchResult::TargetExceededBeforeTimeout(b) => Some(b),
BatchResult::TimeoutExceededBeforeTarget(b) => Some(b),
BatchResult::TimeoutNoData | BatchResult::Suspended => None,
};
let mut partition_data = PartitionData::default()
.with_partition_index(partition_request.partition)
// `kafka-protocol` encodes None here using a length of -1, but librdkafka client library
// complains with: `Protocol parse failure for Fetch v11 ... invalid MessageSetSize -1`
// An empty Bytes will get encoded with a length of 0, which works fine.
.with_records(batch.or(Some(Bytes::new())).to_owned());
match &self.data_preview_state {
SessionDataPreviewState::Unknown => {
unreachable!("Must have already determined data-preview status of session")
}
SessionDataPreviewState::NotDataPreview => {
pending.offset = read.offset;
pending.last_write_head = read.last_write_head;
pending.handle = tokio_util::task::AbortOnDropHandle::new(tokio::spawn(
propagate_task_forwarder(read.next_batch(
crate::read::ReadTarget::Bytes(
partition_request.partition_max_bytes as usize,
),
std::time::Instant::now() + timeout,
)),
));
partition_data = partition_data
.with_high_watermark(pending.last_write_head) // Map to kafka cursor.
.with_last_stable_offset(pending.last_write_head);
}
SessionDataPreviewState::DataPreview(data_preview_states) => {
let data_preview_state = data_preview_states
.get(&key)
.expect("should be able to find data preview state by this point");
partition_data = partition_data
.with_high_watermark(data_preview_state.offset) // Map to kafka cursor.
.with_last_stable_offset(data_preview_state.offset);
self.reads.remove(&key);
}
}
partition_responses.push(partition_data);
}
topic_responses.push(
FetchableTopicResponse::default()
.with_topic(topic_request.topic.clone())
.with_partitions(partition_responses),
);
}
Ok(messages::FetchResponse::default()
.with_session_id(session_id)
.with_responses(topic_responses))
}
/// DescribeConfigs lists configuration metadata of topics.
/// This is used only by `kaf` thus far, is informational, and is currently just a stub.
pub async fn describe_configs(
&mut self,
req: messages::DescribeConfigsRequest,
) -> anyhow::Result<messages::DescribeConfigsResponse> {
use kafka_protocol::messages::describe_configs_response::*;
let mut results = Vec::new();
for resource in req.resources.iter() {
if resource.resource_type == 2 {
// Describe config of a named topic.
let fixtures = [("some-key", "some-value"), ("another-key", "another-value")];
let configs = fixtures
.into_iter()
.map(|(name, value)| {
DescribeConfigsResourceResult::default()
.with_name(StrBytes::from_static_str(name))
.with_value(Some(StrBytes::from_static_str(value)))
.with_read_only(true)
})
.collect();
results.push(
DescribeConfigsResult::default()
.with_resource_name(resource.resource_name.clone())
.with_configs(configs),
)
}
}
Ok(DescribeConfigsResponse::default().with_results(results))
}
/// Produce is assumed to be supported in various places, and clients using librdkafka
/// break when that assumption isn't satisfied. For example, the `Fetch` API > version 0
/// appears to (indirectly) assume that the broker supports `Produce`.
/// For example: Each of these 3 conditions (`MSGVER1`, `MSGVER2`, `THROTTLE_TIME`) require `Produce`,
/// and when it's not present the consumer will sit in a tight loop endlessly failing to
/// send a fetch request because it's missing an API version flag:
/// https://github.com/confluentinc/librdkafka/blob/master/src/rdkafka_fetcher.c#L997-L1005
pub async fn produce(
&mut self,
req: messages::ProduceRequest,
) -> anyhow::Result<messages::ProduceResponse> {
use kafka_protocol::messages::produce_response::*;
let responses = req
.topic_data
.into_iter()
.map(|v| {
TopicProduceResponse::default().with_partition_responses(
v.partition_data
.into_iter()
.map(|part| {
PartitionProduceResponse::default()
.with_index(part.index)
.with_error_code(
kafka_protocol::error::ResponseError::InvalidRequest.code(),
)
})
.collect(),
)
})
.collect();
Ok(ProduceResponse::default().with_responses(responses))
}
#[instrument(skip_all, fields(group=?req.group_id))]
pub async fn join_group(
&mut self,
req: messages::JoinGroupRequest,
header: RequestHeader,
) -> anyhow::Result<messages::JoinGroupResponse> {
let mut mutable_req = req.clone();
for protocol in mutable_req.protocols.iter_mut() {
let mut consumer_protocol_subscription_raw = protocol.metadata.clone();
let consumer_protocol_subscription_version = consumer_protocol_subscription_raw
.try_get_i16()
.context("failed to parse consumer protocol message: subscription version")?;
tracing::debug!(
version = consumer_protocol_subscription_version,
remaining_bytes = consumer_protocol_subscription_raw.len(),
"Got consumer protocol message version"
);
if consumer_protocol_subscription_version > ConsumerProtocolSubscription::VERSIONS.max
|| consumer_protocol_subscription_version
< ConsumerProtocolSubscription::VERSIONS.min
{
anyhow::bail!(
"Recieved ConsumerProtocolSubscription message with version {} which is outside of the acceptable range of ({}, {})",
consumer_protocol_subscription_version,
ConsumerProtocolSubscription::VERSIONS.min,
ConsumerProtocolSubscription::VERSIONS.max
)
}
let formatted = format!("{consumer_protocol_subscription_raw:?}");
let mut consumer_protocol_subscription_msg = ConsumerProtocolSubscription::decode(
&mut consumer_protocol_subscription_raw,
consumer_protocol_subscription_version, // Seems that sometimes v >=1 doesn't decode properly
)
.context(format!(
"failed to parse consumer protocol message body: {formatted}"
))?;
consumer_protocol_subscription_msg
.topics
.iter_mut()
.try_for_each(|topic| {
let transformed = self.encrypt_topic_name(topic.to_owned().into())?.into();
tracing::info!(topic_name = ?topic, "Request to join group");
*topic = transformed;
Ok::<(), anyhow::Error>(())
})?;
let mut new_protocol_subscription = BytesMut::new();
new_protocol_subscription.put_i16(consumer_protocol_subscription_version);
consumer_protocol_subscription_msg.encode(
&mut new_protocol_subscription,
consumer_protocol_subscription_version,
)?;
protocol.metadata = new_protocol_subscription.into();
}
let response = self
.get_kafka_client()
.await?
.connect_to_group_coordinator(req.group_id.as_str())
.await?
.send_request(mutable_req.clone(), Some(header))
.await?;
if let Some(err) = response.error_code.err() {
tracing::debug!(?err, req=?mutable_req, "Request errored");
return Ok(response);
}
// Now re-translate response
let mut mutable_resp = response.clone();
for member in mutable_resp.members.iter_mut() {
let mut consumer_protocol_subscription_raw = member.metadata.clone();
let consumer_protocol_subscription_version =
consumer_protocol_subscription_raw.try_get_i16().context(
"failed to parse consumer protocol message: subscription version re-encode",
)?;
let mut consumer_protocol_subscription_msg = ConsumerProtocolSubscription::decode(
&mut consumer_protocol_subscription_raw,
consumer_protocol_subscription_version, // it seems that sometimes v >= 1 doesn't decode properly
)
.context("failed to parse consumer protocol message: subscription re-encode")?;
consumer_protocol_subscription_msg
.topics
.iter_mut()
.try_for_each(|topic| {
*topic = self.decrypt_topic_name(topic.to_owned().into())?.into();
Ok::<(), anyhow::Error>(())
})?;
let mut new_protocol_subscription = BytesMut::new();
new_protocol_subscription.put_i16(consumer_protocol_subscription_version);
consumer_protocol_subscription_msg.encode(
&mut new_protocol_subscription,
consumer_protocol_subscription_version,
)?;
member.metadata = new_protocol_subscription.into();
}
Ok(mutable_resp)
}
#[instrument(skip_all, fields(group=?req.group_id))]
pub async fn leave_group(
&mut self,
req: messages::LeaveGroupRequest,
header: RequestHeader,
) -> anyhow::Result<messages::LeaveGroupResponse> {
let client = self
.get_kafka_client()
.await?
.connect_to_group_coordinator(req.group_id.as_str())
.await?;
let response = client.send_request(req, Some(header)).await?;
Ok(response)
}
#[tracing::instrument(skip_all)]
pub async fn list_groups(
&mut self,
req: messages::ListGroupsRequest,
header: RequestHeader,
) -> anyhow::Result<messages::ListGroupsResponse> {
// Redpanda seems to randomly disconnect this?
let r = self
.get_kafka_client()
.await?
.send_request(req, Some(header))
.await;
match r {
Ok(mut e) => {
if let Some(err) = e.error_code.err() {
tracing::warn!(err = ?err, "Error listing groups!");
}
// Multiple systems had trouble when this returned the actual list of groups...
// and AFAICT nothing has any trouble when we return an empty list here.
e.groups = vec![]; //e
// .groups
// .into_iter()
// .filter(|grp| !grp.group_id.starts_with("amazon.msk"))
// .collect_vec();
return Ok(e);
}
Err(e) => {
tracing::warn!(e=?e, "Failed to list_groups");
Ok(ListGroupsResponse::default().with_groups(vec![]))
}
}
}
#[instrument(skip_all, fields(group=?req.group_id))]
pub async fn sync_group(
&mut self,
req: messages::SyncGroupRequest,
header: RequestHeader,
) -> anyhow::Result<messages::SyncGroupResponse> {
let mut mutable_req = req.clone();
for assignment in mutable_req.assignments.iter_mut() {
let mut consumer_protocol_assignment_raw = assignment.assignment.clone();
let consumer_protocol_assignment_version = consumer_protocol_assignment_raw
.try_get_i16()
.context("failed to parse consumer protocol message: assignment version")?;
if consumer_protocol_assignment_version > ConsumerProtocolAssignment::VERSIONS.max
|| consumer_protocol_assignment_version < ConsumerProtocolAssignment::VERSIONS.min
{