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
|
package topic
import (
"context"
"fmt"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
type LocalPartition struct {
ListenersWaits int64
AckTsNs int64
// notifying clients
ListenersLock sync.Mutex
ListenersCond *sync.Cond
Partition
LogBuffer *log_buffer.LogBuffer
Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers
publishFolloweMeStream mq_pb.SeaweedMessaging_PublishFollowMeClient
followerGrpcConnection *grpc.ClientConn
Follower string
// Track last activity for idle cleanup
lastActivityTime atomic.Int64 // Unix nano timestamp
}
var TIME_FORMAT = "2006-01-02-15-04-05"
var PartitionGenerationFormat = "v2006-01-02-15-04-05"
func NewLocalPartition(partition Partition, logFlushInterval int, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
lp := &LocalPartition{
Partition: partition,
Publishers: NewLocalPartitionPublishers(),
Subscribers: NewLocalPartitionSubscribers(),
}
lp.ListenersCond = sync.NewCond(&lp.ListenersLock)
lp.lastActivityTime.Store(time.Now().UnixNano()) // Initialize with current time
// Ensure a minimum flush interval to prevent busy-loop when set to 0
// A flush interval of 0 would cause time.Sleep(0) creating a CPU-consuming busy loop
flushInterval := time.Duration(logFlushInterval) * time.Second
if flushInterval == 0 {
flushInterval = 1 * time.Second // Minimum 1 second to avoid busy-loop, allow near-immediate flushing
}
lp.LogBuffer = log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
flushInterval, logFlushFn, readFromDiskFn, func() {
if atomic.LoadInt64(&lp.ListenersWaits) > 0 {
lp.ListenersCond.Broadcast()
}
})
return lp
}
func (p *LocalPartition) Publish(message *mq_pb.DataMessage) error {
p.LogBuffer.AddToBuffer(message)
p.UpdateActivity() // Track publish activity for idle cleanup
// maybe send to the follower
if p.publishFolloweMeStream != nil {
// println("recv", string(message.Key), message.TsNs)
if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Data{
Data: message,
},
}); followErr != nil {
return fmt.Errorf("send to follower %s: %v", p.Follower, followErr)
}
} else {
atomic.StoreInt64(&p.AckTsNs, message.TsNs)
}
return nil
}
func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
var processedPosition log_buffer.MessagePosition
var readPersistedLogErr error
var readInMemoryLogErr error
var isDone bool
p.UpdateActivity() // Track subscribe activity for idle cleanup
// CRITICAL FIX: Use offset-based functions if startPosition is offset-based
// This allows reading historical data by offset, not just by timestamp
if startPosition.IsOffsetBased {
// Wrap eachMessageFn to match the signature expected by LoopProcessLogDataWithOffset
// Also update activity when messages are processed
eachMessageWithOffsetFn := func(logEntry *filer_pb.LogEntry, offset int64) (bool, error) {
p.UpdateActivity() // Track message read activity
return eachMessageFn(logEntry)
}
// Always attempt initial disk read for historical data
// This is fast if no data on disk, and ensures we don't miss old data
// The memory read loop below handles new data with instant notifications
glog.V(2).Infof("%s reading historical data from disk starting at offset %d", clientName, startPosition.Offset)
processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
if readPersistedLogErr != nil {
glog.V(2).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
return readPersistedLogErr
}
if isDone {
return nil
}
// Update position after reading from disk
if processedPosition.Time.UnixNano() != 0 || processedPosition.IsOffsetBased {
startPosition = processedPosition
}
// Step 2: Enter the main loop - read from in-memory buffer, occasionally checking disk
for {
// Read from in-memory buffer (this is the hot path - handles streaming data)
glog.V(4).Infof("SUBSCRIBE: Reading from in-memory buffer for %s at offset %d", clientName, startPosition.Offset)
processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogDataWithOffset(clientName, startPosition, 0, onNoMessageFn, eachMessageWithOffsetFn)
if isDone {
return nil
}
// Update position
// CRITICAL FIX: For offset-based reads, Time is zero, so check Offset instead
if processedPosition.Time.UnixNano() != 0 || processedPosition.IsOffsetBased {
startPosition = processedPosition
}
// If we get ResumeFromDiskError, it means data was flushed to disk
// Read from disk ONCE to catch up, then continue with in-memory buffer
if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
glog.V(4).Infof("SUBSCRIBE: ResumeFromDiskError - reading flushed data from disk for %s at offset %d", clientName, startPosition.Offset)
processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
if readPersistedLogErr != nil {
glog.V(2).Infof("%s read %v persisted log after flush: %v", clientName, p.Partition, readPersistedLogErr)
return readPersistedLogErr
}
if isDone {
return nil
}
// Update position and continue the loop (back to in-memory buffer)
// CRITICAL FIX: For offset-based reads, Time is zero, so check Offset instead
if processedPosition.Time.UnixNano() != 0 || processedPosition.IsOffsetBased {
startPosition = processedPosition
}
// Loop continues - back to reading from in-memory buffer
continue
}
// Any other error is a real error
if readInMemoryLogErr != nil {
glog.V(2).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
return readInMemoryLogErr
}
// If we get here with no error and not done, something is wrong
glog.V(1).Infof("SUBSCRIBE: Unexpected state for %s - no error but not done, continuing", clientName)
}
}
// Original timestamp-based subscription logic
for {
processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
if readPersistedLogErr != nil {
glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
return readPersistedLogErr
}
if isDone {
return nil
}
// CRITICAL FIX: For offset-based reads, Time is zero, so check Offset instead
if processedPosition.Time.UnixNano() != 0 || processedPosition.IsOffsetBased {
startPosition = processedPosition
}
processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
if isDone {
return nil
}
// CRITICAL FIX: For offset-based reads, Time is zero, so check Offset instead
if processedPosition.Time.UnixNano() != 0 || processedPosition.IsOffsetBased {
startPosition = processedPosition
}
if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
continue
}
if readInMemoryLogErr != nil {
glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
return readInMemoryLogErr
}
}
}
func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
return p.LogBuffer.GetEarliestTime()
}
func (p *LocalPartition) HasData() bool {
return !p.LogBuffer.GetEarliestTime().IsZero()
}
func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
return p.LogBuffer.GetEarliestPosition()
}
func (p *LocalPartition) closePublishers() {
p.Publishers.SignalShutdown()
}
func (p *LocalPartition) closeSubscribers() {
p.Subscribers.SignalShutdown()
}
func (p *LocalPartition) WaitUntilNoPublishers() {
for {
if p.Publishers.Size() == 0 {
return
}
time.Sleep(113 * time.Millisecond)
}
}
func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessageRequest_InitMessage, grpcDialOption grpc.DialOption) (err error) {
if p.publishFolloweMeStream != nil {
return nil
}
if initMessage.FollowerBroker == "" {
return nil
}
p.Follower = initMessage.FollowerBroker
ctx := context.Background()
p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.Follower, true, grpcDialOption)
if err != nil {
return fmt.Errorf("fail to dial %s: %v", p.Follower, err)
}
followerClient := mq_pb.NewSeaweedMessagingClient(p.followerGrpcConnection)
p.publishFolloweMeStream, err = followerClient.PublishFollowMe(ctx)
if err != nil {
return fmt.Errorf("fail to create publish client: %w", err)
}
if err = p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Init{
Init: &mq_pb.PublishFollowMeRequest_InitMessage{
Topic: initMessage.Topic,
Partition: initMessage.Partition,
},
},
}); err != nil {
return err
}
// start receiving ack from follower
go func() {
defer func() {
// println("stop receiving ack from follower")
}()
for {
ack, err := p.publishFolloweMeStream.Recv()
if err != nil {
e, _ := status.FromError(err)
if e.Code() == codes.Canceled {
glog.V(0).Infof("local partition %v follower %v stopped", p.Partition, p.Follower)
return
}
glog.Errorf("Receiving local partition %v follower %s ack: %v", p.Partition, p.Follower, err)
return
}
atomic.StoreInt64(&p.AckTsNs, ack.AckTsNs)
// println("recv ack", ack.AckTsNs)
}
}()
return nil
}
func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
if p.Publishers.Size() == 0 && p.Subscribers.Size() == 0 {
p.LogBuffer.ShutdownLogBuffer()
for !p.LogBuffer.IsAllFlushed() {
time.Sleep(113 * time.Millisecond)
}
if p.publishFolloweMeStream != nil {
// send close to the follower
if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Close{
Close: &mq_pb.PublishFollowMeRequest_CloseMessage{},
},
}); followErr != nil {
glog.Errorf("Error closing follower stream: %v", followErr)
}
glog.V(4).Infof("closing grpcConnection to follower")
p.followerGrpcConnection.Close()
p.publishFolloweMeStream = nil
p.Follower = ""
}
hasShutdown = true
}
glog.V(0).Infof("local partition %v Publisher:%d Subscriber:%d follower:%s shutdown %v", p.Partition, p.Publishers.Size(), p.Subscribers.Size(), p.Follower, hasShutdown)
return
}
// MaybeShutdownLocalPartitionForTopic is a topic-aware version that considers system topic retention
func (p *LocalPartition) MaybeShutdownLocalPartitionForTopic(topicName string) (hasShutdown bool) {
// For system topics like _schemas, be more conservative about shutdown
if isSystemTopic(topicName) {
glog.V(0).Infof("System topic %s - skipping aggressive shutdown for partition %v (Publishers:%d Subscribers:%d)",
topicName, p.Partition, p.Publishers.Size(), p.Subscribers.Size())
return false
}
// For regular topics, use the standard shutdown logic
return p.MaybeShutdownLocalPartition()
}
// isSystemTopic checks if a topic should have special retention behavior
func isSystemTopic(topicName string) bool {
systemTopics := []string{
"_schemas", // Schema Registry topic
"__consumer_offsets", // Kafka consumer offsets topic
"__transaction_state", // Kafka transaction state topic
}
for _, systemTopic := range systemTopics {
if topicName == systemTopic {
return true
}
}
// Also check for topics with system prefixes
return strings.HasPrefix(topicName, "_") || strings.HasPrefix(topicName, "__")
}
func (p *LocalPartition) Shutdown() {
p.closePublishers()
p.closeSubscribers()
p.LogBuffer.ShutdownLogBuffer()
glog.V(0).Infof("local partition %v shutting down", p.Partition)
}
func (p *LocalPartition) NotifyLogFlushed(flushTsNs int64) {
if p.publishFolloweMeStream != nil {
if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Flush{
Flush: &mq_pb.PublishFollowMeRequest_FlushMessage{
TsNs: flushTsNs,
},
},
}); followErr != nil {
glog.Errorf("send follower %s flush message: %v", p.Follower, followErr)
}
// println("notifying", p.Follower, "flushed at", flushTsNs)
}
}
// UpdateActivity updates the last activity timestamp for this partition
// Should be called whenever a publisher publishes or a subscriber reads
func (p *LocalPartition) UpdateActivity() {
p.lastActivityTime.Store(time.Now().UnixNano())
}
// IsIdle returns true if the partition has no publishers and no subscribers
func (p *LocalPartition) IsIdle() bool {
return p.Publishers.Size() == 0 && p.Subscribers.Size() == 0
}
// GetIdleDuration returns how long the partition has been idle
func (p *LocalPartition) GetIdleDuration() time.Duration {
lastActivity := p.lastActivityTime.Load()
return time.Since(time.Unix(0, lastActivity))
}
// ShouldCleanup returns true if the partition should be cleaned up
// A partition should be cleaned up if:
// 1. It has no publishers and no subscribers
// 2. It has been idle for longer than the idle timeout
func (p *LocalPartition) ShouldCleanup(idleTimeout time.Duration) bool {
if !p.IsIdle() {
return false
}
return p.GetIdleDuration() > idleTimeout
}
|