crabka-client-streams 0.3.2

KIP-1071 Kafka Streams rebalance-protocol client for Apache Kafka in Rust
Documentation
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
//! Background `StreamsGroupHeartbeat` loop. Mirrors `share/coordinator.rs`: a
//! ticker + `select!` racing each heartbeat against shutdown. Adopts the
//! broker's epoch + assignment, echoes owned tasks back (adopt-and-echo
//! reconciliation), rejoins from epoch 0 on fence, and sends a leave heartbeat
//! (`member_epoch = -1`) on shutdown. Meaningful changes are emitted as
//! [`StreamsEvent`]s.

use std::sync::Arc;
use std::time::Duration;

use tokio::sync::{Mutex, mpsc};
use tokio_util::sync::CancellationToken;

use crabka_client_core::{Client, ClientError};
use crabka_protocol::owned::common::streams_group_heartbeat_request::task_ids::TaskIds as ReqTaskIds;
use crabka_protocol::owned::common::streams_group_heartbeat_request::task_offset::TaskOffset;
use crabka_protocol::owned::common::streams_group_heartbeat_response::task_ids::TaskIds as RespTaskIds;
use crabka_protocol::owned::streams_group_heartbeat_request::StreamsGroupHeartbeatRequest;
use crabka_protocol::owned::streams_group_heartbeat_response::StreamsGroupHeartbeatResponse;

use super::assignment::resolve;
use super::status::map_status;
use super::types::{StreamsAssignment, StreamsEvent, TaskOffsetTracker};
use crate::topology::BuiltTopology;

const FENCED_MEMBER_EPOCH: i16 = 110;
const UNKNOWN_MEMBER_ID: i16 = 25;
const STALE_MEMBER_EPOCH: i16 = 113;

/// The heartbeat RPC the coordinator depends on. Implemented by the real
/// [`Client`]; a fake is injected in tests so the loop is exercisable without a
/// broker.
#[async_trait::async_trait]
pub(crate) trait HeartbeatTransport: Send + Sync + 'static {
    async fn send_heartbeat(
        &self,
        req: StreamsGroupHeartbeatRequest,
    ) -> Result<StreamsGroupHeartbeatResponse, ClientError>;
}

#[async_trait::async_trait]
impl HeartbeatTransport for Client {
    async fn send_heartbeat(
        &self,
        req: StreamsGroupHeartbeatRequest,
    ) -> Result<StreamsGroupHeartbeatResponse, ClientError> {
        self.send(req).await
    }
}

/// State owned by the heartbeat task.
pub(crate) struct CoordinatorState<T: HeartbeatTransport> {
    pub client: T,
    pub group_id: String,
    pub member_id: String,
    pub process_id: String,
    pub instance_id: Option<String>,
    pub rebalance_timeout_ms: i32,
    pub topology: Arc<BuiltTopology>,
    pub member_epoch: Arc<Mutex<i32>>,
    /// Owned tasks last adopted, echoed back as `active_tasks` next heartbeat.
    pub owned_active: Arc<Mutex<Vec<RespTaskIds>>>,
    /// Owned standby tasks last adopted, echoed back next heartbeat.
    pub owned_standby: Arc<Mutex<Vec<RespTaskIds>>>,
    /// Owned warmup tasks last adopted, echoed back next heartbeat.
    pub owned_warmup: Arc<Mutex<Vec<RespTaskIds>>>,
    /// Tracker containing current and end offsets of all tasks.
    pub tracker: Arc<Mutex<TaskOffsetTracker>>,
    pub heartbeat_interval: Duration,
    pub events: mpsc::UnboundedSender<StreamsEvent>,
    /// The last assignment emitted, to suppress duplicate `Assigned` events
    /// (the broker re-sends `active_tasks: Some(...)` every heartbeat).
    pub last_assignment: tokio::sync::Mutex<StreamsAssignment>,
}

enum Outcome {
    Ok,
    Rejoin,
    Transient,
}

