fluxmq 0.1.0

High-performance message broker and streaming platform inspired by Apache Kafka
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
use super::{BrokerId, LogEntry, ReplicationConfig, ReplicationMessage};
use crate::protocol::{Message, Offset, PartitionId, TopicName};
use crate::storage::HybridStorage;
use crate::Result;
use crossbeam::channel;
use std::sync::Arc;
use tokio::sync::{Mutex, RwLock};
use tracing::{debug, error, info, warn};

/// Follower state for a partition
#[derive(Debug)]
pub struct FollowerState {
    broker_id: BrokerId,
    topic: TopicName,
    partition: PartitionId,
    leader_id: BrokerId,
    _config: ReplicationConfig,

    /// Current term known by this follower
    current_term: Arc<RwLock<u64>>,

    /// Last log offset committed by the leader
    commit_index: Arc<RwLock<Offset>>,

    /// Last log offset applied by this follower
    last_applied: Arc<RwLock<Offset>>,

    /// Channel for receiving replication requests from leader
    request_rx: Arc<Mutex<Option<channel::Receiver<ReplicationMessage>>>>,
    _request_tx: channel::Sender<ReplicationMessage>,

    /// Channel for sending responses back to leader
    response_tx: channel::Sender<ReplicationMessage>,
}

impl FollowerState {
    pub fn new(
        broker_id: BrokerId,
        topic: TopicName,
        partition: PartitionId,
        leader_id: BrokerId,
        config: ReplicationConfig,
    ) -> Self {
        let (request_tx, request_rx) = channel::unbounded();
        let (response_tx, _) = channel::unbounded(); // Response channel would connect to leader

        Self {
            broker_id,
            topic,
            partition,
            leader_id,
            _config: config,
            current_term: Arc::new(RwLock::new(0)),
            commit_index: Arc::new(RwLock::new(0)),
            last_applied: Arc::new(RwLock::new(0)),
            request_rx: Arc::new(Mutex::new(Some(request_rx))),
            _request_tx: request_tx,
            response_tx,
        }
    }

    /// Start the follower's replication tasks
    pub async fn start_replication(self: Arc<Self>, storage: Arc<HybridStorage>) -> Result<()> {
        // Start request handler task
        let handler_state = Arc::clone(&self);
        let handler_storage = Arc::clone(&storage);
        tokio::spawn(async move {
            handler_state.handle_requests(handler_storage).await;
        });

        info!(
            "Started follower replication for {}:{} with leader {}",
            self.topic, self.partition, self.leader_id
        );

        Ok(())
    }

    /// Handle incoming replication requests from the leader
    async fn handle_requests(&self, storage: Arc<HybridStorage>) {
        let request_rx = {
            let mut rx_guard = self.request_rx.lock().await;
            if let Some(rx) = rx_guard.take() {
                rx
            } else {
                return;
            }
        };

        loop {
            // Use spawn_blocking to handle crossbeam channel
            let result = tokio::task::spawn_blocking({
                let rx_clone = request_rx.clone();
                move || rx_clone.recv()
            })
            .await;

            match result {
                Ok(Ok(request)) => {
                    if let Err(e) = self.process_request(request, storage.clone()).await {
                        error!("Failed to process replication request: {}", e);
                    }
                }
                Ok(Err(_)) => break, // Channel closed
                Err(_) => break,     // spawn_blocking failed
            }
        }
    }

