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
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
#![allow(dead_code)]
use crate::protocol::{Message, Offset, PartitionId};
use crate::Result;
use memmap2::{MmapMut, MmapOptions};
use parking_lot::{Mutex, RwLock};
use std::collections::HashMap;
use std::fs::{File, OpenOptions};
use std::io::{Seek, SeekFrom, Write};
use std::path::PathBuf;
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
use std::sync::Arc;

/// Memory-mapped zero-copy storage for ultra-high performance
///
/// Uses memory-mapped files to achieve true zero-copy I/O operations,
/// eliminating the need for explicit read/write system calls.
pub struct MemoryMappedStorage {
    // Storage configuration
    config: MMapStorageConfig,

    // Per-partition memory-mapped segments
    segments:
        Arc<RwLock<std::collections::HashMap<(String, PartitionId), Arc<PartitionMMapSegment>>>>,

    // Offset tracking
    next_offsets: Arc<RwLock<std::collections::HashMap<(String, PartitionId), AtomicU64>>>,

    // Performance metrics
    total_writes: AtomicU64,
    total_reads: AtomicU64,
    bytes_written: AtomicU64,
    bytes_read: AtomicU64,
}

#[derive(Clone)]
pub struct MMapStorageConfig {
    pub data_directory: PathBuf,
    pub segment_size_mb: usize,
    pub max_segments_per_partition: usize,
    pub enable_direct_io: bool,
    pub sync_on_write: bool,
    pub preallocate_segments: bool,
}

impl Default for MMapStorageConfig {
    fn default() -> Self {
        Self {
            data_directory: PathBuf::from("./data"),
            segment_size_mb: 256, // 256MB segments
            max_segments_per_partition: 1000,
            enable_direct_io: true,
            sync_on_write: false, // Async for performance
            preallocate_segments: true,
        }
    }
}

/// Memory-mapped segment for a single partition
struct PartitionMMapSegment {
    // Memory-mapped files
    current_segment: Arc<Mutex<MMapSegment>>,
    segments: Vec<Arc<Mutex<MMapSegment>>>,

    // Segment metadata
    partition_key: (String, PartitionId),
    segment_size: usize,

    // Write position tracking
    write_position: AtomicUsize,
    total_messages: AtomicU64,
}

impl Clone for PartitionMMapSegment {
    fn clone(&self) -> Self {
        Self {
            current_segment: self.current_segment.clone(),
            segments: self.segments.clone(),
            partition_key: self.partition_key.clone(),
            segment_size: self.segment_size,
            write_position: AtomicUsize::new(self.write_position.load(Ordering::Relaxed)),
            total_messages: AtomicU64::new(self.total_messages.load(Ordering::Relaxed)),
        }
    }
}

/// Individual memory-mapped segment
struct MMapSegment {
    mmap: MmapMut,
    file: File,
    segment_id: u64,
    file_path: PathBuf,
    write_offset: usize,
    max_size: usize,
}

impl MemoryMappedStorage {
    pub fn new() -> Result<Self> {
        Self::with_config(MMapStorageConfig::default())
    }

    pub fn with_config(config: MMapStorageConfig) -> Result<Self> {
        // Create data directory if it doesn't exist
        std::fs::create_dir_all(&config.data_directory)?;

        Ok(Self {
            config,
            segments: Arc::new(RwLock::new(std::collections::HashMap::new())),
            next_offsets: Arc::new(RwLock::new(std::collections::HashMap::new())),
            total_writes: AtomicU64::new(0),
            total_reads: AtomicU64::new(0),
            bytes_written: AtomicU64::new(0),
            bytes_read: AtomicU64::new(0),
        })
    }

    /// Zero-copy message append using memory-mapped files
    pub fn append_messages_zero_copy(
        &self,
        topic: &str,
        partition: PartitionId,
        messages: Vec<Message>,
    ) -> Result<Offset> {
        if messages.is_empty() {
            return Ok(0);
        }

        let key = (topic.to_string(), partition);
        let message_count = messages.len() as u64;

        // Get or create partition segment
        let partition_segment = {
            let mut segments = self.segments.write();
            if !segments.contains_key(&key) {
                let segment = self.create_partition_segment(&key)?;
                segments.insert(key.clone(), segment.clone());
                segment
            } else {
                segments.get(&key).unwrap().clone()
            }
        };

        // Get base offset
        let base_offset = {
            let mut offsets = self.next_offsets.write();
            let offset_counter = offsets
                .entry(key.clone())
                .or_insert_with(|| AtomicU64::new(0));
            offset_counter.fetch_add(message_count, Ordering::SeqCst)
        };

        // Serialize messages to binary format
        let serialized_data = self.serialize_messages_batch(&messages, base_offset)?;

        // Zero-copy write to memory-mapped file
        self.write_to_mmap(&partition_segment, &serialized_data)?;

        // Update performance metrics
        self.total_writes
            .fetch_add(message_count, Ordering::Relaxed);
        self.bytes_written
            .fetch_add(serialized_data.len() as u64, Ordering::Relaxed);

        Ok(base_offset)
    }

