mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
aggregate multiple filer metadata chagne events
This commit is contained in:
parent
49929e0869
commit
0b8cc9b626
91
weed/filer2/meta_aggregator.go
Normal file
91
weed/filer2/meta_aggregator.go
Normal 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -37,10 +37,10 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
|
||||||
lastReadTime = time.Unix(0, processedTsNs)
|
lastReadTime = time.Unix(0, processedTsNs)
|
||||||
}
|
}
|
||||||
|
|
||||||
err = fs.filer.LocalMetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
err = fs.metaAggregator.MetaLogBuffer.LoopProcessLogData(lastReadTime, func() bool {
|
||||||
fs.listenersLock.Lock()
|
fs.metaAggregator.ListenersLock.Lock()
|
||||||
fs.listenersCond.Wait()
|
fs.metaAggregator.ListenersCond.Wait()
|
||||||
fs.listenersLock.Unlock()
|
fs.metaAggregator.ListenersLock.Unlock()
|
||||||
return true
|
return true
|
||||||
}, eachLogEntryFn)
|
}, eachLogEntryFn)
|
||||||
|
|
||||||
|
@ -134,7 +134,3 @@ func (fs *FilerServer) addClient(clientType string, clientAddress string) (clien
|
||||||
func (fs *FilerServer) deleteClient(clientName string) {
|
func (fs *FilerServer) deleteClient(clientName string) {
|
||||||
glog.V(0).Infof("- listener %v", clientName)
|
glog.V(0).Infof("- listener %v", clientName)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (fs *FilerServer) notifyMetaListeners() {
|
|
||||||
fs.listenersCond.Broadcast()
|
|
||||||
}
|
|
||||||
|
|
|
@ -52,12 +52,14 @@ type FilerOption struct {
|
||||||
Port uint32
|
Port uint32
|
||||||
recursiveDelete bool
|
recursiveDelete bool
|
||||||
Cipher bool
|
Cipher bool
|
||||||
|
Filers []string
|
||||||
}
|
}
|
||||||
|
|
||||||
type FilerServer struct {
|
type FilerServer struct {
|
||||||
option *FilerOption
|
option *FilerOption
|
||||||
secret security.SigningKey
|
secret security.SigningKey
|
||||||
filer *filer2.Filer
|
filer *filer2.Filer
|
||||||
|
metaAggregator *filer2.MetaAggregator
|
||||||
grpcDialOption grpc.DialOption
|
grpcDialOption grpc.DialOption
|
||||||
|
|
||||||
// notifying clients
|
// notifying clients
|
||||||
|
@ -81,12 +83,16 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
|
||||||
glog.Fatal("master list is required!")
|
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
|
fs.filer.Cipher = option.Cipher
|
||||||
|
|
||||||
maybeStartMetrics(fs, option)
|
maybeStartMetrics(fs, option)
|
||||||
|
|
||||||
go fs.filer.KeepConnectedToMaster()
|
go fs.filer.KeepConnectedToMaster()
|
||||||
|
fs.metaAggregator.StartLoopSubscribe(time.Now().UnixNano())
|
||||||
|
|
||||||
v := util.GetViper()
|
v := util.GetViper()
|
||||||
if !util.LoadConfiguration("filer", false) {
|
if !util.LoadConfiguration("filer", false) {
|
||||||
|
|
Loading…
Reference in a new issue