aggregate multiple filer metadata chagne events

This commit is contained in:
Chris Lu 2020-07-05 23:05:02 -07:00
parent 49929e0869
commit 0b8cc9b626
3 changed files with 102 additions and 9 deletions

View file

@ -0,0 +1,91 @@
package filer2
import (
"context"
"fmt"
"io"
"sync"
"time"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util/log_buffer"
)
type MetaAggregator struct {
filers []string
grpcDialOption grpc.DialOption
MetaLogBuffer *log_buffer.LogBuffer
// notifying clients
ListenersLock sync.Mutex
ListenersCond *sync.Cond
}
func NewMetaAggregator(filers []string, grpcDialOption grpc.DialOption) *MetaAggregator {
t := &MetaAggregator{
filers: filers,
grpcDialOption: grpcDialOption,
}
t.ListenersCond = sync.NewCond(&t.ListenersLock)
t.MetaLogBuffer = log_buffer.NewLogBuffer(time.Minute, nil, func() {
t.ListenersCond.Broadcast()
})
return t
}
func (ma *MetaAggregator) StartLoopSubscribe(lastTsNs int64) {
for _, filer := range ma.filers {
go ma.subscribeToOneFiler(filer, lastTsNs)
}
}
func (ma *MetaAggregator) subscribeToOneFiler(filer string, lastTsNs int64) {
processEventFn := func(event *filer_pb.SubscribeMetadataResponse) error {
data, err := proto.Marshal(event)
if err != nil {
glog.Errorf("failed to marshal subscribed filer_pb.SubscribeMetadataResponse %+v: %v", event, err)
return err
}
dir := event.Directory
println("received meta change", dir, "size", len(data))
ma.MetaLogBuffer.AddToBuffer([]byte(dir), data)
return nil
}
for {
err := pb.WithFilerClient(filer, ma.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
stream, err := client.SubscribeLocalMetadata(context.Background(), &filer_pb.SubscribeMetadataRequest{
ClientName: "filer",
PathPrefix: "/",
SinceNs: lastTsNs,
})
if err != nil {
return fmt.Errorf("subscribe: %v", err)
}
for {
resp, listenErr := stream.Recv()
if listenErr == io.EOF {
return nil
}
if listenErr != nil {
return listenErr
}
if err := processEventFn(resp); err != nil {
return fmt.Errorf("process %v: %v", resp, err)
}
lastTsNs = resp.TsNs
}
})
if err != nil {
glog.V(0).Infof("subscribing remote %s meta change: %v", filer, err)
time.Sleep(1733 * time.Millisecond)
}
}
}

View file

@ -37,10 +37,10 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
lastReadTime = time.Unix(0, processedTsNs)
}
err = fs.filer.LocalMetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
fs.listenersLock.Lock()
fs.listenersCond.Wait()
fs.listenersLock.Unlock()
err = fs.metaAggregator.MetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
fs.metaAggregator.ListenersLock.Lock()
fs.metaAggregator.ListenersCond.Wait()
fs.metaAggregator.ListenersLock.Unlock()
return true
}, eachLogEntryFn)
@ -134,7 +134,3 @@ func (fs *FilerServer) addClient(clientType string, clientAddress string) (clien
func (fs *FilerServer) deleteClient(clientName string) {
glog.V(0).Infof("- listener %v", clientName)
}
func (fs *FilerServer) notifyMetaListeners() {
fs.listenersCond.Broadcast()
}

View file

@ -52,12 +52,14 @@ type FilerOption struct {
Port uint32
recursiveDelete bool
Cipher bool
Filers []string
}
type FilerServer struct {
option *FilerOption
secret security.SigningKey
filer *filer2.Filer
metaAggregator *filer2.MetaAggregator
grpcDialOption grpc.DialOption
// notifying clients
@ -81,12 +83,16 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
glog.Fatal("master list is required!")
}
fs.filer = filer2.NewFiler(option.Masters, fs.grpcDialOption, option.Host, option.Port, option.Collection, option.DefaultReplication, fs.notifyMetaListeners)
fs.filer = filer2.NewFiler(option.Masters, fs.grpcDialOption, option.Host, option.Port, option.Collection, option.DefaultReplication, func() {
fs.listenersCond.Broadcast()
})
fs.metaAggregator = filer2.NewMetaAggregator(append(option.Filers, fmt.Sprintf("%s:%d", option.Host, option.Port)), fs.grpcDialOption)
fs.filer.Cipher = option.Cipher
maybeStartMetrics(fs, option)
go fs.filer.KeepConnectedToMaster()
fs.metaAggregator.StartLoopSubscribe(time.Now().UnixNano())
v := util.GetViper()
if !util.LoadConfiguration("filer", false) {