    /// Process a replication request from the leader
    async fn process_request(
        &self,
        request: ReplicationMessage,
        storage: Arc<HybridStorage>,
    ) -> Result<()> {
        match request {
            ReplicationMessage::ReplicateRequest {
                topic,
                partition,
                leader_epoch,
                prev_log_offset,
                entries,
                leader_commit,
            } => {
                debug!(
                    "Processing replicate request for {}:{}, epoch: {}, prev_offset: {}, {} entries",
                    topic, partition, leader_epoch, prev_log_offset, entries.len()
                );

                // Update current term if leader's epoch is higher
                {
                    let mut current_term = self.current_term.write().await;
                    if leader_epoch > *current_term {
                        *current_term = leader_epoch;
                    }
                }

                // Check if we can append these entries
                let can_append = self.can_append_entries(&storage, prev_log_offset).await?;

                if can_append {
                    // Apply the log entries
                    let entries_len = entries.len();
                    let last_offset = self.apply_entries(&storage, entries).await?;

                    // Update commit index if leader's commit is higher
                    {
                        let mut commit_idx = self.commit_index.write().await;
                        if leader_commit > *commit_idx {
                            *commit_idx = leader_commit.min(last_offset);
                        }
                    }

                    // Send success response
                    let response = ReplicationMessage::ReplicateResponse {
                        topic,
                        partition,
                        success: true,
                        last_log_offset: last_offset,
                        follower_id: self.broker_id,
                    };

                    if let Err(_) = self.response_tx.send(response) {
                        warn!("Failed to send replication response to leader");
                    }

                    debug!(
                        "Successfully replicated {} entries, last_offset: {}",
                        entries_len, last_offset
                    );
                } else {
                    // Send failure response - leader will decrement next_index and retry
                    let current_offset = storage.get_latest_offset(&topic, partition).unwrap_or(0);

                    let response = ReplicationMessage::ReplicateResponse {
                        topic,
                        partition,
                        success: false,
                        last_log_offset: current_offset,
                        follower_id: self.broker_id,
                    };

                    if let Err(_) = self.response_tx.send(response) {
                        warn!("Failed to send replication response to leader");
                    }

                    debug!(
                        "Cannot append entries at prev_offset: {}, current_offset: {}",
                        prev_log_offset, current_offset
                    );
                }
            }
            ReplicationMessage::Heartbeat {
                leader_id,
                term,
                commit_index: leader_commit,
            } => {
                debug!(
                    "Received heartbeat from leader {} for {}:{}, term: {}, commit: {}",
                    leader_id, self.topic, self.partition, term, leader_commit
                );

                // Update current term
                let mut success = true;
                {
                    let mut current_term = self.current_term.write().await;
                    if term >= *current_term {
                        *current_term = term;
                    } else {
                        success = false; // Reject heartbeat from outdated leader
                    }
                }

                // Update commit index
                if success {
                    let mut commit_idx = self.commit_index.write().await;
                    if leader_commit > *commit_idx {
                        let current_offset = storage
                            .get_latest_offset(&self.topic, self.partition)
                            .unwrap_or(0);
                        *commit_idx = leader_commit.min(current_offset);
                    }
                }

                // Send heartbeat response
                let response = ReplicationMessage::HeartbeatResponse {
                    follower_id: self.broker_id,
                    term: *self.current_term.read().await,
                    success,
                };

                if let Err(_) = self.response_tx.send(response) {
                    warn!("Failed to send heartbeat response to leader");
                }
            }
            _ => {
                warn!("Received unexpected message type for follower");
            }
        }

        Ok(())
    }

    /// Check if we can append entries at the given previous log offset
    async fn can_append_entries(
        &self,
        storage: &Arc<HybridStorage>,
        prev_log_offset: Offset,
    ) -> Result<bool> {
        let current_offset = storage
            .get_latest_offset(&self.topic, self.partition)
            .unwrap_or(0);

        // If prev_log_offset is 0, we can always append (starting from beginning)
        if prev_log_offset == 0 {
            return Ok(current_offset == 0);
        }

        // Check if our log matches at prev_log_offset
        // In a full implementation, we'd also check the term at that offset
        Ok(current_offset >= prev_log_offset)
    }

    /// Apply log entries to local storage
    async fn apply_entries(
        &self,
        storage: &Arc<HybridStorage>,
        entries: Vec<LogEntry>,
    ) -> Result<Offset> {
        if entries.is_empty() {
            let current_offset = storage
                .get_latest_offset(&self.topic, self.partition)
                .unwrap_or(0);
            return Ok(current_offset);
        }

        let entries_len = entries.len();
        let messages: Vec<Message> = entries.into_iter().map(|entry| entry.message).collect();
        let base_offset = storage.append_messages(&self.topic, self.partition, messages)?;

        // Update last applied offset
        let last_offset = base_offset + (entries_len as u64) - 1;
        {
            let mut last_applied = self.last_applied.write().await;
            *last_applied = last_offset;
        }

        Ok(last_offset)
    }

    /// Get the current commit index
    pub async fn get_commit_index(&self) -> Offset {
        *self.commit_index.read().await
    }

    /// Get the last applied offset
    pub async fn get_last_applied(&self) -> Offset {
        *self.last_applied.read().await
    }

