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
|
package offset
import (
"fmt"
"sync"
"time"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
// PartitionOffsetManager manages sequential offset assignment for a single partition
type PartitionOffsetManager struct {
mu sync.RWMutex
namespace string
topicName string
partition *schema_pb.Partition
nextOffset int64
// Checkpointing for recovery
lastCheckpoint int64
lastCheckpointedOffset int64
storage OffsetStorage
// Background checkpointing
stopCheckpoint chan struct{}
}
// OffsetStorage interface for persisting offset state
type OffsetStorage interface {
// SaveCheckpoint persists the current offset state for recovery
// Takes topic information along with partition to determine the correct storage location
SaveCheckpoint(namespace, topicName string, partition *schema_pb.Partition, offset int64) error
// LoadCheckpoint retrieves the last saved offset state
LoadCheckpoint(namespace, topicName string, partition *schema_pb.Partition) (int64, error)
// GetHighestOffset scans storage to find the highest assigned offset
GetHighestOffset(namespace, topicName string, partition *schema_pb.Partition) (int64, error)
}
// NewPartitionOffsetManager creates a new offset manager for a partition
func NewPartitionOffsetManager(namespace, topicName string, partition *schema_pb.Partition, storage OffsetStorage) (*PartitionOffsetManager, error) {
manager := &PartitionOffsetManager{
namespace: namespace,
topicName: topicName,
partition: partition,
storage: storage,
stopCheckpoint: make(chan struct{}),
}
// Recover offset state
if err := manager.recover(); err != nil {
return nil, fmt.Errorf("failed to recover offset state: %w", err)
}
// Start background checkpoint goroutine
go manager.runPeriodicCheckpoint()
return manager, nil
}
// Close stops the background checkpoint goroutine and performs a final checkpoint
func (m *PartitionOffsetManager) Close() error {
close(m.stopCheckpoint)
// Perform final checkpoint
m.mu.RLock()
currentOffset := m.nextOffset - 1 // Last assigned offset
lastCheckpointed := m.lastCheckpointedOffset
m.mu.RUnlock()
if currentOffset >= 0 && currentOffset > lastCheckpointed {
return m.storage.SaveCheckpoint(m.namespace, m.topicName, m.partition, currentOffset)
}
return nil
}
// AssignOffset assigns the next sequential offset
func (m *PartitionOffsetManager) AssignOffset() int64 {
m.mu.Lock()
offset := m.nextOffset
m.nextOffset++
m.mu.Unlock()
return offset
}
// AssignOffsets assigns a batch of sequential offsets
func (m *PartitionOffsetManager) AssignOffsets(count int64) (baseOffset int64, lastOffset int64) {
m.mu.Lock()
baseOffset = m.nextOffset
lastOffset = m.nextOffset + count - 1
m.nextOffset += count
m.mu.Unlock()
return baseOffset, lastOffset
}
// GetNextOffset returns the next offset that will be assigned
func (m *PartitionOffsetManager) GetNextOffset() int64 {
m.mu.RLock()
defer m.mu.RUnlock()
return m.nextOffset
}
// GetHighWaterMark returns the high water mark (next offset)
func (m *PartitionOffsetManager) GetHighWaterMark() int64 {
return m.GetNextOffset()
}
// recover restores offset state from storage
func (m *PartitionOffsetManager) recover() error {
var checkpointOffset int64 = -1
var highestOffset int64 = -1
// Try to load checkpoint
if offset, err := m.storage.LoadCheckpoint(m.namespace, m.topicName, m.partition); err == nil && offset >= 0 {
checkpointOffset = offset
}
// Try to scan storage for highest offset
if offset, err := m.storage.GetHighestOffset(m.namespace, m.topicName, m.partition); err == nil && offset >= 0 {
highestOffset = offset
}
// Use the higher of checkpoint or storage scan
if checkpointOffset >= 0 && highestOffset >= 0 {
if highestOffset > checkpointOffset {
m.nextOffset = highestOffset + 1
m.lastCheckpoint = highestOffset
m.lastCheckpointedOffset = highestOffset
} else {
m.nextOffset = checkpointOffset + 1
m.lastCheckpoint = checkpointOffset
m.lastCheckpointedOffset = checkpointOffset
}
} else if checkpointOffset >= 0 {
m.nextOffset = checkpointOffset + 1
m.lastCheckpoint = checkpointOffset
m.lastCheckpointedOffset = checkpointOffset
} else if highestOffset >= 0 {
m.nextOffset = highestOffset + 1
m.lastCheckpoint = highestOffset
m.lastCheckpointedOffset = highestOffset
} else {
// No data exists, start from 0
m.nextOffset = 0
m.lastCheckpoint = -1
m.lastCheckpointedOffset = -1
}
return nil
}
// runPeriodicCheckpoint runs in the background and checkpoints every 2 seconds if the offset changed
func (m *PartitionOffsetManager) runPeriodicCheckpoint() {
ticker := time.NewTicker(2 * time.Second)
defer ticker.Stop()
for {
select {
case <-ticker.C:
m.performCheckpointIfChanged()
case <-m.stopCheckpoint:
return
}
}
}
// performCheckpointIfChanged saves checkpoint only if offset has changed since last checkpoint
func (m *PartitionOffsetManager) performCheckpointIfChanged() {
m.mu.RLock()
currentOffset := m.nextOffset - 1 // Last assigned offset
lastCheckpointed := m.lastCheckpointedOffset
m.mu.RUnlock()
// Skip if no messages have been assigned, or no change since last checkpoint
if currentOffset < 0 || currentOffset == lastCheckpointed {
return
}
// Perform checkpoint
if err := m.storage.SaveCheckpoint(m.namespace, m.topicName, m.partition, currentOffset); err != nil {
// Log error but don't fail - checkpointing is for optimization
fmt.Printf("Failed to checkpoint offset %d for %s/%s: %v\n", currentOffset, m.namespace, m.topicName, err)
return
}
// Update last checkpointed offset
m.mu.Lock()
m.lastCheckpointedOffset = currentOffset
m.lastCheckpoint = currentOffset
m.mu.Unlock()
}
// PartitionOffsetRegistry manages offset managers for multiple partitions
type PartitionOffsetRegistry struct {
mu sync.RWMutex
managers map[string]*PartitionOffsetManager
storage OffsetStorage
}
// NewPartitionOffsetRegistry creates a new registry
func NewPartitionOffsetRegistry(storage OffsetStorage) *PartitionOffsetRegistry {
return &PartitionOffsetRegistry{
managers: make(map[string]*PartitionOffsetManager),
storage: storage,
}
}
// GetManager returns the offset manager for a partition, creating it if needed
func (r *PartitionOffsetRegistry) GetManager(namespace, topicName string, partition *schema_pb.Partition) (*PartitionOffsetManager, error) {
// CRITICAL FIX: Use TopicPartitionKey to ensure each topic has its own offset manager
key := TopicPartitionKey(namespace, topicName, partition)
r.mu.RLock()
manager, exists := r.managers[key]
r.mu.RUnlock()
if exists {
return manager, nil
}
// Create new manager
r.mu.Lock()
defer r.mu.Unlock()
// Double-check after acquiring write lock
if manager, exists := r.managers[key]; exists {
return manager, nil
}
manager, err := NewPartitionOffsetManager(namespace, topicName, partition, r.storage)
if err != nil {
return nil, err
}
r.managers[key] = manager
return manager, nil
}
// AssignOffset assigns an offset for the given partition
func (r *PartitionOffsetRegistry) AssignOffset(namespace, topicName string, partition *schema_pb.Partition) (int64, error) {
manager, err := r.GetManager(namespace, topicName, partition)
if err != nil {
return 0, err
}
assignedOffset := manager.AssignOffset()
return assignedOffset, nil
}
// AssignOffsets assigns a batch of offsets for the given partition
func (r *PartitionOffsetRegistry) AssignOffsets(namespace, topicName string, partition *schema_pb.Partition, count int64) (baseOffset, lastOffset int64, err error) {
manager, err := r.GetManager(namespace, topicName, partition)
if err != nil {
return 0, 0, err
}
baseOffset, lastOffset = manager.AssignOffsets(count)
return baseOffset, lastOffset, nil
}
// GetHighWaterMark returns the high water mark for a partition
func (r *PartitionOffsetRegistry) GetHighWaterMark(namespace, topicName string, partition *schema_pb.Partition) (int64, error) {
manager, err := r.GetManager(namespace, topicName, partition)
if err != nil {
return 0, err
}
return manager.GetHighWaterMark(), nil
}
// Close stops all partition managers and performs final checkpoints
func (r *PartitionOffsetRegistry) Close() error {
r.mu.Lock()
defer r.mu.Unlock()
var firstErr error
for _, manager := range r.managers {
if err := manager.Close(); err != nil && firstErr == nil {
firstErr = err
}
}
return firstErr
}
// TopicPartitionKey generates a unique key for a topic-partition combination
// This is the canonical key format used across the offset management system
func TopicPartitionKey(namespace, topicName string, partition *schema_pb.Partition) string {
return fmt.Sprintf("%s/%s/ring:%d:range:%d-%d",
namespace, topicName,
partition.RingSize, partition.RangeStart, partition.RangeStop)
}
// PartitionKey generates a unique key for a partition (without topic context)
// Note: UnixTimeNs is intentionally excluded from the key because it represents
// partition creation time, not partition identity. Using it would cause offset
// tracking to reset whenever a partition is recreated or looked up again.
// DEPRECATED: Use TopicPartitionKey for production code to avoid key collisions
func PartitionKey(partition *schema_pb.Partition) string {
return fmt.Sprintf("ring:%d:range:%d-%d",
partition.RingSize, partition.RangeStart, partition.RangeStop)
}
// partitionKey is the internal lowercase version for backward compatibility within this package
func partitionKey(partition *schema_pb.Partition) string {
return PartitionKey(partition)
}
// OffsetAssignment represents an assigned offset with metadata
type OffsetAssignment struct {
Offset int64
Timestamp int64
Partition *schema_pb.Partition
}
// BatchOffsetAssignment represents a batch of assigned offsets
type BatchOffsetAssignment struct {
BaseOffset int64
LastOffset int64
Count int64
Timestamp int64
Partition *schema_pb.Partition
}
// AssignmentResult contains the result of offset assignment
type AssignmentResult struct {
Assignment *OffsetAssignment
Batch *BatchOffsetAssignment
Error error
}
// OffsetAssigner provides high-level offset assignment operations
type OffsetAssigner struct {
registry *PartitionOffsetRegistry
}
// NewOffsetAssigner creates a new offset assigner
func NewOffsetAssigner(storage OffsetStorage) *OffsetAssigner {
return &OffsetAssigner{
registry: NewPartitionOffsetRegistry(storage),
}
}
// AssignSingleOffset assigns a single offset with timestamp
func (a *OffsetAssigner) AssignSingleOffset(namespace, topicName string, partition *schema_pb.Partition) *AssignmentResult {
offset, err := a.registry.AssignOffset(namespace, topicName, partition)
if err != nil {
return &AssignmentResult{Error: err}
}
return &AssignmentResult{
Assignment: &OffsetAssignment{
Offset: offset,
Timestamp: time.Now().UnixNano(),
Partition: partition,
},
}
}
// AssignBatchOffsets assigns a batch of offsets with timestamp
func (a *OffsetAssigner) AssignBatchOffsets(namespace, topicName string, partition *schema_pb.Partition, count int64) *AssignmentResult {
baseOffset, lastOffset, err := a.registry.AssignOffsets(namespace, topicName, partition, count)
if err != nil {
return &AssignmentResult{Error: err}
}
return &AssignmentResult{
Batch: &BatchOffsetAssignment{
BaseOffset: baseOffset,
LastOffset: lastOffset,
Count: count,
Timestamp: time.Now().UnixNano(),
Partition: partition,
},
}
}
// GetHighWaterMark returns the high water mark for a partition
func (a *OffsetAssigner) GetHighWaterMark(namespace, topicName string, partition *schema_pb.Partition) (int64, error) {
return a.registry.GetHighWaterMark(namespace, topicName, partition)
}
|