11
11
12
12
use std:: time:: Duration ;
13
13
14
- use differential_dataflow:: { difference:: DiffPair , operators:: count:: CountTotal } ;
14
+ use differential_dataflow:: difference:: { DiffPair , DiffVector } ;
15
+ use differential_dataflow:: operators:: count:: CountTotal ;
15
16
use log:: error;
16
17
use timely:: communication:: Allocate ;
17
18
use timely:: dataflow:: operators:: capture:: EventLink ;
@@ -38,6 +39,34 @@ pub enum MaterializedEvent {
38
39
/// Globally unique identifier for the source on which the dataflow depends.
39
40
source : GlobalId ,
40
41
} ,
42
+ /// Tracks statistics for a particular Kafka consumer / partition pair
43
+ /// Reference: https://github.com/edenhill/librdkafka/blob/master/STATISTICS.md
44
+ KafkaConsumerInfo {
45
+ /// Kafka name for the consumer
46
+ consumer_name : String ,
47
+ /// Materialize source identifier
48
+ source_id : SourceInstanceId ,
49
+ /// The Kafka partition ID for these metrics (may be multiple per consumer)
50
+ partition_id : String ,
51
+ /// Number of message sets received from Brokers
52
+ rxmsgs : i64 ,
53
+ /// Number of bytes received from Brokers
54
+ rxbytes : i64 ,
55
+ /// Number of message sets sent to Brokers
56
+ txmsgs : i64 ,
57
+ /// Number of bytes transmitted to Brokers
58
+ txbytes : i64 ,
59
+ /// Partition's low watermark offset on the broker
60
+ lo_offset : i64 ,
61
+ /// Partition's high watermark offset on the broker
62
+ hi_offset : i64 ,
63
+ /// Last stable offset on the broker
64
+ ls_offset : i64 ,
65
+ /// How far into the topic our consumer has read
66
+ app_offset : i64 ,
67
+ /// How many messages remain until our consumer reaches the (hi|lo) watermark
68
+ consumer_lag : i64 ,
69
+ } ,
41
70
/// Peek command, true for install and false for retire.
42
71
Peek ( Peek , bool ) ,
43
72
/// Tracks the source name, id, partition id, and received/ingested offsets
@@ -103,9 +132,10 @@ pub fn construct<A: Allocate>(
103
132
let mut input = demux. new_input ( & logs, Pipeline ) ;
104
133
let ( mut dataflow_out, dataflow) = demux. new_output ( ) ;
105
134
let ( mut dependency_out, dependency) = demux. new_output ( ) ;
135
+ let ( mut frontier_out, frontier) = demux. new_output ( ) ;
136
+ let ( mut kafka_consumer_info_out, kafka_consumer_info) = demux. new_output ( ) ;
106
137
let ( mut peek_out, peek) = demux. new_output ( ) ;
107
138
let ( mut source_info_out, source_info) = demux. new_output ( ) ;
108
- let ( mut frontier_out, frontier) = demux. new_output ( ) ;
109
139
110
140
let mut demux_buffer = Vec :: new ( ) ;
111
141
demux. build ( move |_capability| {
@@ -114,18 +144,20 @@ pub fn construct<A: Allocate>(
114
144
move |_frontiers| {
115
145
let mut dataflow = dataflow_out. activate ( ) ;
116
146
let mut dependency = dependency_out. activate ( ) ;
147
+ let mut frontier = frontier_out. activate ( ) ;
148
+ let mut kafka_consumer_info = kafka_consumer_info_out. activate ( ) ;
117
149
let mut peek = peek_out. activate ( ) ;
118
150
let mut source_info = source_info_out. activate ( ) ;
119
- let mut frontier = frontier_out. activate ( ) ;
120
151
121
152
input. for_each ( |time, data| {
122
153
data. swap ( & mut demux_buffer) ;
123
154
124
155
let mut dataflow_session = dataflow. session ( & time) ;
125
156
let mut dependency_session = dependency. session ( & time) ;
157
+ let mut frontier_session = frontier. session ( & time) ;
158
+ let mut kafka_consumer_info_session = kafka_consumer_info. session ( & time) ;
126
159
let mut peek_session = peek. session ( & time) ;
127
160
let mut source_info_session = source_info. session ( & time) ;
128
- let mut frontier_session = frontier. session ( & time) ;
129
161
130
162
for ( time, worker, datum) in demux_buffer. drain ( ..) {
131
163
let time_ns = time. as_nanos ( ) as Timestamp ;
@@ -176,6 +208,47 @@ pub fn construct<A: Allocate>(
176
208
) ,
177
209
}
178
210
}
211
+ MaterializedEvent :: Frontier ( name, logical, delta) => {
212
+ frontier_session. give ( (
213
+ row_packer. pack ( & [
214
+ Datum :: String ( & name. to_string ( ) ) ,
215
+ Datum :: Int64 ( worker as i64 ) ,
216
+ Datum :: Int64 ( logical as i64 ) ,
217
+ ] ) ,
218
+ time_ms,
219
+ delta as isize ,
220
+ ) ) ;
221
+ }
222
+ MaterializedEvent :: KafkaConsumerInfo {
223
+ consumer_name,
224
+ source_id,
225
+ partition_id,
226
+ rxmsgs,
227
+ rxbytes,
228
+ txmsgs,
229
+ txbytes,
230
+ lo_offset,
231
+ hi_offset,
232
+ ls_offset,
233
+ app_offset,
234
+ consumer_lag,
235
+ } => {
236
+ kafka_consumer_info_session. give ( (
237
+ ( consumer_name, source_id, partition_id) ,
238
+ time_ms,
239
+ DiffVector :: new ( vec ! [
240
+ rxmsgs,
241
+ rxbytes,
242
+ txmsgs,
243
+ txbytes,
244
+ lo_offset,
245
+ hi_offset,
246
+ ls_offset,
247
+ app_offset,
248
+ consumer_lag,
249
+ ] ) ,
250
+ ) ) ;
251
+ }
179
252
MaterializedEvent :: Peek ( peek, is_install) => {
180
253
peek_session. give ( ( peek, worker, is_install, time_ns) )
181
254
}
@@ -192,17 +265,6 @@ pub fn construct<A: Allocate>(
192
265
DiffPair :: new ( offset, timestamp) ,
193
266
) ) ;
194
267
}
195
- MaterializedEvent :: Frontier ( name, logical, delta) => {
196
- frontier_session. give ( (
197
- row_packer. pack ( & [
198
- Datum :: String ( & name. to_string ( ) ) ,
199
- Datum :: Int64 ( worker as i64 ) ,
200
- Datum :: Int64 ( logical as i64 ) ,
201
- ] ) ,
202
- time_ms,
203
- delta as isize ,
204
- ) ) ;
205
- }
206
268
}
207
269
}
208
270
} ) ;
@@ -245,6 +307,30 @@ pub fn construct<A: Allocate>(
245
307
}
246
308
} ) ;
247
309
310
+ let frontier_current = frontier. as_collection ( ) ;
311
+
312
+ use differential_dataflow:: operators:: Count ;
313
+ let kafka_consumer_info_current = kafka_consumer_info. as_collection ( ) . count ( ) . map ( {
314
+ let mut row_packer = repr:: RowPacker :: new ( ) ;
315
+ move |( ( consumer_name, source_id, partition_id) , diff_vector) | {
316
+ row_packer. pack ( & [
317
+ Datum :: String ( & consumer_name) ,
318
+ Datum :: String ( & source_id. source_id . to_string ( ) ) ,
319
+ Datum :: Int64 ( source_id. dataflow_id as i64 ) ,
320
+ Datum :: String ( & partition_id) ,
321
+ Datum :: Int64 ( diff_vector[ 0 ] ) ,
322
+ Datum :: Int64 ( diff_vector[ 1 ] ) ,
323
+ Datum :: Int64 ( diff_vector[ 2 ] ) ,
324
+ Datum :: Int64 ( diff_vector[ 3 ] ) ,
325
+ Datum :: Int64 ( diff_vector[ 4 ] ) ,
326
+ Datum :: Int64 ( diff_vector[ 5 ] ) ,
327
+ Datum :: Int64 ( diff_vector[ 6 ] ) ,
328
+ Datum :: Int64 ( diff_vector[ 7 ] ) ,
329
+ Datum :: Int64 ( diff_vector[ 8 ] ) ,
330
+ ] )
331
+ }
332
+ } ) ;
333
+
248
334
let peek_current = peek
249
335
. map ( move |( name, worker, is_install, time_ns) | {
250
336
let time_ms = ( time_ns / 1_000_000 ) as Timestamp ;
@@ -265,7 +351,6 @@ pub fn construct<A: Allocate>(
265
351
}
266
352
} ) ;
267
353
268
- use differential_dataflow:: operators:: Count ;
269
354
let source_info_current = source_info. as_collection ( ) . count ( ) . map ( {
270
355
let mut row_packer = repr:: RowPacker :: new ( ) ;
271
356
move |( ( name, id, pid) , pair) | {
@@ -282,8 +367,6 @@ pub fn construct<A: Allocate>(
282
367
}
283
368
} ) ;
284
369
285
- let frontier_current = frontier. as_collection ( ) ;
286
-
287
370
// Duration statistics derive from the non-rounded event times.
288
371
let peek_duration = peek
289
372
. unary (
@@ -361,6 +444,10 @@ pub fn construct<A: Allocate>(
361
444
LogVariant :: Materialized ( MaterializedLog :: FrontierCurrent ) ,
362
445
frontier_current,
363
446
) ,
447
+ (
448
+ LogVariant :: Materialized ( MaterializedLog :: KafkaConsumerInfo ) ,
449
+ kafka_consumer_info_current,
450
+ ) ,
364
451
(
365
452
LogVariant :: Materialized ( MaterializedLog :: PeekCurrent ) ,
366
453
peek_current,
0 commit comments