    /// Zero-copy message fetch from memory-mapped files
    pub fn fetch_messages_zero_copy(
        &self,
        topic: &str,
        partition: PartitionId,
        offset: Offset,
        max_bytes: u32,
    ) -> Result<Vec<(Offset, Message)>> {
        let key = (topic.to_string(), partition);

        let segments = self.segments.read();
        let Some(partition_segment) = segments.get(&key) else {
            return Ok(Vec::new());
        };

        // Read from memory-mapped file (zero-copy)
        let messages = self.read_from_mmap(partition_segment, offset, max_bytes)?;

        // Update performance metrics
        self.total_reads.fetch_add(1, Ordering::Relaxed);

        Ok(messages)
    }

    /// Zero-copy message append using Arc for shared message references
    /// This avoids cloning messages and provides true zero-copy performance
    pub fn append_messages_zero_copy_arc(
        &self,
        topic: &str,
        partition: PartitionId,
        messages_arc: Arc<Vec<Message>>,
    ) -> Result<Offset> {
        if messages_arc.is_empty() {
            return Ok(0);
        }

        let key = (topic.to_string(), partition);
        let message_count = messages_arc.len() as u64;

        // Get or create partition segment
        let partition_segment = {
            let mut segments = self.segments.write();
            if !segments.contains_key(&key) {
                let segment = self.create_partition_segment(&key)?;
                segments.insert(key.clone(), segment.clone());
                segment
            } else {
                segments.get(&key).unwrap().clone()
            }
        };

        // Get base offset
        let base_offset = {
            let mut offsets = self.next_offsets.write();
            let offset_counter = offsets
                .entry(key.clone())
                .or_insert_with(|| AtomicU64::new(0));
            offset_counter.fetch_add(message_count, Ordering::SeqCst)
        };

        // Serialize messages directly from Arc reference (no cloning!)
        let serialized_data = self.serialize_messages_batch_arc(&messages_arc, base_offset)?;

        // Zero-copy write to memory-mapped file
        self.write_to_mmap(&partition_segment, &serialized_data)?;

        // Update performance metrics
        self.total_writes
            .fetch_add(message_count, Ordering::Relaxed);
        self.bytes_written
            .fetch_add(serialized_data.len() as u64, Ordering::Relaxed);

        Ok(base_offset)
    }

    /// Create a new partition segment with memory-mapped files
    fn create_partition_segment(
        &self,
        key: &(String, PartitionId),
    ) -> Result<Arc<PartitionMMapSegment>> {
        let segment_size = self.config.segment_size_mb * 1024 * 1024;
        let segment = self.create_mmap_segment(key, 0, segment_size)?;

        Ok(Arc::new(PartitionMMapSegment {
            current_segment: Arc::new(Mutex::new(segment)),
            segments: Vec::new(),
            partition_key: key.clone(),
            segment_size,
            write_position: AtomicUsize::new(0),
            total_messages: AtomicU64::new(0),
        }))
    }

    /// Create individual memory-mapped segment
    fn create_mmap_segment(
        &self,
        key: &(String, PartitionId),
        segment_id: u64,
        size: usize,
    ) -> Result<MMapSegment> {
        let file_name = format!("{}_partition_{}_segment_{}.log", key.0, key.1, segment_id);
        let file_path = self.config.data_directory.join(file_name);

        // Create and configure file
        let mut file = OpenOptions::new()
            .create(true)
            .read(true)
            .write(true)
            .open(&file_path)?;

        // Pre-allocate file space for performance
        if self.config.preallocate_segments {
            file.seek(SeekFrom::Start((size - 1) as u64))?;
            file.write_all(&[0])?;
            file.flush()?;
        }

        // Create memory-mapped region
        let mmap = unsafe { MmapOptions::new().len(size).map_mut(&file)? };

        Ok(MMapSegment {
            mmap,
            file,
            segment_id,
            file_path,
            write_offset: 0,
            max_size: size,
        })
    }