    /// Get the current term
    pub async fn get_current_term(&self) -> u64 {
        *self.current_term.read().await
    }
}

/// Manages follower synchronization with leaders
pub struct FollowerSync {
    broker_id: BrokerId,
    followers: Arc<RwLock<std::collections::HashMap<(TopicName, PartitionId), Arc<FollowerState>>>>,
    storage: Arc<HybridStorage>,
    config: ReplicationConfig,
}

impl FollowerSync {
    pub fn new(
        broker_id: BrokerId,
        storage: Arc<HybridStorage>,
        config: ReplicationConfig,
    ) -> Self {
        Self {
            broker_id,
            followers: Arc::new(RwLock::new(std::collections::HashMap::new())),
            storage,
            config,
        }
    }

    /// Add a new follower partition
    pub async fn add_follower(
        &self,
        topic: &str,
        partition: PartitionId,
        leader_id: BrokerId,
    ) -> Result<()> {
        let follower_state = FollowerState::new(
            self.broker_id,
            topic.to_string(),
            partition,
            leader_id,
            self.config.clone(),
        );

        let follower_state_arc = Arc::new(follower_state);
        follower_state_arc
            .clone()
            .start_replication(Arc::clone(&self.storage))
            .await?;

        {
            let mut followers = self.followers.write().await;
            followers.insert((topic.to_string(), partition), follower_state_arc);
        }

        info!(
            "Added follower for {}:{} with leader {}",
            topic, partition, leader_id
        );
        Ok(())
    }

    /// Remove a follower partition
    pub async fn remove_follower(&self, topic: &str, partition: PartitionId) -> Result<()> {
        {
            let mut followers = self.followers.write().await;
            followers.remove(&(topic.to_string(), partition));
        }

        info!("Removed follower for {}:{}", topic, partition);
        Ok(())
    }

    /// Get follower state for a partition
    pub async fn get_follower(
        &self,
        topic: &str,
        partition: PartitionId,
    ) -> Option<Arc<FollowerState>> {
        let followers = self.followers.read().await;
        followers.get(&(topic.to_string(), partition)).cloned()
    }

    /// Synchronize with leader - in a full implementation, this would initiate catch-up
    pub async fn sync_with_leader(&self, topic: &str, partition: PartitionId) -> Result<()> {
        let follower_state = self.get_follower(topic, partition).await;

        if let Some(follower) = follower_state {
            let current_offset = self
                .storage
                .get_latest_offset(topic, partition)
                .unwrap_or(0);
            let last_applied = follower.get_last_applied().await;

            if current_offset != last_applied {
                info!(
                    "Synchronizing follower for {}:{}, current_offset: {}, last_applied: {}",
                    topic, partition, current_offset, last_applied
                );

                // In a full implementation, would request missing entries from leader
                // For now, just update the last_applied to match current storage
                {
                    let mut last_applied_guard = follower.last_applied.write().await;
                    *last_applied_guard = current_offset;
                }
            }
        }

        Ok(())
    }

    /// Check if we're caught up with all leaders
    pub async fn is_caught_up(&self) -> bool {
        let followers = self.followers.read().await;

        for ((_topic, _partition), follower) in followers.iter() {
            let commit_index = follower.get_commit_index().await;
            let last_applied = follower.get_last_applied().await;

            // If we're behind on any partition, we're not caught up
            if last_applied < commit_index {
                return false;
            }
        }

        true
    }

    /// Get status of all follower partitions
    pub async fn get_follower_status(
        &self,
    ) -> std::collections::HashMap<(TopicName, PartitionId), FollowerStatus> {
        let mut status = std::collections::HashMap::new();
        let followers = self.followers.read().await;

        for ((topic, partition), follower) in followers.iter() {
            let commit_index = follower.get_commit_index().await;
            let last_applied = follower.get_last_applied().await;
            let current_term = follower.get_current_term().await;

            status.insert(
                (topic.clone(), *partition),
                FollowerStatus {
                    leader_id: follower.leader_id,
                    current_term,
                    commit_index,
                    last_applied,
                    is_caught_up: last_applied >= commit_index,
                },
            );
        }

        status
    }
}

/// Status information for a follower partition
#[derive(Debug, Clone)]
pub struct FollowerStatus {
    pub leader_id: BrokerId,
    pub current_term: u64,
    pub commit_index: Offset,
    pub last_applied: Offset,
    pub is_caught_up: bool,
}