/// Drive the loop until `shutdown` fires, then leave.
pub(crate) async fn run<T: HeartbeatTransport>(
    state: CoordinatorState<T>,
    shutdown: CancellationToken,
) {
    let mut ticker = tokio::time::interval(state.heartbeat_interval);
    ticker.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Skip);
    let mut rejoining = false;

    loop {
        tokio::select! {
            () = shutdown.cancelled() => break,
            _ = ticker.tick() => {}
        }
        tokio::select! {
            () = shutdown.cancelled() => break,
            outcome = heartbeat_once(&state, rejoining) => match outcome {
                Outcome::Ok => rejoining = false,
                Outcome::Transient => {}
                Outcome::Rejoin => {
                    *state.member_epoch.lock().await = 0;
                    state.owned_active.lock().await.clear();
                    state.owned_standby.lock().await.clear();
                    state.owned_warmup.lock().await.clear();
                    {
                        let mut lock = state.tracker.lock().await;
                        lock.task_offsets.clear();
                        lock.task_end_offsets.clear();
                    }
                    *state.last_assignment.lock().await = StreamsAssignment::default();
                    rejoining = true;
                    let _ = state.events.send(StreamsEvent::Fenced);
                }
            },
        }
    }

    let leave = state.client.send_heartbeat(StreamsGroupHeartbeatRequest {
        group_id: state.group_id.clone(),
        member_id: state.member_id.clone(),
        member_epoch: -1,
        ..Default::default()
    });
    let _ = tokio::time::timeout(Duration::from_secs(5), leave).await;
}

async fn heartbeat_once<T: HeartbeatTransport>(
    state: &CoordinatorState<T>,
    rejoining: bool,
) -> Outcome {
    let epoch = *state.member_epoch.lock().await;
    let owned = state.owned_active.lock().await.clone();
    let topology = if rejoining || epoch == 0 {
        Some(state.topology.to_wire_request())
    } else {
        None
    };
    let active_tasks = if owned.is_empty() {
        None
    } else {
        Some(owned.iter().map(resp_to_req).collect())
    };
    let owned_standby = state.owned_standby.lock().await.clone();
    let standby_tasks = if owned_standby.is_empty() {
        None
    } else {
        Some(owned_standby.iter().map(resp_to_req).collect())
    };
    let owned_warmup = state.owned_warmup.lock().await.clone();
    let warmup_tasks = if owned_warmup.is_empty() {
        None
    } else {
        Some(owned_warmup.iter().map(resp_to_req).collect())
    };

    let (task_offsets, task_end_offsets) = {
        let tracker = state.tracker.lock().await;
        let to_wire =
            |map: &std::collections::HashMap<(String, i32), i64>| -> Option<Vec<TaskOffset>> {
                if map.is_empty() {
                    None
                } else {
                    let mut list: Vec<TaskOffset> = map
                        .iter()
                        .map(|(key, &offset)| TaskOffset {
                            subtopology_id: key.0.clone(),
                            partition: key.1,
                            offset,
                            ..Default::default()
                        })
                        .collect();
                    list.sort_by(|a, b| match a.subtopology_id.cmp(&b.subtopology_id) {
                        std::cmp::Ordering::Equal => a.partition.cmp(&b.partition),
                        other => other,
                    });
                    Some(list)
                }
            };
        (
            to_wire(&tracker.task_offsets),
            to_wire(&tracker.task_end_offsets),
        )
    };

    let req = StreamsGroupHeartbeatRequest {
        group_id: state.group_id.clone(),
        member_id: state.member_id.clone(),
        member_epoch: epoch,
        process_id: Some(state.process_id.clone()),
        instance_id: state.instance_id.clone(),
        rebalance_timeout_ms: state.rebalance_timeout_ms,
        topology,
        active_tasks,
        standby_tasks,
        warmup_tasks,
        task_offsets,
        task_end_offsets,
        ..Default::default()
    };

    match state.client.send_heartbeat(req).await {
        Ok(r) if r.error_code == 0 => {
            *state.member_epoch.lock().await = r.member_epoch;
            emit_response(state, &r).await;
            Outcome::Ok
        }
        Ok(r)
            if r.error_code == FENCED_MEMBER_EPOCH
                || r.error_code == UNKNOWN_MEMBER_ID
                || r.error_code == STALE_MEMBER_EPOCH =>
        {
            tracing::warn!(
                error_code = r.error_code,
                "streams heartbeat fenced; rejoining"
            );
            Outcome::Rejoin
        }
        Ok(r) => {
            tracing::warn!(
                error_code = r.error_code,
                "unexpected streams heartbeat error"
            );
            Outcome::Transient
        }
        Err(e) => {
            tracing::warn!(error = %e, "streams heartbeat send failed");
            Outcome::Transient
        }
    }
}

