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
use crate::metrics::{CoreMetrics, Timer};
use crate::storage::{LogManager, TieredStorage};
use crate::{Config, Message, Result};
use bytes::Bytes;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;
use tokio::sync::RwLock;
use tracing::{debug, info, warn};
/// A single partition within a topic
#[derive(Debug)]
pub struct Partition {
/// Topic name (for tiered storage)
topic: String,
/// Partition ID
id: u32,
/// Storage Manager
log_manager: Arc<RwLock<LogManager>>,
/// Tiered storage (optional, for hot/warm/cold data tiering)
tiered_storage: Option<Arc<TieredStorage>>,
/// Current offset (next offset to be assigned)
/// Lock-free atomic for 5-10x throughput improvement
next_offset: AtomicU64,
/// Low watermark: records before this offset are logically deleted.
/// Set via `set_low_watermark()` (e.g., from DeleteRecords API).
low_watermark: AtomicU64,
}
impl Partition {
/// Create a new partition
pub async fn new(config: &Config, topic: &str, id: u32) -> Result<Self> {
Self::new_with_tiered_storage(config, topic, id, None).await
}
/// Create a new partition with optional tiered storage
pub async fn new_with_tiered_storage(
config: &Config,
topic: &str,
id: u32,
tiered_storage: Option<Arc<TieredStorage>>,
) -> Result<Self> {
info!(
"Creating partition {} for topic {} (tiered_storage: {})",
id,
topic,
tiered_storage.is_some()
);
let base_dir = std::path::PathBuf::from(&config.data_dir);
let log_manager = LogManager::new(base_dir, topic, id, config.max_segment_size).await?;
// Recover offset from storage
let recovered_offset = log_manager.recover_next_offset().await?;
let next_offset = AtomicU64::new(recovered_offset);
Ok(Self {
topic: topic.to_string(),
id,
log_manager: Arc::new(RwLock::new(log_manager)),
tiered_storage,
next_offset,
low_watermark: AtomicU64::new(0),
})
}
/// Get the partition ID
pub fn id(&self) -> u32 {
self.id
}
/// Get the topic name
pub fn topic(&self) -> &str {
&self.topic
}
/// Check if tiered storage is enabled
pub fn has_tiered_storage(&self) -> bool {
self.tiered_storage.is_some()
}
/// Append a message to the partition
/// Lock-free implementation using AtomicU64 for 5-10x throughput
pub async fn append(&self, mut message: Message) -> Result<u64> {
let timer = Timer::new();
// Acquire write lock first, then allocate offset to avoid gaps on failure
let mut log = self.log_manager.write().await;
// Lock-free offset allocation - single atomic operation
// Using AcqRel: ensures our write is visible to other threads (Release)
// and we see all previous writes (Acquire). SeqCst is unnecessary here.
let offset = self.next_offset.fetch_add(1, Ordering::AcqRel);
message.offset = offset;
// Pre-serialize for tiered storage BEFORE consuming the message,
// avoiding a full clone. Only serialize if tiered storage is enabled.
let tiered_bytes = if self.tiered_storage.is_some() {
match message.to_bytes() {
Ok(data) => Some(Bytes::from(data)),
Err(e) => {
warn!("Failed to serialize for tiered storage: {} (continuing)", e);
None
}
}
} else {
None
};
// Write to log manager (primary storage) — consumes message, no clone needed
if let Err(e) = log.append(offset, message).await {
// Reclaim the offset on failure to prevent gaps
let _ = self.next_offset.compare_exchange(
offset + 1,
offset,
Ordering::AcqRel,
Ordering::Relaxed,
);
return Err(e);
}
drop(log);
// Write pre-serialized bytes to tiered storage (no second serialization)
if let (Some(tiered), Some(data)) = (&self.tiered_storage, tiered_bytes) {
if let Err(e) = tiered
.write(&self.topic, self.id, offset, offset + 1, data)
.await
{
// Log warning but don't fail - log manager has the authoritative copy
warn!(
"Failed to write to tiered storage: {} (data safe in log)",
e
);
}
}
// Record metrics
CoreMetrics::increment_messages_appended();
CoreMetrics::record_append_latency_us(timer.elapsed_us());
debug!(
"Appended message at offset {} to partition {}",
offset, self.id
);
Ok(offset)
}
/// Read messages from a given offset
pub async fn read(&self, start_offset: u64, max_messages: usize) -> Result<Vec<Message>> {
let timer = Timer::new();
// Respect low watermark — don't serve records before it
let wm = self.low_watermark.load(Ordering::Acquire);
let effective_offset = start_offset.max(wm);
let log = self.log_manager.read().await;
// Estimate size: 4KB per message to be safe/generous for the 'max_bytes' parameter of log.read
let messages = log.read(effective_offset, max_messages * 4096).await?;
let result: Vec<Message> = messages.into_iter().take(max_messages).collect();
// Record metrics
CoreMetrics::add_messages_read(result.len() as u64);
CoreMetrics::record_read_latency_us(timer.elapsed_us());
debug!(
"Read {} messages from partition {} starting at offset {}",
result.len(),
self.id,
start_offset
);
Ok(result)
}
/// Get the latest offset
pub async fn latest_offset(&self) -> u64 {
self.next_offset.load(Ordering::Acquire)
}
pub async fn earliest_offset(&self) -> Option<u64> {
let log_earliest = {
let log = self.log_manager.read().await;
log.earliest_offset()
};
let wm = self.low_watermark.load(Ordering::Acquire);
Some(log_earliest.max(wm))
}
/// Set the low watermark for this partition.
///
/// Records before this offset are logically deleted and will not be
/// returned by `read()`. This implements the DeleteRecords API behavior.
/// The watermark can only advance forward (monotonically increasing).
///
/// Also triggers physical segment truncation: segments whose data is
/// entirely below the watermark are deleted from disk to reclaim space.
pub async fn set_low_watermark(&self, offset: u64) {
self.low_watermark.fetch_max(offset, Ordering::Release);
// Physically remove segments that are entirely below the watermark.
// This reclaims disk space — the logical watermark alone only hides
// records from consumers.
let mut log = self.log_manager.write().await;
match log.truncate_before(offset) {
Ok(0) => {}
Ok(n) => {
info!(
"Partition {}/{}: truncated {} segment(s) below watermark {}",
self.topic, self.id, n, offset
);
}
Err(e) => {
warn!(
"Partition {}/{}: segment truncation failed: {}",
self.topic, self.id, e
);
}
}
}
/// Get the current low watermark
pub fn low_watermark(&self) -> u64 {
self.low_watermark.load(Ordering::Acquire)
}
pub async fn message_count(&self) -> usize {
let earliest = self.earliest_offset().await.unwrap_or(0);
let next = self.next_offset.load(Ordering::SeqCst);
(next.saturating_sub(earliest)) as usize
}
/// Batch append multiple messages for 20-50x throughput improvement
/// Single fsync per batch instead of per-message
pub async fn append_batch(&self, messages: Vec<Message>) -> Result<Vec<u64>> {
if messages.is_empty() {
return Ok(Vec::new());
}
let timer = Timer::new();
let batch_size = messages.len();
// Allocate offsets atomically for entire batch
let start_offset = self
.next_offset
.fetch_add(batch_size as u64, Ordering::SeqCst);
let mut offsets = Vec::with_capacity(batch_size);
let mut batch_messages = Vec::with_capacity(batch_size);
let mut batch_data = Vec::new();
// Prepare messages with offsets
for (i, mut message) in messages.into_iter().enumerate() {
let offset = start_offset + i as u64;
message.offset = offset;
// Collect data for tiered storage
if self.tiered_storage.is_some() {
if let Ok(data) = message.to_bytes() {
batch_data.extend_from_slice(&data);
}
}
batch_messages.push((offset, message));
offsets.push(offset);
}
// Write to log manager using optimized batch append
{
let mut log = self.log_manager.write().await;
if let Err(e) = log.append_batch(batch_messages).await {
// Roll back atomically reserved offsets on failure.
// CAS ensures we only roll back if no concurrent append advanced
// past our reservation.
let _ = self.next_offset.compare_exchange(
start_offset + batch_size as u64,
start_offset,
Ordering::SeqCst,
Ordering::Relaxed,
);
return Err(e);
}
}
// Also write to tiered storage if enabled
if let Some(tiered) = &self.tiered_storage {
if !batch_data.is_empty() {
let end_offset = start_offset + batch_size as u64;
if let Err(e) = tiered
.write(
&self.topic,
self.id,
start_offset,
end_offset,
Bytes::from(batch_data),
)
.await
{
// Log warning but don't fail - log manager has the authoritative copy
warn!(
"Failed to write batch to tiered storage: {} (data safe in log)",
e
);
}
}
}
// Record metrics
CoreMetrics::increment_batch_appends();
CoreMetrics::add_messages_appended(batch_size as u64);
CoreMetrics::record_batch_append_latency_us(timer.elapsed_us());
debug!(
"Batch appended {} messages to partition {} (offsets {}-{})",
batch_size,
self.id,
start_offset,
start_offset + batch_size as u64 - 1
);
Ok(offsets)
}
/// Flush partition data to disk ensuring durability
pub async fn flush(&self) -> Result<()> {
let log = self.log_manager.read().await;
log.flush().await?;
// Also flush tiered storage hot tier if enabled
if let Some(tiered) = &self.tiered_storage {
tiered.flush_hot_tier(&self.topic, self.id).await?;
}
Ok(())
}
/// Find the first offset with timestamp >= target_timestamp (milliseconds since epoch)
/// Returns None if no matching offset is found.
pub async fn find_offset_for_timestamp(&self, target_timestamp: i64) -> Result<Option<u64>> {
let log = self.log_manager.read().await;
log.find_offset_for_timestamp(target_timestamp).await
}
/// Get tiered storage statistics for this partition
pub fn tiered_storage_stats(&self) -> Option<crate::storage::TieredStorageStatsSnapshot> {
self.tiered_storage.as_ref().map(|ts| ts.stats())
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::Config;
use bytes::Bytes;
use std::fs;
fn get_test_config() -> Config {
let config = Config {
data_dir: format!("/tmp/rivven-test-partition-{}", uuid::Uuid::new_v4()),
..Default::default()
};
let _ = fs::remove_dir_all(&config.data_dir);
config
}
#[tokio::test]
async fn test_partition_persistence() {
let config = get_test_config();
let topic = "test-topic";
let part_id = 0;
// 1. Create partition and write messages
{
let partition = Partition::new(&config, topic, part_id).await.unwrap();
partition
.append(Message::new(Bytes::from("msg1")))
.await
.unwrap();
partition
.append(Message::new(Bytes::from("msg2")))
.await
.unwrap();
let stored = partition.read(0, 10).await.unwrap();
assert_eq!(stored.len(), 2);
assert_eq!(stored[0].value, Bytes::from("msg1"));
assert_eq!(stored[1].value, Bytes::from("msg2"));
}
// 2. Re-open partition to test persistence and recovery
{
let partition = Partition::new(&config, topic, part_id).await.unwrap();
// Check next offset
assert_eq!(partition.latest_offset().await, 2);
// Read old messages
let stored = partition.read(0, 10).await.unwrap();
assert_eq!(stored.len(), 2);
assert_eq!(stored[0].value, Bytes::from("msg1"));
// Append new message
partition
.append(Message::new(Bytes::from("msg3")))
.await
.unwrap();
let stored = partition.read(0, 10).await.unwrap();
assert_eq!(stored.len(), 3);
assert_eq!(stored[2].value, Bytes::from("msg3"));
}
fs::remove_dir_all(&config.data_dir).unwrap();
}
#[tokio::test]
async fn test_find_offset_for_timestamp() {
let config = get_test_config();
let topic = "test-topic-ts";
let part_id = 0;
let partition = Partition::new(&config, topic, part_id).await.unwrap();
// Append some messages
for i in 0..5 {
let msg = Message::new(Bytes::from(format!("msg{}", i)));
partition.append(msg).await.unwrap();
// Small delay to ensure distinct timestamps
tokio::time::sleep(tokio::time::Duration::from_millis(10)).await;
}
// Read messages to get their timestamps
let messages = partition.read(0, 10).await.unwrap();
assert_eq!(messages.len(), 5);
// Get timestamp of the third message (offset 2)
let ts_msg2 = messages[2].timestamp.timestamp_millis();
// Find offset for that timestamp - should return offset 2
let found_offset = partition.find_offset_for_timestamp(ts_msg2).await.unwrap();
assert_eq!(
found_offset,
Some(2),
"Should find offset 2 for timestamp {}",
ts_msg2
);
// Find offset for timestamp before all messages - should return offset 0
let very_old_ts = ts_msg2 - 10000; // 10 seconds before
let found_offset = partition
.find_offset_for_timestamp(very_old_ts)
.await
.unwrap();
assert_eq!(
found_offset,
Some(0),
"Should find offset 0 for very old timestamp"
);
// Find offset for timestamp in the future - should return None
let future_ts = chrono::Utc::now().timestamp_millis() + 60000; // 1 minute in future
let found_offset = partition
.find_offset_for_timestamp(future_ts)
.await
.unwrap();
assert_eq!(
found_offset, None,
"Should return None for future timestamp"
);
fs::remove_dir_all(&config.data_dir).unwrap();
}
}