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.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.metaLogCollection = collection
|
||||||
f.metaLogReplication = replication
|
f.metaLogReplication = replication
|
||||||
|
|
||||||
|
|
|
@ -43,7 +43,7 @@ func NewMetaAggregator(filer *Filer, self pb.ServerAddress, grpcDialOption grpc.
|
||||||
peerStatues: make(map[pb.ServerAddress]int),
|
peerStatues: make(map[pb.ServerAddress]int),
|
||||||
}
|
}
|
||||||
t.ListenersCond = sync.NewCond(&t.ListenersLock)
|
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()
|
t.ListenersCond.Broadcast()
|
||||||
})
|
})
|
||||||
return t
|
return t
|
||||||
|
|
|
@ -8,10 +8,14 @@ import (
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
|
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"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/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
"google.golang.org/grpc/status"
|
"google.golang.org/grpc/status"
|
||||||
|
"google.golang.org/protobuf/proto"
|
||||||
|
"math"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -78,7 +82,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
|
||||||
|
|
||||||
// drain existing topic partition subscriptions
|
// drain existing topic partition subscriptions
|
||||||
for _, assignment := range request.BrokerPartitionAssignments {
|
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 {
|
if request.IsDraining {
|
||||||
// TODO drain existing topic partition subscriptions
|
// 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()
|
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 {
|
for {
|
||||||
if err := b.appendToFile(targetFile, buf); err != nil {
|
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)
|
glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition)
|
||||||
isConnected := true
|
isConnected := true
|
||||||
sleepIntervalCount := 0
|
sleepIntervalCount := 0
|
||||||
|
|
||||||
|
var counter int64
|
||||||
defer func() {
|
defer func() {
|
||||||
isConnected = false
|
isConnected = false
|
||||||
localTopicPartition.Subscribers.RemoveSubscriber(clientName)
|
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 startPosition log_buffer.MessagePosition
|
||||||
var inMemoryOnly bool
|
|
||||||
if req.GetInit()!=nil && req.GetInit().GetPartitionOffset() != nil {
|
if req.GetInit()!=nil && req.GetInit().GetPartitionOffset() != nil {
|
||||||
offset := req.GetInit().GetPartitionOffset()
|
offset := req.GetInit().GetPartitionOffset()
|
||||||
if offset.StartTsNs != 0 {
|
if offset.StartTsNs != 0 {
|
||||||
|
@ -69,19 +70,10 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
|
||||||
startPosition = log_buffer.NewMessagePosition(1, -2)
|
startPosition = log_buffer.NewMessagePosition(1, -2)
|
||||||
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
|
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
|
||||||
startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -2)
|
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 {
|
if !isConnected {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
@ -121,8 +113,8 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
|
||||||
glog.Errorf("Error sending setup response: %v", err)
|
glog.Errorf("Error sending setup response: %v", err)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
counter++
|
||||||
return nil
|
return nil
|
||||||
})
|
})
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -2,8 +2,8 @@ package topic
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"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/pb/mq_pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
||||||
"time"
|
"time"
|
||||||
|
@ -22,32 +22,54 @@ type LocalPartition struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
var TIME_FORMAT = "2006-01-02-15-04-05"
|
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{
|
return &LocalPartition{
|
||||||
Partition: partition,
|
Partition: partition,
|
||||||
isLeader: isLeader,
|
isLeader: isLeader,
|
||||||
FollowerBrokers: followerBrokers,
|
FollowerBrokers: followerBrokers,
|
||||||
logBuffer: log_buffer.NewLogBuffer(
|
logBuffer: log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
|
||||||
fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
|
2*time.Minute, logFlushFn, readFromDiskFn, func() {}),
|
||||||
2*time.Minute,
|
|
||||||
logFlushFn,
|
|
||||||
func() {
|
|
||||||
|
|
||||||
},
|
|
||||||
),
|
|
||||||
Publishers: NewLocalPartitionPublishers(),
|
Publishers: NewLocalPartitionPublishers(),
|
||||||
Subscribers: NewLocalPartitionSubscribers(),
|
Subscribers: NewLocalPartitionSubscribers(),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type OnEachMessageFn func(logEntry *filer_pb.LogEntry) error
|
|
||||||
|
|
||||||
func (p *LocalPartition) Publish(message *mq_pb.DataMessage) {
|
func (p *LocalPartition) Publish(message *mq_pb.DataMessage) {
|
||||||
p.logBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
|
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) {
|
func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
|
||||||
p.logBuffer.LoopProcessLogData(clientName, startPosition, inMemoryOnly, 0, onNoMessageFn, eachMessageFn)
|
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 {
|
func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
|
||||||
|
@ -62,13 +84,13 @@ func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.Message
|
||||||
return p.logBuffer.GetEarliestPosition()
|
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)
|
isLeader := assignment.LeaderBroker == string(self)
|
||||||
followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
|
followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
|
||||||
for i, followerBroker := range assignment.FollowerBrokers {
|
for i, followerBroker := range assignment.FollowerBrokers {
|
||||||
followers[i] = pb.ServerAddress(followerBroker)
|
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() {
|
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)
|
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.ListenersLock.Lock()
|
||||||
fs.filer.MetaAggregator.ListenersCond.Wait()
|
fs.filer.MetaAggregator.ListenersCond.Wait()
|
||||||
fs.filer.MetaAggregator.ListenersLock.Unlock()
|
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)
|
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.listenersLock.Lock()
|
||||||
fs.listenersCond.Wait()
|
fs.listenersCond.Wait()
|
||||||
fs.listenersLock.Unlock()
|
fs.listenersLock.Unlock()
|
||||||
|
|
|
@ -22,7 +22,9 @@ type dataToFlush struct {
|
||||||
data *bytes.Buffer
|
data *bytes.Buffer
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type EachLogEntryFuncType func(logEntry *filer_pb.LogEntry) error
|
||||||
type LogFlushFuncType func(startTime, stopTime time.Time, buf []byte)
|
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 {
|
type LogBuffer struct {
|
||||||
name string
|
name string
|
||||||
|
@ -36,15 +38,17 @@ type LogBuffer struct {
|
||||||
lastFlushTime time.Time
|
lastFlushTime time.Time
|
||||||
sizeBuf []byte
|
sizeBuf []byte
|
||||||
flushInterval time.Duration
|
flushInterval time.Duration
|
||||||
flushFn LogFlushFuncType
|
flushFn LogFlushFuncType
|
||||||
notifyFn func()
|
ReadFromDiskFn LogReadFromDiskFuncType
|
||||||
|
notifyFn func()
|
||||||
isStopping *atomic.Bool
|
isStopping *atomic.Bool
|
||||||
flushChan chan *dataToFlush
|
flushChan chan *dataToFlush
|
||||||
lastTsNs int64
|
lastTsNs int64
|
||||||
sync.RWMutex
|
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{
|
lb := &LogBuffer{
|
||||||
name: name,
|
name: name,
|
||||||
prevBuffers: newSealedBuffers(PreviousBufferCount),
|
prevBuffers: newSealedBuffers(PreviousBufferCount),
|
||||||
|
@ -52,6 +56,7 @@ func NewLogBuffer(name string, flushInterval time.Duration, flushFn LogFlushFunc
|
||||||
sizeBuf: make([]byte, 4),
|
sizeBuf: make([]byte, 4),
|
||||||
flushInterval: flushInterval,
|
flushInterval: flushInterval,
|
||||||
flushFn: flushFn,
|
flushFn: flushFn,
|
||||||
|
ReadFromDiskFn: readFromDiskFn,
|
||||||
notifyFn: notifyFn,
|
notifyFn: notifyFn,
|
||||||
flushChan: make(chan *dataToFlush, 256),
|
flushChan: make(chan *dataToFlush, 256),
|
||||||
isStopping: new(atomic.Bool),
|
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 {
|
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()
|
toFlush = logBuffer.copyToFlush()
|
||||||
logBuffer.startTime = ts
|
logBuffer.startTime = ts
|
||||||
if len(logBuffer.buf) < size+4 {
|
if len(logBuffer.buf) < size+4 {
|
||||||
|
@ -209,7 +214,7 @@ func (d *dataToFlush) releaseMemory() {
|
||||||
bufferPool.Put(d.data)
|
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()
|
logBuffer.RLock()
|
||||||
defer logBuffer.RUnlock()
|
defer logBuffer.RUnlock()
|
||||||
|
|
||||||
|
@ -238,11 +243,6 @@ func (logBuffer *LogBuffer) ReadFromBuffer(lastReadPosition MessagePosition, inM
|
||||||
println("2.2 no data")
|
println("2.2 no data")
|
||||||
return nil, -2,nil
|
return nil, -2,nil
|
||||||
} else if lastReadPosition.Before(tsMemory) && lastReadPosition.BatchIndex +1 < tsBatchIndex { // case 2.3
|
} 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() {
|
if !logBuffer.lastFlushTime.IsZero() {
|
||||||
glog.V(0).Infof("resume with last flush time: %v", logBuffer.lastFlushTime)
|
glog.V(0).Infof("resume with last flush time: %v", logBuffer.lastFlushTime)
|
||||||
return nil, -2, ResumeFromDiskError
|
return nil, -2, ResumeFromDiskError
|
||||||
|
|
|
@ -15,7 +15,7 @@ func TestNewLogBufferFirstBuffer(t *testing.T) {
|
||||||
flushInterval := time.Second
|
flushInterval := time.Second
|
||||||
lb := NewLogBuffer("test", flushInterval, func(startTime, stopTime time.Time, buf []byte) {
|
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))
|
fmt.Printf("flush from %v to %v %d bytes\n", startTime, stopTime, len(buf))
|
||||||
}, func() {
|
}, nil, func() {
|
||||||
})
|
})
|
||||||
|
|
||||||
startTime := time.Now()
|
startTime := time.Now()
|
||||||
|
@ -28,7 +28,7 @@ func TestNewLogBufferFirstBuffer(t *testing.T) {
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
go func() {
|
go func() {
|
||||||
defer wg.Done()
|
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
|
// stop if no more messages
|
||||||
return receivedMessageCount < messageCount
|
return receivedMessageCount < messageCount
|
||||||
}, func(logEntry *filer_pb.LogEntry) error {
|
}, 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) {
|
waitForDataFn func() bool, eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (lastReadPosition MessagePosition, isDone bool, err error) {
|
||||||
// loop through all messages
|
// loop through all messages
|
||||||
var bytesBuf *bytes.Buffer
|
var bytesBuf *bytes.Buffer
|
||||||
|
@ -48,7 +48,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition
|
||||||
if bytesBuf != nil {
|
if bytesBuf != nil {
|
||||||
logBuffer.ReleaseMemory(bytesBuf)
|
logBuffer.ReleaseMemory(bytesBuf)
|
||||||
}
|
}
|
||||||
bytesBuf, batchIndex, err = logBuffer.ReadFromBuffer(lastReadPosition, inMemoryOnly)
|
bytesBuf, batchIndex, err = logBuffer.ReadFromBuffer(lastReadPosition)
|
||||||
if err == ResumeFromDiskError {
|
if err == ResumeFromDiskError {
|
||||||
time.Sleep(1127 * time.Millisecond)
|
time.Sleep(1127 * time.Millisecond)
|
||||||
return lastReadPosition, isDone, ResumeFromDiskError
|
return lastReadPosition, isDone, ResumeFromDiskError
|
||||||
|
|
Loading…
Reference in a new issue