Browse Source

only broad cast when there are waiting threads

pull/5637/head
chrislu 9 months ago
parent
commit
27bb38228b
  1. 3
      weed/server/filer_grpc_server_sub_meta.go
  2. 17
      weed/server/filer_server.go

3
weed/server/filer_grpc_server_sub_meta.go

@ -4,6 +4,7 @@ import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/stats"
"strings"
"sync/atomic"
"time"
"google.golang.org/protobuf/proto"
@ -150,7 +151,9 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
lastReadTime, isDone, readInMemoryLogErr = fs.filer.LocalMetaLogBuffer.LoopProcessLogData("localMeta:"+clientName, lastReadTime, req.UntilNs, func() bool {
fs.listenersLock.Lock()
atomic.AddInt64(&fs.listenersWaits, 1)
fs.listenersCond.Wait()
atomic.AddInt64(&fs.listenersWaits, -1)
fs.listenersLock.Unlock()
if !fs.hasClient(req.ClientId, req.ClientEpoch) {
return false

17
weed/server/filer_server.go

@ -7,6 +7,7 @@ import (
"os"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/seaweedfs/seaweedfs/weed/stats"
@ -76,7 +77,13 @@ type FilerOption struct {
}
type FilerServer struct {
inFlightDataSize int64
inFlightDataSize int64
listenersWaits int64
// notifying clients
listenersLock sync.Mutex
listenersCond *sync.Cond
inFlightDataLimitCond *sync.Cond
filer_pb.UnimplementedSeaweedFilerServer
@ -90,10 +97,6 @@ type FilerServer struct {
metricsAddress string
metricsIntervalSec int
// notifying clients
listenersLock sync.Mutex
listenersCond *sync.Cond
// track known metadata listeners
knownListenersLock sync.Mutex
knownListeners map[int32]int32
@ -135,7 +138,9 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
v.SetDefault("filer.options.max_file_name_length", 255)
maxFilenameLength := v.GetUint32("filer.options.max_file_name_length")
fs.filer = filer.NewFiler(*option.Masters, fs.grpcDialOption, option.Host, option.FilerGroup, option.Collection, option.DefaultReplication, option.DataCenter, maxFilenameLength, func() {
fs.listenersCond.Broadcast()
if atomic.LoadInt64(&fs.listenersWaits) > 0 {
fs.listenersCond.Broadcast()
}
})
fs.filer.Cipher = option.Cipher
// we do not support IP whitelist right now

Loading…
Cancel
Save