filer: ensure seamless meta data updates

This commit is contained in:
Chris Lu 2021-01-11 02:08:55 -08:00
parent 6e12a3a490
commit 394513f598
5 changed files with 69 additions and 41 deletions

View file

@ -170,7 +170,7 @@ func ReadEachLogEntry(r io.Reader, sizeBuf []byte, ns int64, eachLogEntryFn func
return lastTsNs, err
}
if logEntry.TsNs <= ns {
return lastTsNs, nil
continue
}
// println("each log: ", logEntry.TsNs)
if err := eachLogEntryFn(logEntry); err != nil {

View file

@ -101,20 +101,21 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
return nil
}
if err = broker.readPersistedLogBuffer(&tp, lastReadTime, eachLogEntryFn); err != nil {
if err != io.EOF {
// println("stopping from persisted logs", err.Error())
return err
}
}
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
}
// fmt.Printf("subscriber %s read %d on disk log %v\n", subscriberId, messageCount, lastReadTime)
for {
if err = broker.readPersistedLogBuffer(&tp, lastReadTime, eachLogEntryFn); err != nil {
if err != io.EOF {
// println("stopping from persisted logs", err.Error())
return err
}
}
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
}
lastReadTime, err = lock.logBuffer.LoopProcessLogData(lastReadTime, func() bool {
lock.Mutex.Lock()
lock.cond.Wait()
@ -122,6 +123,9 @@ func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_Subs
return isConnected
}, eachLogEntryFn)
if err != nil {
if err == log_buffer.ResumeFromDiskError {
continue
}
glog.Errorf("processed to %v: %v", lastReadTime, err)
time.Sleep(3127 * time.Millisecond)
if err != log_buffer.ResumeError {

View file

@ -29,16 +29,20 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
eachLogEntryFn := eachLogEntryFn(eachEventNotificationFn)
processedTsNs, err := fs.filer.ReadPersistedLogBuffer(lastReadTime, eachLogEntryFn)
if err != nil {
return fmt.Errorf("reading from persisted logs: %v", err)
}
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
}
var processedTsNs int64
var err error
for {
processedTsNs, err = fs.filer.ReadPersistedLogBuffer(lastReadTime, eachLogEntryFn)
if err != nil {
return fmt.Errorf("reading from persisted logs: %v", err)
}
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
}
lastReadTime, err = fs.filer.MetaAggregator.MetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
fs.filer.MetaAggregator.ListenersLock.Lock()
fs.filer.MetaAggregator.ListenersCond.Wait()
@ -46,6 +50,9 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
return true
}, eachLogEntryFn)
if err != nil {
if err == log_buffer.ResumeFromDiskError {
continue
}
glog.Errorf("processed to %v: %v", lastReadTime, err)
time.Sleep(3127 * time.Millisecond)
if err != log_buffer.ResumeError {
@ -73,19 +80,23 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
eachLogEntryFn := eachLogEntryFn(eachEventNotificationFn)
// println("reading from persisted logs ...")
processedTsNs, err := fs.filer.ReadPersistedLogBuffer(lastReadTime, eachLogEntryFn)
if err != nil {
return fmt.Errorf("reading from persisted logs: %v", err)
}
var processedTsNs int64
var err error
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
}
glog.V(0).Infof("after local log reads, %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
// println("reading from in memory logs ...")
for {
// println("reading from persisted logs ...")
processedTsNs, err = fs.filer.ReadPersistedLogBuffer(lastReadTime, eachLogEntryFn)
if err != nil {
return fmt.Errorf("reading from persisted logs: %v", err)
}
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
}
// glog.V(0).Infof("after local log reads, %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
// println("reading from in memory logs ...")
lastReadTime, err = fs.filer.LocalMetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
fs.listenersLock.Lock()
fs.listenersCond.Wait()
@ -93,6 +104,9 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
return true
}, eachLogEntryFn)
if err != nil {
if err == log_buffer.ResumeFromDiskError {
continue
}
glog.Errorf("processed to %v: %v", lastReadTime, err)
time.Sleep(3127 * time.Millisecond)
if err != log_buffer.ResumeError {

View file

@ -28,6 +28,7 @@ type LogBuffer struct {
pos int
startTime time.Time
stopTime time.Time
lastFlushTime time.Time
sizeBuf []byte
flushInterval time.Duration
flushFn func(startTime, stopTime time.Time, buf []byte)
@ -129,6 +130,7 @@ func (m *LogBuffer) loopFlush() {
// fmt.Printf("flush [%v, %v] size %d\n", d.startTime, d.stopTime, len(d.data.Bytes()))
m.flushFn(d.startTime, d.stopTime, d.data.Bytes())
d.releaseMemory()
m.lastFlushTime = d.stopTime
}
}
}
@ -174,10 +176,14 @@ func (d *dataToFlush) releaseMemory() {
bufferPool.Put(d.data)
}
func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Buffer) {
func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Buffer, err error) {
m.RLock()
defer m.RUnlock()
if !m.lastFlushTime.IsZero() && m.lastFlushTime.After(lastReadTime) {
return nil, ResumeFromDiskError
}
/*
fmt.Printf("read buffer %p: %v last stop time: [%v,%v], pos %d, entries:%d, prevBufs:%d\n", m, lastReadTime, m.startTime, m.stopTime, m.pos, len(m.idx), len(m.prevBuffers.buffers))
for i, prevBuf := range m.prevBuffers.buffers {
@ -186,11 +192,11 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
*/
if lastReadTime.Equal(m.stopTime) {
return nil
return nil, nil
}
if lastReadTime.After(m.stopTime) {
// glog.Fatalf("unexpected last read time %v, older than latest %v", lastReadTime, m.stopTime)
return nil
return nil, nil
}
if lastReadTime.Before(m.startTime) {
// println("checking ", lastReadTime.UnixNano())
@ -198,19 +204,19 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
if buf.startTime.After(lastReadTime) {
if i == 0 {
// println("return the earliest in memory", buf.startTime.UnixNano())
return copiedBytes(buf.buf[:buf.size])
return copiedBytes(buf.buf[:buf.size]), nil
}
// println("return the", i, "th in memory", buf.startTime.UnixNano())
return copiedBytes(buf.buf[:buf.size])
return copiedBytes(buf.buf[:buf.size]), nil
}
if !buf.startTime.After(lastReadTime) && buf.stopTime.After(lastReadTime) {
pos := buf.locateByTs(lastReadTime)
// fmt.Printf("locate buffer[%d] pos %d\n", i, pos)
return copiedBytes(buf.buf[pos:buf.size])
return copiedBytes(buf.buf[pos:buf.size]), nil
}
}
// println("return the current buf", lastReadTime.UnixNano())
return copiedBytes(m.buf[:m.pos])
return copiedBytes(m.buf[:m.pos]), nil
}
lastTs := lastReadTime.UnixNano()
@ -243,7 +249,7 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
}
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)
return copiedBytes(m.buf[pos:m.pos])
return copiedBytes(m.buf[pos:m.pos]), nil
}
h = mid
}
@ -251,7 +257,7 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
}
// FIXME: this could be that the buffer has been flushed already
return nil
return nil, nil
}
func (m *LogBuffer) ReleaseMemory(b *bytes.Buffer) {

View file

@ -14,6 +14,7 @@ import (
var (
ResumeError = fmt.Errorf("resume")
ResumeFromDiskError = fmt.Errorf("resumeFromDisk")
)
func (logBuffer *LogBuffer) LoopProcessLogData(
@ -34,7 +35,10 @@ func (logBuffer *LogBuffer) LoopProcessLogData(
if bytesBuf != nil {
logBuffer.ReleaseMemory(bytesBuf)
}
bytesBuf = logBuffer.ReadFromBuffer(lastReadTime)
bytesBuf, err = logBuffer.ReadFromBuffer(lastReadTime)
if err == ResumeFromDiskError {
return lastReadTime, ResumeFromDiskError
}
// fmt.Printf("ReadFromBuffer by %v\n", lastReadTime)
if bytesBuf == nil {
if waitForDataFn() {