/// Emit `NotReady` (status present) and/or `Assigned` (tasks present and
/// changed since last emission), and update the owned-active set for the next
/// echo.
async fn emit_response<T: HeartbeatTransport>(
    state: &CoordinatorState<T>,
    r: &StreamsGroupHeartbeatResponse,
) {
    if let Some(statuses) = &r.status
        && !statuses.is_empty()
    {
        let mapped = statuses.iter().map(map_status).collect();
        let _ = state.events.send(StreamsEvent::NotReady(mapped));
    }
    if let Some(tasks) = &r.active_tasks {
        *state.owned_active.lock().await = tasks.clone();
    }
    if let Some(tasks) = &r.standby_tasks {
        *state.owned_standby.lock().await = tasks.clone();
    }
    if let Some(tasks) = &r.warmup_tasks {
        *state.owned_warmup.lock().await = tasks.clone();
    }
    let mut last = state.last_assignment.lock().await;
    if let Some(ev) = assignment_event(r, &state.topology, &mut last) {
        let _ = state.events.send(ev);
    }
}

/// Build the assignment from a heartbeat response and decide whether it
/// changed since `last`. Returns the event to emit, or `None` if unchanged.
fn assignment_event(
    r: &StreamsGroupHeartbeatResponse,
    topology: &BuiltTopology,
    last: &mut StreamsAssignment,
) -> Option<StreamsEvent> {
    let assignment = StreamsAssignment {
        active: resolve(r.active_tasks.as_ref(), topology),
        standby: resolve(r.standby_tasks.as_ref(), topology),
        warmup: resolve(r.warmup_tasks.as_ref(), topology),
    };
    if assignment == *last {
        None
    } else {
        *last = assignment.clone();
        Some(StreamsEvent::Assigned(assignment))
    }
}

fn resp_to_req(t: &RespTaskIds) -> ReqTaskIds {
    ReqTaskIds {
        subtopology_id: t.subtopology_id.clone(),
        partitions: t.partitions.clone(),
        ..Default::default()
    }
}

#[cfg(test)]
mod tests {
    use std::collections::VecDeque;
    use std::sync::Mutex as StdMutex;

    use assert2::check;
    use crabka_protocol::owned::common::streams_group_heartbeat_response::task_ids::TaskIds as RespTaskIds2;
    use tokio_util::sync::CancellationToken;

    use super::*;
    use crate::topology::Topology;

    // ---------------------------------------------------------------------------
    // Fake transport
    // ---------------------------------------------------------------------------

    struct FakeTransport {
        responses: StdMutex<VecDeque<Result<StreamsGroupHeartbeatResponse, ClientError>>>,
        sent: Arc<StdMutex<Vec<StreamsGroupHeartbeatRequest>>>,
    }

    impl FakeTransport {
        fn new(responses: Vec<StreamsGroupHeartbeatResponse>) -> Self {
            Self {
                responses: StdMutex::new(responses.into_iter().map(Ok).collect()),
                sent: Arc::new(StdMutex::new(Vec::new())),
            }
        }

        fn sent_arc(&self) -> Arc<StdMutex<Vec<StreamsGroupHeartbeatRequest>>> {
            Arc::clone(&self.sent)
        }
    }

