1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-09-06 00:50:37 +02:00

add batch index for each memory buffer

This commit is contained in:
chrislu 2024-01-08 00:03:08 -08:00
parent d0d24f1e40
commit 49428a303b
14 changed files with 753 additions and 584 deletions

View file

@ -73,6 +73,7 @@ func (lc *LockClient) StartLock(key string, owner string) (lock *LiveLock) {
return nil return nil
}, func(err error) (shouldContinue bool) { }, func(err error) (shouldContinue bool) {
if err != nil { if err != nil {
glog.Warningf("create lock %s: %s", key, err)
time.Sleep(time.Second) time.Sleep(time.Second)
} }
return lock.renewToken == "" return lock.renewToken == ""

View file

@ -3,6 +3,7 @@ package filer
import ( import (
"context" "context"
"fmt" "fmt"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"io" "io"
"math" "math"
"regexp" "regexp"
@ -113,11 +114,10 @@ var (
VolumeNotFoundPattern = regexp.MustCompile(`volume \d+? not found`) VolumeNotFoundPattern = regexp.MustCompile(`volume \d+? not found`)
) )
func (f *Filer) ReadPersistedLogBuffer(startTime time.Time, stopTsNs int64, eachLogEntryFn func(logEntry *filer_pb.LogEntry) error) (lastTsNs int64, isDone bool, err error) { func (f *Filer) ReadPersistedLogBuffer(startPosition log_buffer.MessagePosition, stopTsNs int64, eachLogEntryFn func(logEntry *filer_pb.LogEntry) error) (lastTsNs int64, isDone bool, err error) {
startTime = startTime.UTC() startDate := fmt.Sprintf("%04d-%02d-%02d", startPosition.Year(), startPosition.Month(), startPosition.Day())
startDate := fmt.Sprintf("%04d-%02d-%02d", startTime.Year(), startTime.Month(), startTime.Day()) startHourMinute := fmt.Sprintf("%02d-%02d", startPosition.Hour(), startPosition.Minute())
startHourMinute := fmt.Sprintf("%02d-%02d", startTime.Hour(), startTime.Minute())
var stopDate, stopHourMinute string var stopDate, stopHourMinute string
if stopTsNs != 0 { if stopTsNs != 0 {
stopTime := time.Unix(0, stopTsNs+24*60*60*int64(time.Nanosecond)).UTC() stopTime := time.Unix(0, stopTsNs+24*60*60*int64(time.Nanosecond)).UTC()
@ -126,7 +126,7 @@ func (f *Filer) ReadPersistedLogBuffer(startTime time.Time, stopTsNs int64, each
} }
sizeBuf := make([]byte, 4) sizeBuf := make([]byte, 4)
startTsNs := startTime.UnixNano() startTsNs := startPosition.UnixNano()
dayEntries, _, listDayErr := f.ListDirectoryEntries(context.Background(), SystemLogDir, startDate, true, math.MaxInt32, "", "", "") dayEntries, _, listDayErr := f.ListDirectoryEntries(context.Background(), SystemLogDir, startDate, true, math.MaxInt32, "", "", "")
if listDayErr != nil { if listDayErr != nil {

View file

@ -1,7 +1,6 @@
package broker package broker
import ( import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
@ -41,7 +40,7 @@ func (b *MessageQueueBroker) PublisherToPubBalancer(stream mq_pb.SeaweedMessagin
} }
if receivedStats := req.GetStats(); receivedStats != nil { if receivedStats := req.GetStats(); receivedStats != nil {
b.Balancer.OnBrokerStatsUpdated(initMessage.Broker, brokerStats, receivedStats) b.Balancer.OnBrokerStatsUpdated(initMessage.Broker, brokerStats, receivedStats)
glog.V(4).Infof("received from %v: %+v", initMessage.Broker, receivedStats) // glog.V(4).Infof("received from %v: %+v", initMessage.Broker, receivedStats)
} }
} }

View file

@ -7,6 +7,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_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"
"time" "time"
) )
@ -38,12 +39,30 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
}() }()
ctx := stream.Context() ctx := stream.Context()
startTime := time.Now() var startPosition log_buffer.MessagePosition
if startTs := req.GetInit().GetPartitionOffset().GetTsNs(); startTs > 0 { var inMemoryOnly bool
startTime = time.Unix(0, startTs) if req.GetInit()!=nil && req.GetInit().GetPartitionOffset() != nil {
offset := req.GetInit().GetPartitionOffset()
if offset.StartTsNs != 0 {
startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2)
}
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
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, startTime, func() bool { localTopicPartition.Subscribe(clientName, startPosition, inMemoryOnly, func() bool {
if !isConnected { if !isConnected {
return false return false
} }
@ -51,7 +70,7 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
if sleepIntervalCount > 10 { if sleepIntervalCount > 10 {
sleepIntervalCount = 10 sleepIntervalCount = 10
} }
time.Sleep(time.Duration(sleepIntervalCount) * 2339 * time.Millisecond) time.Sleep(time.Duration(sleepIntervalCount) * 337 * time.Millisecond)
// Check if the client has disconnected by monitoring the context // Check if the client has disconnected by monitoring the context
select { select {

View file

@ -3,6 +3,7 @@ package main
import ( import (
"flag" "flag"
"fmt" "fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" "github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/credentials/insecure"
@ -40,13 +41,15 @@ func main() {
brokers := strings.Split(*seedBrokers, ",") brokers := strings.Split(*seedBrokers, ",")
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, processorConfig) subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, processorConfig)
counter := 0
subscriber.SetEachMessageFunc(func(key, value []byte) (bool, error) { subscriber.SetEachMessageFunc(func(key, value []byte) (bool, error) {
println(string(key), "=>", string(value)) counter++
println(string(key), "=>", string(value), counter)
return true, nil return true, nil
}) })
subscriber.SetCompletionFunc(func() { subscriber.SetCompletionFunc(func() {
println("done subscribing") glog.V(0).Infof("done recived %d messages", counter)
}) })
if err := subscriber.Subscribe(); err != nil { if err := subscriber.Subscribe(); err != nil {

View file

@ -114,7 +114,8 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s
RangeStart: partition.RangeStart, RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop, RangeStop: partition.RangeStop,
}, },
TsNs: sub.alreadyProcessedTsNs, StartTsNs: sub.alreadyProcessedTsNs,
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
}, },
Filter: sub.ContentConfig.Filter, Filter: sub.ContentConfig.Filter,
}, },

View file

@ -47,8 +47,20 @@ 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, startReadTime time.Time, onNoMessageFn func() bool, eachMessageFn OnEachMessageFn) { func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition, inMemoryOnly bool, onNoMessageFn func() bool, eachMessageFn OnEachMessageFn) {
p.logBuffer.LoopProcessLogData(clientName, startReadTime, 0, onNoMessageFn, eachMessageFn) p.logBuffer.LoopProcessLogData(clientName, startPosition, inMemoryOnly, 0, onNoMessageFn, eachMessageFn)
}
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 FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment) *LocalPartition { func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment) *LocalPartition {

View file

@ -73,9 +73,17 @@ message Offset {
repeated PartitionOffset partition_offsets = 2; repeated PartitionOffset partition_offsets = 2;
} }
enum PartitionOffsetStartType {
EARLIEST = 0;
EARLIEST_IN_MEMORY = 1;
LATEST = 2;
}
message PartitionOffset { message PartitionOffset {
Partition partition = 1; Partition partition = 1;
int64 ts_ns = 2; int64 start_ts_ns = 2;
int64 stop_ts_ns = 3;
PartitionOffsetStartType start_type = 4;
} }
////////////////////////////////////////////////// //////////////////////////////////////////////////

File diff suppressed because it is too large Load diff

View file

@ -32,7 +32,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
} }
defer fs.deleteClient("", clientName, req.ClientId, req.ClientEpoch) defer fs.deleteClient("", clientName, req.ClientId, req.ClientEpoch)
lastReadTime := time.Unix(0, req.SinceNs) lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
glog.V(0).Infof(" %v starts to subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime) glog.V(0).Infof(" %v starts to subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName) eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
@ -57,12 +57,12 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
} }
if processedTsNs != 0 { if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs) lastReadTime = log_buffer.NewMessagePosition(processedTsNs, -2)
} }
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, req.UntilNs, func() bool { lastReadTime, isDone, readInMemoryLogErr = fs.filer.MetaAggregator.MetaLogBuffer.LoopProcessLogData("aggMeta:"+clientName, lastReadTime, false, 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()
@ -113,7 +113,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
fs.deleteClient("local", clientName, req.ClientId, req.ClientEpoch) fs.deleteClient("local", clientName, req.ClientId, req.ClientEpoch)
}() }()
lastReadTime := time.Unix(0, req.SinceNs) lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
glog.V(0).Infof(" + %v local subscribe %s from %+v clientId:%d", clientName, req.PathPrefix, lastReadTime, req.ClientId) glog.V(0).Infof(" + %v local subscribe %s from %+v clientId:%d", clientName, req.PathPrefix, lastReadTime, req.ClientId)
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName) eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
@ -138,7 +138,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
} }
if processedTsNs != 0 { if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs) lastReadTime = log_buffer.NewMessagePosition(processedTsNs, -2)
} else { } else {
if readInMemoryLogErr == log_buffer.ResumeFromDiskError { if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
time.Sleep(1127 * time.Millisecond) time.Sleep(1127 * time.Millisecond)
@ -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, req.UntilNs, func() bool { lastReadTime, isDone, readInMemoryLogErr = fs.filer.LocalMetaLogBuffer.LoopProcessLogData("localMeta:"+clientName, lastReadTime, false, req.UntilNs, func() bool {
fs.listenersLock.Lock() fs.listenersLock.Lock()
fs.listenersCond.Wait() fs.listenersCond.Wait()
fs.listenersLock.Unlock() fs.listenersLock.Unlock()

View file

@ -26,6 +26,7 @@ type LogBuffer struct {
name string name string
prevBuffers *SealedBuffers prevBuffers *SealedBuffers
buf []byte buf []byte
batchIndex int64
idx []int idx []int
pos int pos int
startTime time.Time startTime time.Time
@ -58,17 +59,17 @@ func NewLogBuffer(name string, flushInterval time.Duration, flushFn func(startTi
return lb return lb
} }
func (m *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64) { func (logBuffer *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64) {
var toFlush *dataToFlush var toFlush *dataToFlush
m.Lock() logBuffer.Lock()
defer func() { defer func() {
m.Unlock() logBuffer.Unlock()
if toFlush != nil { if toFlush != nil {
m.flushChan <- toFlush logBuffer.flushChan <- toFlush
} }
if m.notifyFn != nil { if logBuffer.notifyFn != nil {
m.notifyFn() logBuffer.notifyFn()
} }
}() }()
@ -80,12 +81,12 @@ func (m *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64)
} else { } else {
ts = time.Unix(0, processingTsNs) ts = time.Unix(0, processingTsNs)
} }
if m.lastTsNs >= processingTsNs { if logBuffer.lastTsNs >= processingTsNs {
// this is unlikely to happen, but just in case // this is unlikely to happen, but just in case
processingTsNs = m.lastTsNs + 1 processingTsNs = logBuffer.lastTsNs + 1
ts = time.Unix(0, processingTsNs) ts = time.Unix(0, processingTsNs)
} }
m.lastTsNs = processingTsNs logBuffer.lastTsNs = processingTsNs
logEntry := &filer_pb.LogEntry{ logEntry := &filer_pb.LogEntry{
TsNs: processingTsNs, TsNs: processingTsNs,
PartitionKeyHash: util.HashToInt32(partitionKey), PartitionKeyHash: util.HashToInt32(partitionKey),
@ -96,108 +97,119 @@ func (m *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64)
size := len(logEntryData) size := len(logEntryData)
if m.pos == 0 { if logBuffer.pos == 0 {
m.startTime = ts logBuffer.startTime = ts
} }
if m.startTime.Add(m.flushInterval).Before(ts) || len(m.buf)-m.pos < size+4 { if logBuffer.startTime.Add(logBuffer.flushInterval).Before(ts) || len(logBuffer.buf)-logBuffer.pos < size+4 {
// glog.V(4).Infof("%s copyToFlush1 start time %v, ts %v, remaining %d bytes", m.name, m.startTime, ts, len(m.buf)-m.pos) glog.V(0).Infof("%s copyToFlush1 batch:%d start time %v, ts %v, remaining %d bytes", logBuffer.name, logBuffer.batchIndex, logBuffer.startTime, ts, len(logBuffer.buf)-logBuffer.pos)
toFlush = m.copyToFlush() toFlush = logBuffer.copyToFlush()
m.startTime = ts logBuffer.startTime = ts
if len(m.buf) < size+4 { if len(logBuffer.buf) < size+4 {
m.buf = make([]byte, 2*size+4) logBuffer.buf = make([]byte, 2*size+4)
} }
} }
m.stopTime = ts logBuffer.stopTime = ts
m.idx = append(m.idx, m.pos) logBuffer.idx = append(logBuffer.idx, logBuffer.pos)
util.Uint32toBytes(m.sizeBuf, uint32(size)) util.Uint32toBytes(logBuffer.sizeBuf, uint32(size))
copy(m.buf[m.pos:m.pos+4], m.sizeBuf) copy(logBuffer.buf[logBuffer.pos:logBuffer.pos+4], logBuffer.sizeBuf)
copy(m.buf[m.pos+4:m.pos+4+size], logEntryData) copy(logBuffer.buf[logBuffer.pos+4:logBuffer.pos+4+size], logEntryData)
m.pos += size + 4 logBuffer.pos += size + 4
// fmt.Printf("partitionKey %v entry size %d total %d count %d\n", string(partitionKey), size, m.pos, len(m.idx)) // fmt.Printf("partitionKey %v entry size %d total %d count %d\n", string(partitionKey), size, m.pos, len(m.idx))
} }
func (m *LogBuffer) IsStopping() bool { func (logBuffer *LogBuffer) IsStopping() bool {
return m.isStopping.Load() return logBuffer.isStopping.Load()
} }
func (m *LogBuffer) Shutdown() { func (logBuffer *LogBuffer) Shutdown() {
isAlreadyStopped := m.isStopping.Swap(true) isAlreadyStopped := logBuffer.isStopping.Swap(true)
if isAlreadyStopped { if isAlreadyStopped {
return return
} }
toFlush := m.copyToFlush() toFlush := logBuffer.copyToFlush()
m.flushChan <- toFlush logBuffer.flushChan <- toFlush
close(m.flushChan) close(logBuffer.flushChan)
} }
func (m *LogBuffer) loopFlush() { func (logBuffer *LogBuffer) loopFlush() {
for d := range m.flushChan { for d := range logBuffer.flushChan {
if d != nil { if d != nil {
// glog.V(4).Infof("%s flush [%v, %v] size %d", m.name, d.startTime, d.stopTime, len(d.data.Bytes())) // glog.V(4).Infof("%s flush [%v, %v] size %d", m.name, d.startTime, d.stopTime, len(d.data.Bytes()))
m.flushFn(d.startTime, d.stopTime, d.data.Bytes()) logBuffer.flushFn(d.startTime, d.stopTime, d.data.Bytes())
d.releaseMemory() d.releaseMemory()
// local logbuffer is different from aggregate logbuffer here // local logbuffer is different from aggregate logbuffer here
m.lastFlushTime = d.stopTime logBuffer.lastFlushTime = d.stopTime
} }
} }
} }
func (m *LogBuffer) loopInterval() { func (logBuffer *LogBuffer) loopInterval() {
for !m.IsStopping() { for !logBuffer.IsStopping() {
time.Sleep(m.flushInterval) time.Sleep(logBuffer.flushInterval)
if m.IsStopping() { if logBuffer.IsStopping() {
return return
} }
m.Lock() logBuffer.Lock()
toFlush := m.copyToFlush() toFlush := logBuffer.copyToFlush()
m.Unlock() logBuffer.Unlock()
if toFlush != nil { if toFlush != nil {
glog.V(0).Infof("%s flush [%v, %v] size %d", m.name, toFlush.startTime, toFlush.stopTime, len(toFlush.data.Bytes())) glog.V(0).Infof("%s flush [%v, %v] size %d", logBuffer.name, toFlush.startTime, toFlush.stopTime, len(toFlush.data.Bytes()))
m.flushChan <- toFlush logBuffer.flushChan <- toFlush
} else { } else {
// glog.V(0).Infof("%s no flush", m.name) // glog.V(0).Infof("%s no flush", m.name)
} }
} }
} }
func (m *LogBuffer) copyToFlush() *dataToFlush { func (logBuffer *LogBuffer) copyToFlush() *dataToFlush {
if m.pos > 0 { if logBuffer.pos > 0 {
// fmt.Printf("flush buffer %d pos %d empty space %d\n", len(m.buf), m.pos, len(m.buf)-m.pos) // fmt.Printf("flush buffer %d pos %d empty space %d\n", len(m.buf), m.pos, len(m.buf)-m.pos)
var d *dataToFlush var d *dataToFlush
if m.flushFn != nil { if logBuffer.flushFn != nil {
d = &dataToFlush{ d = &dataToFlush{
startTime: m.startTime, startTime: logBuffer.startTime,
stopTime: m.stopTime, stopTime: logBuffer.stopTime,
data: copiedBytes(m.buf[:m.pos]), data: copiedBytes(logBuffer.buf[:logBuffer.pos]),
} }
// glog.V(4).Infof("%s flushing [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime) // glog.V(4).Infof("%s flushing [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime)
} else { } else {
// glog.V(4).Infof("%s removed from memory [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime) // glog.V(4).Infof("%s removed from memory [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime)
m.lastFlushTime = m.stopTime logBuffer.lastFlushTime = logBuffer.stopTime
} }
m.buf = m.prevBuffers.SealBuffer(m.startTime, m.stopTime, m.buf, m.pos) logBuffer.buf = logBuffer.prevBuffers.SealBuffer(logBuffer.startTime, logBuffer.stopTime, logBuffer.buf, logBuffer.pos, logBuffer.batchIndex)
m.startTime = time.Unix(0, 0) logBuffer.startTime = time.Unix(0, 0)
m.stopTime = time.Unix(0, 0) logBuffer.stopTime = time.Unix(0, 0)
m.pos = 0 logBuffer.pos = 0
m.idx = m.idx[:0] logBuffer.idx = logBuffer.idx[:0]
logBuffer.batchIndex++
return d return d
} }
return nil return nil
} }
func (logBuffer *LogBuffer) GetEarliestTime() time.Time{
return logBuffer.startTime
}
func (logBuffer *LogBuffer) GetEarliestPosition() MessagePosition{
return MessagePosition{
Time: logBuffer.startTime,
BatchIndex: logBuffer.batchIndex,
}
}
func (d *dataToFlush) releaseMemory() { func (d *dataToFlush) releaseMemory() {
d.data.Reset() d.data.Reset()
bufferPool.Put(d.data) bufferPool.Put(d.data)
} }
func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Buffer, err error) { func (logBuffer *LogBuffer) ReadFromBuffer(lastReadPosition MessagePosition, inMemoryOnly bool) (bufferCopy *bytes.Buffer, batchIndex int64, err error) {
m.RLock() logBuffer.RLock()
defer m.RUnlock() defer logBuffer.RUnlock()
// Read from disk and memory // Read from disk and memory
// 1. read from disk, last time is = td // 1. read from disk, last time is = td
@ -209,52 +221,61 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
// if td < tm, case 2.3 // if td < tm, case 2.3
// read from disk again // read from disk again
var tsMemory time.Time var tsMemory time.Time
if !m.startTime.IsZero() { var tsBatchIndex int64
tsMemory = m.startTime if !logBuffer.startTime.IsZero() {
tsMemory = logBuffer.startTime
tsBatchIndex = logBuffer.batchIndex
} }
for _, prevBuf := range m.prevBuffers.buffers { for _, prevBuf := range logBuffer.prevBuffers.buffers {
if !prevBuf.startTime.IsZero() && prevBuf.startTime.Before(tsMemory) { if !prevBuf.startTime.IsZero() && prevBuf.startTime.Before(tsMemory) {
tsMemory = prevBuf.startTime tsMemory = prevBuf.startTime
tsBatchIndex = prevBuf.batchIndex
} }
} }
if tsMemory.IsZero() { // case 2.2 if tsMemory.IsZero() { // case 2.2
return nil, nil println("2.2 no data")
} else if lastReadTime.Before(tsMemory) { // case 2.3 return nil, -2,nil
if !m.lastFlushTime.IsZero() { } else if lastReadPosition.Before(tsMemory) && lastReadPosition.BatchIndex +1 < tsBatchIndex { // case 2.3
glog.V(0).Infof("resume with last flush time: %v", m.lastFlushTime) if inMemoryOnly {
return nil, ResumeFromDiskError 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
} }
} }
// the following is case 2.1 // the following is case 2.1
if lastReadTime.Equal(m.stopTime) { if lastReadPosition.Equal(logBuffer.stopTime) {
return nil, nil return nil, logBuffer.batchIndex, nil
} }
if lastReadTime.After(m.stopTime) { if lastReadPosition.After(logBuffer.stopTime) {
// glog.Fatalf("unexpected last read time %v, older than latest %v", lastReadTime, m.stopTime) // glog.Fatalf("unexpected last read time %v, older than latest %v", lastReadPosition, m.stopTime)
return nil, nil return nil, logBuffer.batchIndex, nil
} }
if lastReadTime.Before(m.startTime) { if lastReadPosition.Before(logBuffer.startTime) {
// println("checking ", lastReadTime.UnixNano()) // println("checking ", lastReadPosition.UnixNano())
for _, buf := range m.prevBuffers.buffers { for _, buf := range logBuffer.prevBuffers.buffers {
if buf.startTime.After(lastReadTime) { if buf.startTime.After(lastReadPosition.Time) {
// glog.V(4).Infof("%s return the %d sealed buffer %v", m.name, i, buf.startTime) // glog.V(4).Infof("%s return the %d sealed buffer %v", m.name, i, buf.startTime)
// println("return the", i, "th in memory", buf.startTime.UnixNano()) // println("return the", i, "th in memory", buf.startTime.UnixNano())
return copiedBytes(buf.buf[:buf.size]), nil return copiedBytes(buf.buf[:buf.size]), buf.batchIndex, nil
} }
if !buf.startTime.After(lastReadTime) && buf.stopTime.After(lastReadTime) { if !buf.startTime.After(lastReadPosition.Time) && buf.stopTime.After(lastReadPosition.Time) {
pos := buf.locateByTs(lastReadTime) pos := buf.locateByTs(lastReadPosition.Time)
// fmt.Printf("locate buffer[%d] pos %d\n", i, pos) // fmt.Printf("locate buffer[%d] pos %d\n", i, pos)
return copiedBytes(buf.buf[pos:buf.size]), nil return copiedBytes(buf.buf[pos:buf.size]), buf.batchIndex, nil
} }
} }
// glog.V(4).Infof("%s return the current buf %v", m.name, lastReadTime) // glog.V(4).Infof("%s return the current buf %v", m.name, lastReadPosition)
return copiedBytes(m.buf[:m.pos]), nil return copiedBytes(logBuffer.buf[:logBuffer.pos]), logBuffer.batchIndex,nil
} }
lastTs := lastReadTime.UnixNano() lastTs := lastReadPosition.UnixNano()
l, h := 0, len(m.idx)-1 l, h := 0, len(logBuffer.idx)-1
/* /*
for i, pos := range m.idx { for i, pos := range m.idx {
@ -272,18 +293,18 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
for l <= h { for l <= h {
mid := (l + h) / 2 mid := (l + h) / 2
pos := m.idx[mid] pos := logBuffer.idx[mid]
_, t := readTs(m.buf, pos) _, t := readTs(logBuffer.buf, pos)
if t <= lastTs { if t <= lastTs {
l = mid + 1 l = mid + 1
} else if lastTs < t { } else if lastTs < t {
var prevT int64 var prevT int64
if mid > 0 { if mid > 0 {
_, prevT = readTs(m.buf, m.idx[mid-1]) _, prevT = readTs(logBuffer.buf, logBuffer.idx[mid-1])
} }
if prevT <= lastTs { if prevT <= lastTs {
// fmt.Printf("found l=%d, m-1=%d(ts=%d), m=%d(ts=%d), h=%d [%d, %d) \n", l, mid-1, prevT, mid, t, h, pos, m.pos) // fmt.Printf("found l=%d, m-1=%d(ts=%d), m=%d(ts=%d), h=%d [%d, %d) \n", l, mid-1, prevT, mid, t, h, pos, m.pos)
return copiedBytes(m.buf[pos:m.pos]), nil return copiedBytes(logBuffer.buf[pos:logBuffer.pos]), logBuffer.batchIndex, nil
} }
h = mid h = mid
} }
@ -291,10 +312,11 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
} }
// FIXME: this could be that the buffer has been flushed already // FIXME: this could be that the buffer has been flushed already
return nil, nil println("Not sure why no data", lastReadPosition.BatchIndex, tsBatchIndex)
return nil, -2, nil
} }
func (m *LogBuffer) ReleaseMemory(b *bytes.Buffer) { func (logBuffer *LogBuffer) ReleaseMemory(b *bytes.Buffer) {
bufferPool.Put(b) bufferPool.Put(b)
} }

View file

@ -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, 0, func() bool { lastProcessedTime, isDone, err := lb.LoopProcessLogData("test", startTime, false, 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 {

View file

@ -17,11 +17,24 @@ var (
ResumeFromDiskError = fmt.Errorf("resumeFromDisk") ResumeFromDiskError = fmt.Errorf("resumeFromDisk")
) )
func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime time.Time, stopTsNs int64, type MessagePosition struct {
waitForDataFn func() bool, eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (lastReadTime time.Time, isDone bool, err error) { time.Time // this is the timestamp of the message
BatchIndex int64 // this is only used when the timestamp is not enough to identify the next message, when the timestamp is in the previous batch.
}
func NewMessagePosition(tsNs int64, batchIndex int64) MessagePosition {
return MessagePosition{
Time: time.Unix(0, tsNs).UTC(),
BatchIndex: batchIndex,
}
}
func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition MessagePosition, inMemoryOnly bool, stopTsNs int64,
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
lastReadTime = startReadTime var batchIndex int64
lastReadPosition = startPosition
defer func() { defer func() {
if bytesBuf != nil { if bytesBuf != nil {
logBuffer.ReleaseMemory(bytesBuf) logBuffer.ReleaseMemory(bytesBuf)
@ -33,13 +46,20 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
if bytesBuf != nil { if bytesBuf != nil {
logBuffer.ReleaseMemory(bytesBuf) logBuffer.ReleaseMemory(bytesBuf)
} }
bytesBuf, err = logBuffer.ReadFromBuffer(lastReadTime) bytesBuf, batchIndex, err = logBuffer.ReadFromBuffer(lastReadPosition, inMemoryOnly)
if err == ResumeFromDiskError { if err == ResumeFromDiskError {
time.Sleep(1127 * time.Millisecond) time.Sleep(1127 * time.Millisecond)
return lastReadTime, isDone, ResumeFromDiskError return lastReadPosition, isDone, ResumeFromDiskError
} }
// glog.V(4).Infof("%s ReadFromBuffer by %v", readerName, lastReadTime) readSize := 0
if bytesBuf != nil {
readSize = bytesBuf.Len()
}
glog.V(0).Infof("%s ReadFromBuffer at %v batch:%d, read size:%v batch:%d", readerName, lastReadPosition, lastReadPosition.BatchIndex, readSize, batchIndex)
if bytesBuf == nil { if bytesBuf == nil {
if batchIndex >= 0 {
lastReadPosition = NewMessagePosition(lastReadPosition.UnixNano(), batchIndex)
}
if stopTsNs != 0 { if stopTsNs != 0 {
isDone = true isDone = true
return return
@ -52,7 +72,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
} }
buf := bytesBuf.Bytes() buf := bytesBuf.Bytes()
// fmt.Printf("ReadFromBuffer %s by %v size %d\n", readerName, lastReadTime, len(buf)) // fmt.Printf("ReadFromBuffer %s by %v size %d\n", readerName, lastReadPosition, len(buf))
batchSize := 0 batchSize := 0
@ -61,7 +81,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
size := util.BytesToUint32(buf[pos : pos+4]) size := util.BytesToUint32(buf[pos : pos+4])
if pos+4+int(size) > len(buf) { if pos+4+int(size) > len(buf) {
err = ResumeError err = ResumeError
glog.Errorf("LoopProcessLogData: %s read buffer %v read %d [%d,%d) from [0,%d)", readerName, lastReadTime, batchSize, pos, pos+int(size)+4, len(buf)) glog.Errorf("LoopProcessLogData: %s read buffer %v read %d [%d,%d) from [0,%d)", readerName, lastReadPosition, batchSize, pos, pos+int(size)+4, len(buf))
return return
} }
entryData := buf[pos+4 : pos+4+int(size)] entryData := buf[pos+4 : pos+4+int(size)]
@ -76,7 +96,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
isDone = true isDone = true
return return
} }
lastReadTime = time.Unix(0, logEntry.TsNs) lastReadPosition = NewMessagePosition(logEntry.TsNs, batchIndex)
if err = eachLogDataFn(logEntry); err != nil { if err = eachLogDataFn(logEntry); err != nil {
return return
@ -87,7 +107,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
} }
// glog.V(4).Infof("%s sent messages ts[%+v,%+v] size %d\n", readerName, startReadTime, lastReadTime, batchSize) // glog.V(4).Infof("%s sent messages ts[%+v,%+v] size %d\n", readerName, startReadTime, lastReadPosition, batchSize)
} }
} }

View file

@ -10,6 +10,7 @@ type MemBuffer struct {
size int size int
startTime time.Time startTime time.Time
stopTime time.Time stopTime time.Time
batchIndex int64
} }
type SealedBuffers struct { type SealedBuffers struct {
@ -29,7 +30,7 @@ func newSealedBuffers(size int) *SealedBuffers {
return sbs return sbs
} }
func (sbs *SealedBuffers) SealBuffer(startTime, stopTime time.Time, buf []byte, pos int) (newBuf []byte) { func (sbs *SealedBuffers) SealBuffer(startTime, stopTime time.Time, buf []byte, pos int, batchIndex int64) (newBuf []byte) {
oldMemBuffer := sbs.buffers[0] oldMemBuffer := sbs.buffers[0]
size := len(sbs.buffers) size := len(sbs.buffers)
for i := 0; i < size-1; i++ { for i := 0; i < size-1; i++ {
@ -37,11 +38,13 @@ func (sbs *SealedBuffers) SealBuffer(startTime, stopTime time.Time, buf []byte,
sbs.buffers[i].size = sbs.buffers[i+1].size sbs.buffers[i].size = sbs.buffers[i+1].size
sbs.buffers[i].startTime = sbs.buffers[i+1].startTime sbs.buffers[i].startTime = sbs.buffers[i+1].startTime
sbs.buffers[i].stopTime = sbs.buffers[i+1].stopTime sbs.buffers[i].stopTime = sbs.buffers[i+1].stopTime
sbs.buffers[i].batchIndex = sbs.buffers[i+1].batchIndex
} }
sbs.buffers[size-1].buf = buf sbs.buffers[size-1].buf = buf
sbs.buffers[size-1].size = pos sbs.buffers[size-1].size = pos
sbs.buffers[size-1].startTime = startTime sbs.buffers[size-1].startTime = startTime
sbs.buffers[size-1].stopTime = stopTime sbs.buffers[size-1].stopTime = stopTime
sbs.buffers[size-1].batchIndex = batchIndex
return oldMemBuffer.buf return oldMemBuffer.buf
} }