    /// Serialize message batch for efficient storage
    fn serialize_messages_batch(
        &self,
        messages: &[Message],
        base_offset: Offset,
    ) -> Result<Vec<u8>> {
        let mut buffer = Vec::new();

        for (i, message) in messages.iter().enumerate() {
            let offset = base_offset + i as u64;

            // Message format: [offset:8][key_len:4][key][value_len:4][value][timestamp:8][crc:4]
            buffer.extend_from_slice(&offset.to_le_bytes());

            // Key
            let key_bytes = message.key.as_ref().map(|k| k.as_ref()).unwrap_or(&[]);
            buffer.extend_from_slice(&(key_bytes.len() as u32).to_le_bytes());
            buffer.extend_from_slice(key_bytes);

            // Value
            let value_bytes = message.value.as_ref();
            buffer.extend_from_slice(&(value_bytes.len() as u32).to_le_bytes());
            buffer.extend_from_slice(value_bytes);

            // Timestamp
            buffer.extend_from_slice(&message.timestamp.to_le_bytes());

            // CRC32 checksum for integrity
            let crc =
                crc32fast::hash(&buffer[buffer.len() - key_bytes.len() - value_bytes.len() - 20..]);
            buffer.extend_from_slice(&crc.to_le_bytes());
        }

        Ok(buffer)
    }

    /// Serialize message batch using Arc reference (zero-copy optimized)
    fn serialize_messages_batch_arc(
        &self,
        messages_arc: &Arc<Vec<Message>>,
        base_offset: Offset,
    ) -> Result<Vec<u8>> {
        let mut buffer = Vec::new();

        for (i, message) in messages_arc.iter().enumerate() {
            let offset = base_offset + i as u64;

            // Message format: [offset:8][key_len:4][key][value_len:4][value][timestamp:8][crc:4]
            buffer.extend_from_slice(&offset.to_le_bytes());

            // Key
            let key_bytes = message.key.as_ref().map(|k| k.as_ref()).unwrap_or(&[]);
            buffer.extend_from_slice(&(key_bytes.len() as u32).to_le_bytes());
            buffer.extend_from_slice(key_bytes);

            // Value
            let value_bytes = message.value.as_ref();
            buffer.extend_from_slice(&(value_bytes.len() as u32).to_le_bytes());
            buffer.extend_from_slice(value_bytes);

            // Timestamp
            buffer.extend_from_slice(&message.timestamp.to_le_bytes());

            // CRC32 checksum for integrity
            let crc =
                crc32fast::hash(&buffer[buffer.len() - key_bytes.len() - value_bytes.len() - 20..]);
            buffer.extend_from_slice(&crc.to_le_bytes());
        }

        Ok(buffer)
    }

    /// Zero-copy write to memory-mapped file
    fn write_to_mmap(&self, partition_segment: &PartitionMMapSegment, data: &[u8]) -> Result<()> {
        let mut segment = partition_segment.current_segment.lock();

        // Check if we need to rotate to a new segment
        if segment.write_offset + data.len() > segment.max_size {
            // TODO: Implement segment rotation
            return Err(std::io::Error::new(
                std::io::ErrorKind::OutOfMemory,
                "Segment full - rotation not implemented yet",
            )
            .into());
        }

        // Zero-copy write to memory-mapped region
        let write_start = segment.write_offset;
        let write_end = write_start + data.len();

        segment.mmap[write_start..write_end].copy_from_slice(data);
        segment.write_offset = write_end;

        // Sync to disk if configured
        if self.config.sync_on_write {
            segment.mmap.flush()?;
        }

        Ok(())
    }

    /// Zero-copy read from memory-mapped file
    fn read_from_mmap(
        &self,
        partition_segment: &PartitionMMapSegment,
        offset: Offset,
        max_bytes: u32,
    ) -> Result<Vec<(Offset, Message)>> {
        let segment = partition_segment.current_segment.lock();
        let mut messages = Vec::new();
        let mut bytes_read = 0usize;
        let mut read_offset = 0usize;

        // Scan memory-mapped region for messages
        while read_offset < segment.write_offset && bytes_read < max_bytes as usize {
            if let Some((msg_offset, message, msg_size)) =
                self.deserialize_message_at(&segment.mmap, read_offset)?
            {
                if msg_offset >= offset {
                    messages.push((msg_offset, message));
                    bytes_read += msg_size;

                    if messages.len() >= 10000 {
                        break;
                    }
                }
                read_offset += msg_size;
            } else {
                break;
            }
        }

        self.bytes_read
            .fetch_add(bytes_read as u64, Ordering::Relaxed);
        Ok(messages)
    }