    #[async_trait::async_trait]
    impl HeartbeatTransport for FakeTransport {
        async fn send_heartbeat(
            &self,
            req: StreamsGroupHeartbeatRequest,
        ) -> Result<StreamsGroupHeartbeatResponse, ClientError> {
            self.sent.lock().unwrap().push(req);
            self.responses
                .lock()
                .unwrap()
                .pop_front()
                .unwrap_or_else(|| Ok(ok_resp(7, vec![0])))
        }
    }

    // FakeTransport wrapped in Arc also implements the trait (for the run-loop test)
    #[async_trait::async_trait]
    impl HeartbeatTransport for Arc<FakeTransport> {
        async fn send_heartbeat(
            &self,
            req: StreamsGroupHeartbeatRequest,
        ) -> Result<StreamsGroupHeartbeatResponse, ClientError> {
            self.as_ref().send_heartbeat(req).await
        }
    }

    // ---------------------------------------------------------------------------
    // Helpers
    // ---------------------------------------------------------------------------

    fn built() -> Arc<BuiltTopology> {
        use crate::processor::serde::{BytesSerde, Consumed, Produced};
        let mut t = Topology::new();
        let src = t.add_source("src", ["in"], Consumed::with(BytesSerde, BytesSerde));
        t.add_sink("snk", "out", [&src], Produced::with(BytesSerde, BytesSerde));
        Arc::new(t.build("app").unwrap())
    }

    fn ok_resp(epoch: i32, active: Vec<i32>) -> StreamsGroupHeartbeatResponse {
        StreamsGroupHeartbeatResponse {
            error_code: 0,
            member_epoch: epoch,
            heartbeat_interval_ms: 1,
            active_tasks: Some(vec![RespTaskIds2 {
                subtopology_id: "0".into(),
                partitions: active,
                ..Default::default()
            }]),
            ..Default::default()
        }
    }

    fn err_resp(code: i16) -> StreamsGroupHeartbeatResponse {
        StreamsGroupHeartbeatResponse {
            error_code: code,
            ..Default::default()
        }
    }

    fn state_with<T: HeartbeatTransport>(
        client: T,
    ) -> (CoordinatorState<T>, mpsc::UnboundedReceiver<StreamsEvent>) {
        let (tx, rx) = mpsc::unbounded_channel();
        let st = CoordinatorState {
            client,
            group_id: "g".into(),
            member_id: "m".into(),
            process_id: "p".into(),
            instance_id: None,
            rebalance_timeout_ms: 30_000,
            topology: built(),
            member_epoch: Arc::new(Mutex::new(7)),
            owned_active: Arc::new(Mutex::new(Vec::new())),
            owned_standby: Arc::new(Mutex::new(Vec::new())),
            owned_warmup: Arc::new(Mutex::new(Vec::new())),
            tracker: Arc::new(Mutex::new(TaskOffsetTracker::default())),
            heartbeat_interval: Duration::from_millis(1),
            events: tx,
            last_assignment: tokio::sync::Mutex::new(StreamsAssignment::default()),
        };
        (st, rx)
    }

    // ---------------------------------------------------------------------------
    // heartbeat_once tests
    // ---------------------------------------------------------------------------

    #[tokio::test]
    async fn heartbeat_ok_adopts_epoch_and_emits_assignment() {
        let fake = FakeTransport::new(vec![ok_resp(9, vec![0, 1])]);
        let (st, mut rx) = state_with(fake);
        let outcome = heartbeat_once(&st, false).await;
        check!(matches!(outcome, Outcome::Ok));
        check!(*st.member_epoch.lock().await == 9);
        check!(matches!(rx.try_recv(), Ok(StreamsEvent::Assigned(_))));
    }

    #[tokio::test]
    async fn heartbeat_fenced_member_epoch_requests_rejoin() {
        let fake = FakeTransport::new(vec![err_resp(110)]);
        let (st, _rx) = state_with(fake);
        check!(matches!(heartbeat_once(&st, false).await, Outcome::Rejoin));
    }

