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
use super::segment::{Segment, SegmentSyncPolicy};
use crate::{Message, Result};
use std::collections::HashMap;
use std::fs;
use std::path::PathBuf;
use std::sync::Arc;
/// Maximum number of messages to return per read operation
const MAX_MESSAGES_PER_READ: usize = 10_000;
#[derive(Debug)]
pub struct LogManager {
dir: PathBuf,
segments: Vec<Segment>,
active_segment_index: usize,
max_segment_size: u64,
/// Fsync policy for all segments
sync_policy: SegmentSyncPolicy,
}
impl LogManager {
pub async fn new(
base_dir: PathBuf,
topic: &str,
partition: u32,
max_segment_size: u64,
) -> Result<Self> {
Self::with_sync_policy(
base_dir,
topic,
partition,
max_segment_size,
SegmentSyncPolicy::EveryNWrites(1),
)
.await
}
/// Create a new LogManager with an explicit segment sync policy.
pub async fn with_sync_policy(
base_dir: PathBuf,
topic: &str,
partition: u32,
max_segment_size: u64,
sync_policy: SegmentSyncPolicy,
) -> Result<Self> {
let dir = base_dir
.join(topic)
.join(format!("partition-{}", partition));
fs::create_dir_all(&dir)?;
let mut segments = Vec::new();
// Load existing segments
// Simply look for .log files and sort them
let mut paths: Vec<_> = fs::read_dir(&dir)?
.filter_map(|entry| entry.ok())
.map(|entry| entry.path())
.filter(|path| path.extension().is_some_and(|ext| ext == "log"))
.collect();
paths.sort();
if paths.is_empty() {
// Create initial segment with configured sync policy
segments.push(Segment::with_sync_policy(&dir, 0, sync_policy)?);
} else {
for path in paths {
// Handle malformed filenames gracefully (non-UTF8, missing stem, etc.)
let Some(stem) = path.file_stem() else {
tracing::warn!(path = ?path, "Skipping segment file with no stem");
continue;
};
let Some(filename) = stem.to_str() else {
tracing::warn!(path = ?path, "Skipping segment file with non-UTF8 name");
continue;
};
// Assumes filename is just the offset (e.g. "00000000000000000000")
if let Ok(base_offset) = filename.parse::<u64>() {
segments.push(Segment::with_sync_policy(&dir, base_offset, sync_policy)?);
} else {
tracing::warn!(filename = %filename, "Skipping segment file with unparseable offset");
}
}
}
if segments.is_empty() {
segments.push(Segment::with_sync_policy(&dir, 0, sync_policy)?);
}
let active_segment_index = segments.len() - 1;
Ok(Self {
dir,
segments,
active_segment_index,
max_segment_size,
sync_policy,
})
}
pub async fn append(&mut self, offset: u64, message: Message) -> Result<u64> {
let segment = &mut self.segments[self.active_segment_index];
// Check if we need to roll
if segment.size() >= self.max_segment_size {
// Flush current segment before rolling
segment.flush().await?;
let new_segment = Segment::with_sync_policy(&self.dir, offset, self.sync_policy)?;
self.segments.push(new_segment);
self.active_segment_index += 1;
}
let segment = &mut self.segments[self.active_segment_index];
segment.append(offset, message).await
}
/// Take the deferred sync handle from the active segment.
///
/// Call this while still holding the write lock, then release the lock
/// and run `sync_data()` on the returned file descriptor. This allows
/// concurrent readers to proceed during the slow fdatasync syscall.
pub fn take_pending_sync(&self) -> Option<Arc<std::fs::File>> {
self.segments[self.active_segment_index].take_pending_sync()
}
/// Batch append for high-throughput scenarios
/// Splits batches across segment boundaries when needed
pub async fn append_batch(&mut self, messages: Vec<(u64, Message)>) -> Result<Vec<u64>> {
if messages.is_empty() {
return Ok(Vec::new());
}
let mut all_positions = Vec::with_capacity(messages.len());
let mut remaining = messages;
while !remaining.is_empty() {
let segment = &mut self.segments[self.active_segment_index];
// Roll if current segment is already at capacity
if segment.size() >= self.max_segment_size {
segment.flush().await?;
let new_segment =
Segment::with_sync_policy(&self.dir, remaining[0].0, self.sync_policy)?;
self.segments.push(new_segment);
self.active_segment_index += 1;
}
let segment = &self.segments[self.active_segment_index];
let available = self.max_segment_size.saturating_sub(segment.size());
// Find how many messages fit in the remaining capacity
let mut accumulated = 0u64;
let mut split_at = 0;
for (_, m) in remaining.iter() {
// Accurate size estimation: value + key + headers + fixed overhead
// Fixed overhead (64 bytes): CRC(4) + length prefix(4) + offset(8) + timestamp(12) + postcard framing(~36)
let key_len = m.key.as_ref().map(|k| k.len()).unwrap_or(0);
let headers_len = m
.headers
.iter()
.map(|(k, v)| k.len() + v.len() + 8)
.sum::<usize>();
let msg_size = (m.value.len() + key_len + headers_len + 64) as u64;
if accumulated + msg_size > available && split_at > 0 {
break;
}
accumulated += msg_size;
split_at += 1;
// Always include at least one message to guarantee progress
if accumulated > available {
break;
}
}
// Split without cloning: drain the batch portion, keep the rest
let rest = remaining.split_off(split_at);
let batch = remaining;
let segment = &mut self.segments[self.active_segment_index];
let positions = segment.append_batch(batch).await?;
all_positions.extend(positions);
remaining = rest;
}
Ok(all_positions)
}
pub async fn read(&self, offset: u64, max_bytes: usize) -> Result<Vec<Message>> {
let mut messages = Vec::new();
let mut bytes_collected = 0;
// Optimized: Find first segment where base_offset <= offset
let start_segment_idx = self
.segments
.partition_point(|seg| seg.base_offset() <= offset)
.saturating_sub(1);
for segment in self.segments.iter().skip(start_segment_idx) {
// If we have enough data, stop
if bytes_collected >= max_bytes {
break;
}
// Read from segment
let batch = segment.read(offset, max_bytes - bytes_collected).await?;
for msg in batch {
if msg.offset < offset {
continue;
}
if messages.len() < MAX_MESSAGES_PER_READ && bytes_collected < max_bytes {
// Estimate size (header + key + val)
let size = 8 + msg.key.as_ref().map(|k| k.len()).unwrap_or(0) + msg.value.len();
bytes_collected += size;
messages.push(msg);
}
}
}
Ok(messages)
}
pub fn earliest_offset(&self) -> u64 {
self.segments.first().map(|s| s.base_offset()).unwrap_or(0)
}
pub async fn recover_next_offset(&mut self) -> Result<u64> {
if let Some(last_segment) = self.segments.last_mut() {
if let Some(last_offset) = last_segment.recover_last_offset().await? {
return Ok(last_offset + 1);
}
// If last segment is empty, check `base_offset` of it?
// Usually base_offset is the next offset of previous segment.
// But if it's completely empty (newly created), next offset is `base_offset`.
return Ok(last_segment.base_offset());
}
Ok(0)
}
/// Flush all segments to disk ensuring durability
pub async fn flush(&self) -> Result<()> {
for segment in &self.segments {
segment.flush().await?;
}
Ok(())
}
/// Physically remove segments whose data is entirely below the given offset.
///
/// A segment is eligible for deletion when the *next* segment's `base_offset`
/// is ≤ `watermark`, meaning every record in the segment is before the
/// watermark. The currently-active segment is never deleted.
///
/// Returns the number of segments removed.
pub fn truncate_before(&mut self, watermark: u64) -> crate::Result<usize> {
if self.segments.len() <= 1 {
return Ok(0);
}
// Find how many segments are fully below the watermark.
// A segment at index `i` is fully below if the *next* segment's
// base_offset <= watermark (all records in segment[i] < base[i+1] <= watermark).
let mut remove_count = 0;
for i in 0..self.segments.len().saturating_sub(1) {
let next_base = self.segments[i + 1].base_offset();
if next_base <= watermark {
remove_count = i + 1;
} else {
break;
}
}
if remove_count == 0 {
return Ok(0);
}
// Delete files for removed segments
for seg in self.segments.drain(..remove_count) {
if let Err(e) = seg.delete_files() {
tracing::warn!(
"Failed to delete segment files at offset {}: {}",
seg.base_offset(),
e
);
}
}
// Adjust active segment index
self.active_segment_index = self.active_segment_index.saturating_sub(remove_count);
Ok(remove_count)
}
/// Find the first offset with timestamp >= target_timestamp (milliseconds since epoch)
/// Scans through segments to find the earliest message matching the timestamp.
/// Returns None if no matching offset is found.
pub async fn find_offset_for_timestamp(&self, target_timestamp: i64) -> Result<Option<u64>> {
// Scan segments from oldest to newest
for segment in &self.segments {
// Check timestamp bounds to skip segments that are entirely before target
if let Some((_min_ts, max_ts)) = segment.timestamp_bounds().await? {
// If the entire segment is before our target, skip it
if max_ts < target_timestamp {
continue;
}
// If the segment might contain our target, search it
if let Some(offset) = segment.find_offset_for_timestamp(target_timestamp).await? {
return Ok(Some(offset));
}
}
}
Ok(None)
}
/// Key-based log compaction for sealed (non-active) segments.
///
/// Reads all messages from eligible segments, keeps only the latest value
/// per key (by highest offset), removes tombstones (empty value = deletion
/// marker), and rewrites compacted segments in-place.
///
/// Only sealed segments (not the active segment) are compacted. The active
/// segment is left untouched to avoid interfering with concurrent appends.
///
/// Returns the number of messages removed by compaction.
pub async fn compact(&mut self) -> Result<usize> {
// Need at least 2 segments (1 sealed + 1 active) to compact
if self.segments.len() <= 1 {
return Ok(0);
}
let sealed_count = self.segments.len() - 1; // exclude active segment
let mut total_removed = 0;
// Phase 1: Read all messages from sealed segments, build key→latest map
let mut key_latest: HashMap<Vec<u8>, (u64, usize)> = HashMap::new(); // key → (offset, segment_idx)
let mut all_messages: Vec<Vec<Message>> = Vec::with_capacity(sealed_count);
for seg_idx in 0..sealed_count {
let messages = self.segments[seg_idx]
.read(self.segments[seg_idx].base_offset(), usize::MAX)
.await?;
for msg in &messages {
if let Some(key) = &msg.key {
let key_bytes = key.to_vec();
// Later offsets always win (higher offset = more recent)
match key_latest.get(&key_bytes) {
Some(&(existing_offset, _)) if existing_offset >= msg.offset => {}
_ => {
key_latest.insert(key_bytes, (msg.offset, seg_idx));
}
}
}
}
all_messages.push(messages);
}
// Phase 2: For each sealed segment, filter to keep only latest-keyed
// messages and non-tombstones, then rewrite if any messages were removed
for (seg_idx, messages) in all_messages.iter().enumerate().take(sealed_count) {
let base_offset = self.segments[seg_idx].base_offset();
let compacted: Vec<&Message> = messages
.iter()
.filter(|msg| {
match &msg.key {
Some(key) => {
let key_bytes = key.to_vec();
// Keep if this is the latest value for this key
if let Some(&(latest_offset, _)) = key_latest.get(&key_bytes) {
if msg.offset != latest_offset {
return false; // superseded by later write
}
// Remove tombstones (empty value)
if msg.value.is_empty() {
return false;
}
true
} else {
true // shouldn't happen, but keep
}
}
None => true, // keyless messages are always kept
}
})
.collect();
let removed = messages.len() - compacted.len();
if removed == 0 {
continue; // no compaction needed for this segment
}
total_removed += removed;
// Write-to-temp-then-rename to prevent data loss on crash.
//
// Approach:
// 1. Create a temporary segment in a ".compacting" subdirectory
// 2. Write compacted messages + flush/fsync
// 3. Delete old segment files
// 4. Rename temp files into place
// 5. Replace in-memory segment
let tmp_dir = self.dir.join(".compacting");
std::fs::create_dir_all(&tmp_dir)?;
let mut tmp_segment =
Segment::with_sync_policy(&tmp_dir, base_offset, self.sync_policy)?;
let batch: Vec<(u64, Message)> = compacted
.into_iter()
.map(|m| (m.offset, m.clone()))
.collect();
if !batch.is_empty() {
tmp_segment.append_batch(batch).await?;
tmp_segment.flush().await?;
}
// Drop tmp_segment to close file handles before rename
let tmp_log = tmp_dir.join(format!("{:020}.log", base_offset));
let tmp_idx = tmp_dir.join(format!("{:020}.index", base_offset));
drop(tmp_segment);
// Crash-safe rename: rename temp files directly into final
// position (POSIX rename atomically replaces the destination),
// THEN delete the temp directory. No intermediate .bak/.old step
// is needed — if the rename succeeds the old data is already gone;
// if it fails (crash) the original files are still intact.
let final_log = self.dir.join(format!("{:020}.log", base_offset));
let final_idx = self.dir.join(format!("{:020}.index", base_offset));
// 1. Rename temp files into final position (atomic overwrite on POSIX)
if tmp_log.exists() {
std::fs::rename(&tmp_log, &final_log)?;
}
if tmp_idx.exists() {
std::fs::rename(&tmp_idx, &final_idx)?;
}
// 2. fsync parent directory to ensure renames are durable
{
let dir_file = std::fs::File::open(&self.dir)?;
dir_file.sync_all()?;
}
// 3. Clean up temp directory (best-effort)
let _ = std::fs::remove_dir(&tmp_dir);
// Re-open the final segment at its canonical path
let new_segment = Segment::with_sync_policy(&self.dir, base_offset, self.sync_policy)?;
self.segments[seg_idx] = new_segment;
tracing::debug!(segment_base = base_offset, removed, "Compacted segment");
}
if total_removed > 0 {
tracing::info!(
removed = total_removed,
segments = sealed_count,
"Log compaction complete"
);
}
Ok(total_removed)
}
}