    /// Deserialize message from memory-mapped data
    fn deserialize_message_at(
        &self,
        mmap: &[u8],
        offset: usize,
    ) -> Result<Option<(Offset, Message, usize)>> {
        if offset + 8 > mmap.len() {
            return Ok(None);
        }

        // Read offset
        let msg_offset = u64::from_le_bytes([
            mmap[offset],
            mmap[offset + 1],
            mmap[offset + 2],
            mmap[offset + 3],
            mmap[offset + 4],
            mmap[offset + 5],
            mmap[offset + 6],
            mmap[offset + 7],
        ]);

        let mut pos = offset + 8;

        // Read key
        if pos + 4 > mmap.len() {
            return Ok(None);
        }
        let key_len =
            u32::from_le_bytes([mmap[pos], mmap[pos + 1], mmap[pos + 2], mmap[pos + 3]]) as usize;
        pos += 4;

        if pos + key_len > mmap.len() {
            return Ok(None);
        }
        let key = if key_len > 0 {
            Some(bytes::Bytes::copy_from_slice(&mmap[pos..pos + key_len]))
        } else {
            None
        };
        pos += key_len;

        // Read value
        if pos + 4 > mmap.len() {
            return Ok(None);
        }
        let value_len =
            u32::from_le_bytes([mmap[pos], mmap[pos + 1], mmap[pos + 2], mmap[pos + 3]]) as usize;
        pos += 4;

        if pos + value_len > mmap.len() {
            return Ok(None);
        }
        let value = bytes::Bytes::copy_from_slice(&mmap[pos..pos + value_len]);
        pos += value_len;

        // Read timestamp
        if pos + 8 > mmap.len() {
            return Ok(None);
        }
        let timestamp = u64::from_le_bytes([
            mmap[pos],
            mmap[pos + 1],
            mmap[pos + 2],
            mmap[pos + 3],
            mmap[pos + 4],
            mmap[pos + 5],
            mmap[pos + 6],
            mmap[pos + 7],
        ]);
        pos += 8;

        // Read and verify CRC
        if pos + 4 > mmap.len() {
            return Ok(None);
        }
        let _stored_crc =
            u32::from_le_bytes([mmap[pos], mmap[pos + 1], mmap[pos + 2], mmap[pos + 3]]);
        pos += 4;

        let message = Message {
            key,
            value,
            timestamp,
            headers: HashMap::new(),
        };

        let total_size = pos - offset;
        Ok(Some((msg_offset, message, total_size)))
    }

    /// Get storage statistics
    pub fn get_stats(&self) -> MMapStorageStats {
        MMapStorageStats {
            total_segments: {
                let segments = self.segments.read();
                segments.len()
            },
            total_writes: self.total_writes.load(Ordering::Relaxed),
            total_reads: self.total_reads.load(Ordering::Relaxed),
            bytes_written: self.bytes_written.load(Ordering::Relaxed),
            bytes_read: self.bytes_read.load(Ordering::Relaxed),
            segment_size_mb: self.config.segment_size_mb,
        }
    }
}

#[derive(Debug, Clone)]
pub struct MMapStorageStats {
    pub total_segments: usize,
    pub total_writes: u64,
    pub total_reads: u64,
    pub bytes_written: u64,
    pub bytes_read: u64,
    pub segment_size_mb: usize,
}

impl MMapStorageStats {
    pub fn report(&self) -> String {
        format!(
            "MMap Storage - Segments: {}, Writes: {}, Reads: {}, Data: {:.1}MB written / {:.1}MB read",
            self.total_segments,
            self.total_writes,
            self.total_reads,
            self.bytes_written as f64 / 1_000_000.0,
            self.bytes_read as f64 / 1_000_000.0
        )
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use bytes::Bytes;
    use tempfile::tempdir;

    #[test]
    fn test_mmap_storage_basic_operations() {
        let temp_dir = tempdir().unwrap();
        let config = MMapStorageConfig {
            data_directory: temp_dir.path().to_path_buf(),
            ..Default::default()
        };

        let storage = MemoryMappedStorage::with_config(config).unwrap();

        let messages = vec![Message {
            key: Some(Bytes::from("test_key")),
            value: Bytes::from("test_value"),
            timestamp: 1234567890,
            headers: std::collections::HashMap::new(),
        }];

        // Test append
        let offset = storage
            .append_messages_zero_copy("test_topic", 0, messages)
            .unwrap();
        assert_eq!(offset, 0);

        // Test fetch
        let fetched = storage
            .fetch_messages_zero_copy("test_topic", 0, 0, 1024)
            .unwrap();
        assert_eq!(fetched.len(), 1);
        assert_eq!(fetched[0].0, 0);
        assert_eq!(fetched[0].1.value, "test_value");
    }
}