    #[tokio::test]
    async fn heartbeat_unknown_member_id_requests_rejoin() {
        let fake = FakeTransport::new(vec![err_resp(25)]);
        let (st, _rx) = state_with(fake);
        check!(matches!(heartbeat_once(&st, false).await, Outcome::Rejoin));
    }

    #[tokio::test]
    async fn heartbeat_stale_member_epoch_requests_rejoin() {
        let fake = FakeTransport::new(vec![err_resp(113)]);
        let (st, _rx) = state_with(fake);
        check!(matches!(heartbeat_once(&st, false).await, Outcome::Rejoin));
    }

    #[tokio::test]
    async fn heartbeat_unexpected_code_is_transient() {
        let fake = FakeTransport::new(vec![err_resp(99)]);
        let (st, _rx) = state_with(fake);
        check!(matches!(
            heartbeat_once(&st, false).await,
            Outcome::Transient
        ));
    }

    #[tokio::test]
    async fn heartbeat_transport_error_is_transient() {
        struct ErrTransport;
        #[async_trait::async_trait]
        impl HeartbeatTransport for ErrTransport {
            async fn send_heartbeat(
                &self,
                _req: StreamsGroupHeartbeatRequest,
            ) -> Result<StreamsGroupHeartbeatResponse, ClientError> {
                Err(ClientError::Disconnected)
            }
        }
        let (st, _rx) = state_with(ErrTransport);
        check!(matches!(
            heartbeat_once(&st, false).await,
            Outcome::Transient
        ));
    }

    #[tokio::test]
    async fn heartbeat_sends_topology_when_rejoining() {
        let fake = FakeTransport::new(vec![ok_resp(1, vec![])]);
        let sent = fake.sent_arc();
        let (st, _rx) = state_with(fake);
        let _ = heartbeat_once(&st, true).await;
        let sent = sent.lock().unwrap();
        check!(sent[0].topology.is_some());
    }

    #[tokio::test]
    async fn heartbeat_sends_topology_when_epoch_zero() {
        let fake = FakeTransport::new(vec![ok_resp(1, vec![])]);
        let sent = fake.sent_arc();
        let (st, _rx) = state_with(fake);
        *st.member_epoch.lock().await = 0;
        let _ = heartbeat_once(&st, false).await;
        let sent = sent.lock().unwrap();
        check!(sent[0].topology.is_some());
    }

    #[tokio::test]
    async fn heartbeat_echoes_owned_active_tasks() {
        use crabka_protocol::owned::common::streams_group_heartbeat_response::task_ids::TaskIds as RespTids;
        let fake = FakeTransport::new(vec![ok_resp(8, vec![0])]);
        let sent = fake.sent_arc();
        let (st, _rx) = state_with(fake);
        // Pre-populate owned_active
        *st.owned_active.lock().await = vec![RespTids {
            subtopology_id: "0".into(),
            partitions: vec![0, 1],
            ..Default::default()
        }];
        let _ = heartbeat_once(&st, false).await;
        let sent = sent.lock().unwrap();
        check!(sent[0].active_tasks.is_some());
    }

    #[tokio::test]
    async fn emit_response_sends_not_ready_for_status() {
        use crabka_protocol::owned::common::streams_group_heartbeat_response::status::Status;
        let fake = FakeTransport::new(vec![]);
        let (st, mut rx) = state_with(fake);
        let resp = StreamsGroupHeartbeatResponse {
            error_code: 0,
            member_epoch: 1,
            status: Some(vec![Status {
                status_code: 0,
                status_detail: "topo-stale".into(),
                ..Default::default()
            }]),
            ..Default::default()
        };
        emit_response(&st, &resp).await;
        check!(matches!(rx.try_recv(), Ok(StreamsEvent::NotReady(_))));
    }

    // ---------------------------------------------------------------------------
    // run-loop tests
    // ---------------------------------------------------------------------------

