mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
read from disk if not in memory
This commit is contained in:
parent
49f7de9daa
commit
fa59a5d67e
|
@ -69,7 +69,7 @@ func NewFiler(masters pb.ServerDiscovery, grpcDialOption grpc.DialOption, filerH
|
|||
f.UniqueFilerId = -f.UniqueFilerId
|
||||
}
|
||||
|
||||
f.LocalMetaLogBuffer = log_buffer.NewLogBuffer("local", LogFlushInterval, f.logFlushFunc, notifyFn)
|
||||
f.LocalMetaLogBuffer = log_buffer.NewLogBuffer("local", LogFlushInterval, f.logFlushFunc, nil, notifyFn)
|
||||
f.metaLogCollection = collection
|
||||
f.metaLogReplication = replication
|
||||
|
||||
|
|
|
@ -43,7 +43,7 @@ func NewMetaAggregator(filer *Filer, self pb.ServerAddress, grpcDialOption grpc.
|
|||
peerStatues: make(map[pb.ServerAddress]int),
|
||||
}
|
||||
t.ListenersCond = sync.NewCond(&t.ListenersLock)
|
||||
t.MetaLogBuffer = log_buffer.NewLogBuffer("aggr", LogFlushInterval, nil, func() {
|
||||
t.MetaLogBuffer = log_buffer.NewLogBuffer("aggr", LogFlushInterval, nil, nil, func() {
|
||||
t.ListenersCond.Broadcast()
|
||||
})
|
||||
return t
|
||||
|
|
|
@ -8,10 +8,14 @@ import (
|
|||
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"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"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/protobuf/proto"
|
||||
"math"
|
||||
"time"
|
||||
)
|
||||
|
||||
|
@ -78,7 +82,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
|
|||
|
||||
// drain existing topic partition subscriptions
|
||||
for _, assignment := range request.BrokerPartitionAssignments {
|
||||
localPartition := topic.FromPbBrokerPartitionAssignment(self, assignment, b.genLogFlushFunc(request.Topic, assignment.Partition))
|
||||
localPartition := topic.FromPbBrokerPartitionAssignment(self, assignment, b.genLogFlushFunc(request.Topic, assignment.Partition), b.genLogOnDiskReadFunc(request.Topic, assignment.Partition))
|
||||
if request.IsDraining {
|
||||
// TODO drain existing topic partition subscriptions
|
||||
|
||||
|
@ -122,9 +126,10 @@ func (b *MessageQueueBroker) genLogFlushFunc(t *mq_pb.Topic, partition *mq_pb.Pa
|
|||
}
|
||||
|
||||
startTime, stopTime = startTime.UTC(), stopTime.UTC()
|
||||
fileName := startTime.Format(topic.TIME_FORMAT)
|
||||
|
||||
targetFile := fmt.Sprintf("%s/%s",partitionDir, fileName)
|
||||
targetFile := fmt.Sprintf("%s/%s",partitionDir, startTime.Format(topic.TIME_FORMAT))
|
||||
|
||||
// TODO append block with more metadata
|
||||
|
||||
for {
|
||||
if err := b.appendToFile(targetFile, buf); err != nil {
|
||||
|
@ -136,3 +141,126 @@ func (b *MessageQueueBroker) genLogFlushFunc(t *mq_pb.Topic, partition *mq_pb.Pa
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (b *MessageQueueBroker) genLogOnDiskReadFunc(t *mq_pb.Topic, partition *mq_pb.Partition) log_buffer.LogReadFromDiskFuncType {
|
||||
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
|
||||
partitionGeneration := time.Unix(0, partition.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
|
||||
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, partition.RangeStart, partition.RangeStop)
|
||||
|
||||
lookupFileIdFn := func(fileId string) (targetUrls []string, err error) {
|
||||
return b.MasterClient.LookupFileId(fileId)
|
||||
}
|
||||
|
||||
eachChunkFn := func (buf []byte, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
|
||||
for pos := 0; pos+4 < len(buf); {
|
||||
|
||||
size := util.BytesToUint32(buf[pos : pos+4])
|
||||
if pos+4+int(size) > len(buf) {
|
||||
err = fmt.Errorf("LogOnDiskReadFunc: read [%d,%d) from [0,%d)", pos, pos+int(size)+4, len(buf))
|
||||
return
|
||||
}
|
||||
entryData := buf[pos+4 : pos+4+int(size)]
|
||||
|
||||
logEntry := &filer_pb.LogEntry{}
|
||||
if err = proto.Unmarshal(entryData, logEntry); err != nil {
|
||||
pos += 4 + int(size)
|
||||
err = fmt.Errorf("unexpected unmarshal mq_pb.Message: %v", err)
|
||||
return
|
||||
}
|
||||
if logEntry.TsNs < starTsNs {
|
||||
pos += 4 + int(size)
|
||||
continue
|
||||
}
|
||||
if stopTsNs != 0 && logEntry.TsNs > stopTsNs {
|
||||
println("stopTsNs", stopTsNs, "logEntry.TsNs", logEntry.TsNs)
|
||||
return
|
||||
}
|
||||
|
||||
if err = eachLogEntryFn(logEntry); err != nil {
|
||||
err = fmt.Errorf("process log entry %v: %v", logEntry, err)
|
||||
return
|
||||
}
|
||||
|
||||
processedTsNs = logEntry.TsNs
|
||||
|
||||
pos += 4 + int(size)
|
||||
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
eachFileFn := func(entry *filer_pb.Entry, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
|
||||
if len(entry.Content) > 0 {
|
||||
glog.Warningf("this should not happen. unexpected content in %s/%s", partitionDir, entry.Name)
|
||||
return
|
||||
}
|
||||
var urlStrings []string
|
||||
for _, chunk := range entry.Chunks {
|
||||
if chunk.Size == 0 {
|
||||
continue
|
||||
}
|
||||
if chunk.IsChunkManifest{
|
||||
glog.Warningf("this should not happen. unexpected chunk manifest in %s/%s", partitionDir, entry.Name)
|
||||
return
|
||||
}
|
||||
urlStrings, err = lookupFileIdFn(chunk.FileId)
|
||||
if err != nil {
|
||||
err = fmt.Errorf("lookup %s: %v", chunk.FileId, err)
|
||||
return
|
||||
}
|
||||
if len(urlStrings) == 0 {
|
||||
err = fmt.Errorf("no url found for %s", chunk.FileId)
|
||||
return
|
||||
}
|
||||
|
||||
// try one of the urlString until util.Get(urlString) succeeds
|
||||
var processed bool
|
||||
for _, urlString := range urlStrings {
|
||||
// TODO optimization opportunity: reuse the buffer
|
||||
var data []byte
|
||||
if data, _, err = util.Get(urlString); err == nil {
|
||||
processed = true
|
||||
if processedTsNs, err = eachChunkFn(data, eachLogEntryFn, starTsNs, stopTsNs); err != nil {
|
||||
return
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
if !processed {
|
||||
err = fmt.Errorf("no data processed for %s %s", entry.Name, chunk.FileId)
|
||||
return
|
||||
}
|
||||
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
return func(startPosition log_buffer.MessagePosition, stopTsNs int64, eachLogEntryFn log_buffer.EachLogEntryFuncType) (lastReadPosition log_buffer.MessagePosition, isDone bool, err error) {
|
||||
startFileName := startPosition.UTC().Format(topic.TIME_FORMAT)
|
||||
startTsNs := startPosition.Time.UnixNano()
|
||||
stopTime := time.Unix(0, stopTsNs)
|
||||
var processedTsNs int64
|
||||
err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
|
||||
return filer_pb.SeaweedList(client, partitionDir, "", func(entry *filer_pb.Entry, isLast bool) error {
|
||||
if entry.IsDirectory {
|
||||
return nil
|
||||
}
|
||||
if stopTsNs!=0 && entry.Name > stopTime.UTC().Format(topic.TIME_FORMAT) {
|
||||
isDone = true
|
||||
return nil
|
||||
}
|
||||
if entry.Name < startPosition.UTC().Format(topic.TIME_FORMAT) {
|
||||
return nil
|
||||
}
|
||||
if processedTsNs, err = eachFileFn(entry, eachLogEntryFn, startTsNs, stopTsNs); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
||||
}, startFileName, true, math.MaxInt32)
|
||||
})
|
||||
lastReadPosition = log_buffer.NewMessagePosition(processedTsNs, -2)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,14 +52,15 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
|
|||
glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition)
|
||||
isConnected := true
|
||||
sleepIntervalCount := 0
|
||||
|
||||
var counter int64
|
||||
defer func() {
|
||||
isConnected = false
|
||||
localTopicPartition.Subscribers.RemoveSubscriber(clientName)
|
||||
glog.V(0).Infof("Subscriber %s on %v %v disconnected", clientName, t, partition)
|
||||
glog.V(0).Infof("Subscriber %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
|
||||
}()
|
||||
|
||||
var startPosition log_buffer.MessagePosition
|
||||
var inMemoryOnly bool
|
||||
if req.GetInit()!=nil && req.GetInit().GetPartitionOffset() != nil {
|
||||
offset := req.GetInit().GetPartitionOffset()
|
||||
if offset.StartTsNs != 0 {
|
||||
|
@ -69,19 +70,10 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
|
|||
startPosition = log_buffer.NewMessagePosition(1, -2)
|
||||
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
|
||||
startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -2)
|
||||
} else if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY {
|
||||
inMemoryOnly = true
|
||||
for !localTopicPartition.HasData() {
|
||||
time.Sleep(337 * time.Millisecond)
|
||||
}
|
||||
memPosition := localTopicPartition.GetEarliestInMemoryMessagePosition()
|
||||
if startPosition.Before(memPosition.Time) {
|
||||
startPosition = memPosition
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
localTopicPartition.Subscribe(clientName, startPosition, inMemoryOnly, func() bool {
|
||||
return localTopicPartition.Subscribe(clientName, startPosition, func() bool {
|
||||
if !isConnected {
|
||||
return false
|
||||
}
|
||||
|
@ -121,8 +113,8 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
|
|||
glog.Errorf("Error sending setup response: %v", err)
|
||||
return err
|
||||
}
|
||||
|
||||
counter++
|
||||
return nil
|
||||
})
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -2,8 +2,8 @@ package topic
|
|||
|
||||
import (
|
||||
"fmt"
|
||||
"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"
|
||||
"time"
|
||||
|
@ -22,32 +22,54 @@ type LocalPartition struct {
|
|||
}
|
||||
|
||||
var TIME_FORMAT = "2006-01-02-15-04-05"
|
||||
func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress, logFlushFn log_buffer.LogFlushFuncType) *LocalPartition {
|
||||
func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
|
||||
return &LocalPartition{
|
||||
Partition: partition,
|
||||
isLeader: isLeader,
|
||||
FollowerBrokers: followerBrokers,
|
||||
logBuffer: log_buffer.NewLogBuffer(
|
||||
fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
|
||||
2*time.Minute,
|
||||
logFlushFn,
|
||||
func() {
|
||||
|
||||
},
|
||||
),
|
||||
logBuffer: log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
|
||||
2*time.Minute, logFlushFn, readFromDiskFn, func() {}),
|
||||
Publishers: NewLocalPartitionPublishers(),
|
||||
Subscribers: NewLocalPartitionSubscribers(),
|
||||
}
|
||||
}
|
||||
|
||||
type OnEachMessageFn func(logEntry *filer_pb.LogEntry) error
|
||||
|
||||
func (p *LocalPartition) Publish(message *mq_pb.DataMessage) {
|
||||
p.logBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
|
||||
}
|
||||
|
||||
func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition, inMemoryOnly bool, onNoMessageFn func() bool, eachMessageFn OnEachMessageFn) {
|
||||
p.logBuffer.LoopProcessLogData(clientName, startPosition, inMemoryOnly, 0, onNoMessageFn, eachMessageFn)
|
||||
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
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
startPosition = processedPosition
|
||||
processedPosition, isDone, readInMemoryLogErr = p.logBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
|
||||
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
|
||||
}
|
||||
if isDone {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
|
||||
|
@ -62,13 +84,13 @@ func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.Message
|
|||
return p.logBuffer.GetEarliestPosition()
|
||||
}
|
||||
|
||||
func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment, logFlushFn log_buffer.LogFlushFuncType) *LocalPartition {
|
||||
func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
|
||||
isLeader := assignment.LeaderBroker == string(self)
|
||||
followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
|
||||
for i, followerBroker := range assignment.FollowerBrokers {
|
||||
followers[i] = pb.ServerAddress(followerBroker)
|
||||
}
|
||||
return NewLocalPartition(FromPbPartition(assignment.Partition), isLeader, followers, logFlushFn)
|
||||
return NewLocalPartition(FromPbPartition(assignment.Partition), isLeader, followers, logFlushFn, readFromDiskFn)
|
||||
}
|
||||
|
||||
func (p *LocalPartition) closePublishers() {
|
||||
|
|
|
@ -62,7 +62,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
|
|||
|
||||
glog.V(4).Infof("read in memory %v aggregated subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
|
||||
|
||||
lastReadTime, isDone, readInMemoryLogErr = fs.filer.MetaAggregator.MetaLogBuffer.LoopProcessLogData("aggMeta:"+clientName, lastReadTime, false, req.UntilNs, func() bool {
|
||||
lastReadTime, isDone, readInMemoryLogErr = fs.filer.MetaAggregator.MetaLogBuffer.LoopProcessLogData("aggMeta:"+clientName, lastReadTime, req.UntilNs, func() bool {
|
||||
fs.filer.MetaAggregator.ListenersLock.Lock()
|
||||
fs.filer.MetaAggregator.ListenersCond.Wait()
|
||||
fs.filer.MetaAggregator.ListenersLock.Unlock()
|
||||
|
@ -148,7 +148,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
|
|||
|
||||
glog.V(0).Infof("read in memory %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
|
||||
|
||||
lastReadTime, isDone, readInMemoryLogErr = fs.filer.LocalMetaLogBuffer.LoopProcessLogData("localMeta:"+clientName, lastReadTime, false, req.UntilNs, func() bool {
|
||||
lastReadTime, isDone, readInMemoryLogErr = fs.filer.LocalMetaLogBuffer.LoopProcessLogData("localMeta:"+clientName, lastReadTime, req.UntilNs, func() bool {
|
||||
fs.listenersLock.Lock()
|
||||
fs.listenersCond.Wait()
|
||||
fs.listenersLock.Unlock()
|
||||
|
|
|
@ -22,7 +22,9 @@ type dataToFlush struct {
|
|||
data *bytes.Buffer
|
||||
}
|
||||
|
||||
type EachLogEntryFuncType func(logEntry *filer_pb.LogEntry) error
|
||||
type LogFlushFuncType func(startTime, stopTime time.Time, buf []byte)
|
||||
type LogReadFromDiskFuncType func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error)
|
||||
|
||||
type LogBuffer struct {
|
||||
name string
|
||||
|
@ -37,6 +39,7 @@ type LogBuffer struct {
|
|||
sizeBuf []byte
|
||||
flushInterval time.Duration
|
||||
flushFn LogFlushFuncType
|
||||
ReadFromDiskFn LogReadFromDiskFuncType
|
||||
notifyFn func()
|
||||
isStopping *atomic.Bool
|
||||
flushChan chan *dataToFlush
|
||||
|
@ -44,7 +47,8 @@ type LogBuffer struct {
|
|||
sync.RWMutex
|
||||
}
|
||||
|
||||
func NewLogBuffer(name string, flushInterval time.Duration, flushFn LogFlushFuncType, notifyFn func()) *LogBuffer {
|
||||
func NewLogBuffer(name string, flushInterval time.Duration, flushFn LogFlushFuncType,
|
||||
readFromDiskFn LogReadFromDiskFuncType, notifyFn func()) *LogBuffer {
|
||||
lb := &LogBuffer{
|
||||
name: name,
|
||||
prevBuffers: newSealedBuffers(PreviousBufferCount),
|
||||
|
@ -52,6 +56,7 @@ func NewLogBuffer(name string, flushInterval time.Duration, flushFn LogFlushFunc
|
|||
sizeBuf: make([]byte, 4),
|
||||
flushInterval: flushInterval,
|
||||
flushFn: flushFn,
|
||||
ReadFromDiskFn: readFromDiskFn,
|
||||
notifyFn: notifyFn,
|
||||
flushChan: make(chan *dataToFlush, 256),
|
||||
isStopping: new(atomic.Bool),
|
||||
|
@ -104,7 +109,7 @@ func (logBuffer *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsN
|
|||
}
|
||||
|
||||
if logBuffer.startTime.Add(logBuffer.flushInterval).Before(ts) || len(logBuffer.buf)-logBuffer.pos < size+4 {
|
||||
glog.V(0).Infof("%s copyToFlush1 batch:%d count:%d start time %v, ts %v, remaining %d bytes", logBuffer.name, logBuffer.batchIndex, len(logBuffer.idx), logBuffer.startTime, ts, len(logBuffer.buf)-logBuffer.pos)
|
||||
// glog.V(0).Infof("%s copyToFlush1 batch:%d count:%d start time %v, ts %v, remaining %d bytes", logBuffer.name, logBuffer.batchIndex, len(logBuffer.idx), logBuffer.startTime, ts, len(logBuffer.buf)-logBuffer.pos)
|
||||
toFlush = logBuffer.copyToFlush()
|
||||
logBuffer.startTime = ts
|
||||
if len(logBuffer.buf) < size+4 {
|
||||
|
@ -209,7 +214,7 @@ func (d *dataToFlush) releaseMemory() {
|
|||
bufferPool.Put(d.data)
|
||||
}
|
||||
|
||||
func (logBuffer *LogBuffer) ReadFromBuffer(lastReadPosition MessagePosition, inMemoryOnly bool) (bufferCopy *bytes.Buffer, batchIndex int64, err error) {
|
||||
func (logBuffer *LogBuffer) ReadFromBuffer(lastReadPosition MessagePosition) (bufferCopy *bytes.Buffer, batchIndex int64, err error) {
|
||||
logBuffer.RLock()
|
||||
defer logBuffer.RUnlock()
|
||||
|
||||
|
@ -238,11 +243,6 @@ func (logBuffer *LogBuffer) ReadFromBuffer(lastReadPosition MessagePosition, inM
|
|||
println("2.2 no data")
|
||||
return nil, -2,nil
|
||||
} else if lastReadPosition.Before(tsMemory) && lastReadPosition.BatchIndex +1 < tsBatchIndex { // case 2.3
|
||||
if inMemoryOnly {
|
||||
println("2.3 no data", lastReadPosition.BatchIndex, tsBatchIndex)
|
||||
// FIXME: this is wrong: the data has been flushed to disk already
|
||||
return nil, tsBatchIndex,nil
|
||||
}
|
||||
if !logBuffer.lastFlushTime.IsZero() {
|
||||
glog.V(0).Infof("resume with last flush time: %v", logBuffer.lastFlushTime)
|
||||
return nil, -2, ResumeFromDiskError
|
||||
|
|
|
@ -15,7 +15,7 @@ func TestNewLogBufferFirstBuffer(t *testing.T) {
|
|||
flushInterval := time.Second
|
||||
lb := NewLogBuffer("test", flushInterval, func(startTime, stopTime time.Time, buf []byte) {
|
||||
fmt.Printf("flush from %v to %v %d bytes\n", startTime, stopTime, len(buf))
|
||||
}, func() {
|
||||
}, nil, func() {
|
||||
})
|
||||
|
||||
startTime := time.Now()
|
||||
|
@ -28,7 +28,7 @@ func TestNewLogBufferFirstBuffer(t *testing.T) {
|
|||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
lastProcessedTime, isDone, err := lb.LoopProcessLogData("test", startTime, false, 0, func() bool {
|
||||
lastProcessedTime, isDone, err := lb.LoopProcessLogData("test", startTime, 0, func() bool {
|
||||
// stop if no more messages
|
||||
return receivedMessageCount < messageCount
|
||||
}, func(logEntry *filer_pb.LogEntry) error {
|
||||
|
|
|
@ -29,7 +29,7 @@ func NewMessagePosition(tsNs int64, batchIndex int64) MessagePosition {
|
|||
}
|
||||
}
|
||||
|
||||
func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition MessagePosition, inMemoryOnly bool, stopTsNs int64,
|
||||
func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition MessagePosition, stopTsNs int64,
|
||||
waitForDataFn func() bool, eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (lastReadPosition MessagePosition, isDone bool, err error) {
|
||||
// loop through all messages
|
||||
var bytesBuf *bytes.Buffer
|
||||
|
@ -48,7 +48,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition
|
|||
if bytesBuf != nil {
|
||||
logBuffer.ReleaseMemory(bytesBuf)
|
||||
}
|
||||
bytesBuf, batchIndex, err = logBuffer.ReadFromBuffer(lastReadPosition, inMemoryOnly)
|
||||
bytesBuf, batchIndex, err = logBuffer.ReadFromBuffer(lastReadPosition)
|
||||
if err == ResumeFromDiskError {
|
||||
time.Sleep(1127 * time.Millisecond)
|
||||
return lastReadPosition, isDone, ResumeFromDiskError
|
||||
|
|
Loading…
Reference in a new issue