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
use super::{CodecWriteError, Direction, message_latency};
use crate::codec::{CodecBuilder, CodecReadError, CodecState};
use crate::frame::kafka::KafkaFrame;
use crate::frame::{Frame, MessageType};
use crate::message::{Encodable, Message, MessageId, Messages};
use anyhow::{Result, anyhow};
use bytes::BytesMut;
use kafka_protocol::messages::{ApiKey, RequestKind, ResponseKind};
use kafka_protocol::protocol::StrBytes;
use metrics::Histogram;
use std::sync::mpsc;
use std::time::Instant;
use tokio_util::codec::{Decoder, Encoder};
#[derive(Copy, Clone, Debug, PartialEq)]
pub struct RequestHeader {
pub api_key: ApiKey,
pub version: i16,
}
#[derive(Clone)]
pub struct KafkaCodecBuilder {
direction: Direction,
message_latency: Histogram,
}
// Depending on if the codec is used in a sink or a source requires different processing logic:
// * Sources parse requests which do not require any special handling
// * Sinks parse responses which requires first matching up the version and api_key with its corresponding request
// + To achieve this Sinks use an mpsc channel to send header data from the encoder to the decoder
impl CodecBuilder for KafkaCodecBuilder {
type Decoder = KafkaDecoder;
type Encoder = KafkaEncoder;
fn new(direction: Direction, destination_name: String) -> Self {
let message_latency = message_latency(direction, destination_name);
Self {
direction,
message_latency,
}
}
fn build(&self) -> (KafkaDecoder, KafkaEncoder) {
let (tx, rx) = match self.direction {
Direction::Source => (None, None),
Direction::Sink => {
let (tx, rx) = mpsc::channel();
(Some(tx), Some(rx))
}
};
(
KafkaDecoder::new(rx, self.direction),
KafkaEncoder::new(tx, self.direction, self.message_latency.clone()),
)
}
fn protocol(&self) -> MessageType {
MessageType::Kafka
}
}
#[derive(Debug)]
pub struct RequestInfo {
header: RequestHeader,
id: MessageId,
expect_raw_sasl: Option<SaslMessageState>,
}
// Keeps track of the next expected sasl message
#[derive(Debug, Clone, PartialEq, Copy)]
pub enum SaslMessageState {
/// The next message will be a sasl message in the PLAIN mechanism
Plain,
/// The next message will be the first sasl message in the SCRAM mechanism
ScramFirst,
/// The next message will be the final sasl message in the SCRAM mechanism
ScramFinal,
}
impl SaslMessageState {
fn from_name(mechanism: &StrBytes) -> Result<SaslMessageState> {
match mechanism.as_str() {
"PLAIN" => Ok(SaslMessageState::Plain),
"SCRAM-SHA-512" => Ok(SaslMessageState::ScramFirst),
"SCRAM-SHA-256" => Ok(SaslMessageState::ScramFirst),
mechanism => Err(anyhow!("Unknown sasl mechanism {mechanism}")),
}
}
}
pub struct KafkaDecoder {
// Some when Sink (because it receives responses)
request_header_rx: Option<mpsc::Receiver<RequestInfo>>,
direction: Direction,
expect_raw_sasl: Option<SaslMessageState>,
}
impl KafkaDecoder {
pub fn new(
request_header_rx: Option<mpsc::Receiver<RequestInfo>>,
direction: Direction,
) -> Self {
KafkaDecoder {
request_header_rx,
direction,
expect_raw_sasl: None,
}
}
}
fn get_length_of_full_message(src: &BytesMut) -> Option<usize> {
if src.len() >= 4 {
let size = u32::from_be_bytes(src[0..4].try_into().unwrap()) as usize + 4;
if size <= src.len() { Some(size) } else { None }
} else {
None
}
}
impl Decoder for KafkaDecoder {
type Item = Messages;
type Error = CodecReadError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
let received_at = Instant::now();
if let Some(size) = get_length_of_full_message(src) {
let bytes = src.split_to(size);
tracing::debug!(
"{}: incoming kafka message:\n{}",
self.direction,
pretty_hex::pretty_hex(&bytes)
);
let request_info = self
.request_header_rx
.as_ref()
.map(|rx| rx.recv())
.transpose()
.map_err(|_| CodecReadError::Parser(anyhow!("kafka encoder half was lost")))?;
struct Meta {
request_header: RequestHeader,
message_id: Option<u128>,
}
let meta = if let Some(RequestInfo { header, id, .. }) = request_info {
Meta {
request_header: header,
message_id: Some(id),
}
} else if self.expect_raw_sasl.is_some() {
Meta {
request_header: RequestHeader {
api_key: ApiKey::SaslAuthenticate,
version: 0,
},
// This code path is only used for requests, so message_id can be None.
message_id: None,
}
} else {
Meta {
request_header: RequestHeader {
api_key: ApiKey::try_from(i16::from_be_bytes(
bytes[4..6].try_into().unwrap(),
))
.unwrap(),
version: i16::from_be_bytes(bytes[6..8].try_into().unwrap()),
},
// This code path is only used for requests, so message_id can be None.
message_id: None,
}
};
let mut message = if let Some(id) = meta.message_id.as_ref() {
let mut message = Message::from_bytes_at_instant(
bytes.freeze(),
CodecState::Kafka(KafkaCodecState {
request_header: Some(meta.request_header),
raw_sasl: self.expect_raw_sasl.is_some(),
}),
Some(received_at),
);
message.set_request_id(*id);
message
} else {
Message::from_bytes_at_instant(
bytes.freeze(),
CodecState::Kafka(KafkaCodecState {
request_header: None,
raw_sasl: self.expect_raw_sasl.is_some(),
}),
Some(received_at),
)
};
// advanced to the next state of expect_raw_sasl
self.expect_raw_sasl = match self.expect_raw_sasl {
Some(SaslMessageState::Plain) => None,
Some(SaslMessageState::ScramFirst) => Some(SaslMessageState::ScramFinal),
Some(SaslMessageState::ScramFinal) => None,
None => None,
};
if let Some(request_info) = request_info {
// set expect_raw_sasl for responses
if let Some(expect_raw_sasl) = request_info.expect_raw_sasl {
self.expect_raw_sasl = Some(expect_raw_sasl);
}
} else {
// set expect_raw_sasl for requests
if meta.request_header.api_key == ApiKey::SaslHandshake
&& meta.request_header.version == 0
{
// Only parse the full frame once we manually check its a v0 sasl handshake
if let Some(Frame::Kafka(KafkaFrame::Request {
body: RequestKind::SaslHandshake(sasl_handshake),
..
})) = message.frame()
{
self.expect_raw_sasl = Some(
SaslMessageState::from_name(&sasl_handshake.mechanism)
.map_err(CodecReadError::Parser)?,
);
// Clear raw bytes of the message to force the encoder to encode from frame.
// This is needed because the encoder only has access to the frame if it does not have any raw bytes,
// and the encoder needs to inspect the frame to set its own sasl state.
message.invalidate_cache();
}
}
}
Ok(Some(vec![message]))
} else {
Ok(None)
}
}
}
pub struct KafkaEncoder {
message_latency: Histogram,
// Some when Sink (because it sends requests)
request_header_tx: Option<mpsc::Sender<RequestInfo>>,
direction: Direction,
}
impl KafkaEncoder {
pub fn new(
request_header_tx: Option<mpsc::Sender<RequestInfo>>,
direction: Direction,
message_latency: Histogram,
) -> Self {
KafkaEncoder {
message_latency,
request_header_tx,
direction,
}
}
}
impl Encoder<Messages> for KafkaEncoder {
type Error = CodecWriteError;
fn encode(&mut self, item: Messages, dst: &mut BytesMut) -> Result<(), Self::Error> {
item.into_iter().try_for_each(|mut m| {
let start = dst.len();
m.ensure_message_type(MessageType::Kafka)
.map_err(CodecWriteError::Encoder)?;
let response_is_dummy = m.response_is_dummy();
let id = m.id();
let received_at = m.received_from_source_or_sink_at;
let message_contains_raw_sasl = if let CodecState::Kafka(codec_state) = m.codec_state {
codec_state.raw_sasl
} else {
false
};
let mut expect_raw_sasl = None;
let result = match m.into_encodable() {
Encodable::Bytes(bytes) => {
dst.extend_from_slice(&bytes);
Ok(())
}
Encodable::Frame(frame) => {
if message_contains_raw_sasl {
match *frame {
Frame::Kafka(KafkaFrame::Request {
body: RequestKind::SaslAuthenticate(body),
..
}) => {
dst.extend_from_slice(&body.auth_bytes);
}
Frame::Kafka(KafkaFrame::Response {
body: ResponseKind::SaslAuthenticate(body),
..
}) => {
dst.extend_from_slice(&body.auth_bytes);
}
_ => unreachable!(
"Expected kafka sasl authenticate request or response but was not"
),
}
Ok(())
} else {
let frame = frame.into_kafka().unwrap();
// it is garanteed that all v0 SaslHandshakes will be in a parsed state since we parse + invalidate_cache in the KafkaDecoder.
if let KafkaFrame::Request {
body: RequestKind::SaslHandshake(sasl_handshake),
header,
} = &frame
{
if header.request_api_version == 0 {
expect_raw_sasl = Some(
SaslMessageState::from_name(&sasl_handshake.mechanism)
.map_err(CodecWriteError::Encoder)?,
);
}
}
frame.encode(dst)
}
}
};
// Skip if the message wrote nothing to dst, possibly due to being a dummy message.
// or if it will generate a dummy response
if !dst[start..].is_empty() && !response_is_dummy {
if let Some(tx) = self.request_header_tx.as_ref() {
let header = if message_contains_raw_sasl {
RequestHeader {
api_key: ApiKey::SaslAuthenticate,
version: 0,
}
} else {
let api_key =
i16::from_be_bytes(dst[start + 4..start + 6].try_into().unwrap());
let version =
i16::from_be_bytes(dst[start + 6..start + 8].try_into().unwrap());
// TODO: handle unknown API key
let api_key = ApiKey::try_from(api_key).map_err(|_| {
CodecWriteError::Encoder(anyhow!("unknown api key {api_key}"))
})?;
RequestHeader { api_key, version }
};
let request_info = RequestInfo {
header,
id,
expect_raw_sasl,
};
tx.send(request_info)
.map_err(|e| CodecWriteError::Encoder(anyhow!(e)))?;
}
}
if let Some(received_at) = received_at {
self.message_latency.record(received_at.elapsed());
}
tracing::debug!(
"{}: outgoing kafka message:\n{}",
self.direction,
pretty_hex::pretty_hex(&&dst[start..])
);
result.map_err(CodecWriteError::Encoder)
})
}
}
#[cfg(feature = "kafka")]
#[derive(Debug, Clone, PartialEq, Copy)]
pub struct KafkaCodecState {
/// When the message is:
/// a request - this value is None
/// a response - this value is Some and contains the header values of the corresponding request.
pub request_header: Option<RequestHeader>,
/// When `true` this message is not a valid kafka protocol message and is instead a raw SASL message.
/// KafkaFrame will parse this as a SaslHandshake to hide the legacy raw SASL message from transform implementations.
pub raw_sasl: bool,
}