    #[tokio::test]
    async fn run_loop_heartbeats_then_leaves_on_shutdown() {
        let fake = Arc::new(FakeTransport::new(vec![ok_resp(8, vec![0, 1])]));
        let sent = fake.sent_arc();
        let (st, mut rx) = state_with(Arc::clone(&fake));
        let shutdown = CancellationToken::new();
        let handle = tokio::spawn(run(st, shutdown.clone()));
        tokio::time::sleep(Duration::from_millis(20)).await;
        shutdown.cancel();
        handle.await.unwrap();
        // A leave heartbeat (member_epoch == -1) must have been sent last.
        let sent = sent.lock().unwrap();
        check!(sent.iter().any(|r| r.member_epoch == -1));
        // At least one Assigned event was emitted.
        check!(matches!(rx.try_recv(), Ok(StreamsEvent::Assigned(_))));
    }

    #[tokio::test]
    async fn run_loop_fenced_emits_fenced_event_and_resets_epoch() {
        // First response fences, subsequent ones succeed.
        let fake = Arc::new(FakeTransport::new(vec![err_resp(110)]));
        let (st, mut rx) = state_with(Arc::clone(&fake));
        let shutdown = CancellationToken::new();
        let handle = tokio::spawn(run(st, shutdown.clone()));
        tokio::time::sleep(Duration::from_millis(30)).await;
        shutdown.cancel();
        handle.await.unwrap();
        // Collect all events
        let mut events = Vec::new();
        while let Ok(ev) = rx.try_recv() {
            events.push(ev);
        }
        check!(events.iter().any(|e| matches!(e, StreamsEvent::Fenced)));
    }

    #[tokio::test]
    async fn run_loop_shutdown_immediately_sends_leave() {
        let fake = Arc::new(FakeTransport::new(vec![]));
        let sent = fake.sent_arc();
        let (st, _rx) = state_with(Arc::clone(&fake));
        let shutdown = CancellationToken::new();
        // Cancel immediately before any tick
        shutdown.cancel();
        run(st, shutdown).await;
        let sent = sent.lock().unwrap();
        // Even with immediate shutdown, leave heartbeat must be sent
        check!(sent.iter().any(|r| r.member_epoch == -1));
    }

    // ---------------------------------------------------------------------------
    // assignment_event (existing tests preserved)
    // ---------------------------------------------------------------------------

    fn built_plain() -> BuiltTopology {
        use crate::processor::serde::{BytesSerde, Consumed, Produced};
        let mut t = Topology::new();
        let src = t.add_source("src", ["in"], Consumed::with(BytesSerde, BytesSerde));
        t.add_sink("snk", "out", [&src], Produced::with(BytesSerde, BytesSerde));
        t.build("app").unwrap()
    }

    fn resp_plain(active: Vec<i32>) -> StreamsGroupHeartbeatResponse {
        use crabka_protocol::owned::common::streams_group_heartbeat_response::task_ids::TaskIds;
        StreamsGroupHeartbeatResponse {
            active_tasks: Some(vec![TaskIds {
                subtopology_id: "0".into(),
                partitions: active,
                ..Default::default()
            }]),
            ..Default::default()
        }
    }

    #[test]
    fn identical_assignment_is_not_re_emitted() {
        let topo = built_plain();
        let mut last = StreamsAssignment::default();
        let r = resp_plain(vec![0, 1]);
        check!(assignment_event(&r, &topo, &mut last).is_some());
        check!(assignment_event(&r, &topo, &mut last).is_none());
    }

    #[test]
    fn empty_assignment_is_not_emitted_from_default() {
        let topo = built_plain();
        let mut last = StreamsAssignment::default();
        let empty = StreamsGroupHeartbeatResponse {
            active_tasks: Some(vec![]),
            ..Default::default()
        };
        check!(assignment_event(&empty, &topo, &mut last).is_none());
    }

    #[test]
    fn changed_assignment_is_re_emitted() {
        let topo = built_plain();
        let mut last = StreamsAssignment::default();
        check!(assignment_event(&resp_plain(vec![0]), &topo, &mut last).is_some());
        check!(assignment_event(&resp_plain(vec![0, 1]), &topo, &mut last).is_some());
    }
}