mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
fresh filer store bootstrap from the oldest peer
This commit is contained in:
parent
490e0850bf
commit
6adc42147f
|
@ -12,7 +12,7 @@ var (
|
||||||
Stores []FilerStore
|
Stores []FilerStore
|
||||||
)
|
)
|
||||||
|
|
||||||
func (f *Filer) LoadConfiguration(config *util.ViperProxy) {
|
func (f *Filer) LoadConfiguration(config *util.ViperProxy) (isFresh bool) {
|
||||||
|
|
||||||
validateOneEnabledStore(config)
|
validateOneEnabledStore(config)
|
||||||
|
|
||||||
|
@ -24,7 +24,7 @@ func (f *Filer) LoadConfiguration(config *util.ViperProxy) {
|
||||||
if err := store.Initialize(config, store.GetName()+"."); err != nil {
|
if err := store.Initialize(config, store.GetName()+"."); err != nil {
|
||||||
glog.Fatalf("failed to initialize store for %s: %+v", store.GetName(), err)
|
glog.Fatalf("failed to initialize store for %s: %+v", store.GetName(), err)
|
||||||
}
|
}
|
||||||
f.SetStore(store)
|
isFresh = f.SetStore(store)
|
||||||
glog.V(0).Infof("configured filer store to %s", store.GetName())
|
glog.V(0).Infof("configured filer store to %s", store.GetName())
|
||||||
hasDefaultStoreConfigured = true
|
hasDefaultStoreConfigured = true
|
||||||
break
|
break
|
||||||
|
@ -77,6 +77,7 @@ func (f *Filer) LoadConfiguration(config *util.ViperProxy) {
|
||||||
glog.V(0).Infof("configure filer %s for %s", store.GetName(), location)
|
glog.V(0).Infof("configure filer %s for %s", store.GetName(), location)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
func validateOneEnabledStore(config *util.ViperProxy) {
|
func validateOneEnabledStore(config *util.ViperProxy) {
|
||||||
|
|
|
@ -7,6 +7,7 @@ import (
|
||||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||||
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
|
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
|
||||||
"os"
|
"os"
|
||||||
|
"sort"
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
@ -68,13 +69,33 @@ func NewFiler(masters map[string]pb.ServerAddress, grpcDialOption grpc.DialOptio
|
||||||
return f
|
return f
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *Filer) AggregateFromPeers(self pb.ServerAddress, existingNodes []*master_pb.ClusterNodeUpdate) {
|
func (f *Filer) MaybeBootstrapFromPeers(self pb.ServerAddress, existingNodes []*master_pb.ClusterNodeUpdate, snapshotTime time.Time) (err error) {
|
||||||
|
if len(existingNodes) == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
sort.Slice(existingNodes, func(i, j int) bool {
|
||||||
|
return existingNodes[i].CreatedAtNs < existingNodes[j].CreatedAtNs
|
||||||
|
})
|
||||||
|
earliestNode := existingNodes[0]
|
||||||
|
if earliestNode.Address == string(self) {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
glog.V(0).Infof("bootstrap from %v", earliestNode.Address)
|
||||||
|
err = pb.FollowMetadata(pb.ServerAddress(earliestNode.Address), f.GrpcDialOption, "bootstrap", int32(f.UniqueFileId), "/", nil,
|
||||||
|
0, snapshotTime.UnixNano(), f.Signature, func(resp *filer_pb.SubscribeMetadataResponse) error {
|
||||||
|
return Replay(f.Store, resp)
|
||||||
|
}, true)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
func (f *Filer) AggregateFromPeers(self pb.ServerAddress, existingNodes []*master_pb.ClusterNodeUpdate, startFrom time.Time) {
|
||||||
|
|
||||||
f.MetaAggregator = NewMetaAggregator(f, self, f.GrpcDialOption)
|
f.MetaAggregator = NewMetaAggregator(f, self, f.GrpcDialOption)
|
||||||
f.MasterClient.OnPeerUpdate = f.MetaAggregator.OnPeerUpdate
|
f.MasterClient.OnPeerUpdate = f.MetaAggregator.OnPeerUpdate
|
||||||
|
|
||||||
for _, peerUpdate := range existingNodes {
|
for _, peerUpdate := range existingNodes {
|
||||||
f.MetaAggregator.OnPeerUpdate(peerUpdate)
|
f.MetaAggregator.OnPeerUpdate(peerUpdate, startFrom)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -104,14 +125,13 @@ func (f *Filer) ListExistingPeerUpdates() (existingNodes []*master_pb.ClusterNod
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *Filer) SetStore(store FilerStore) {
|
func (f *Filer) SetStore(store FilerStore) (isFresh bool) {
|
||||||
f.Store = NewFilerStoreWrapper(store)
|
f.Store = NewFilerStoreWrapper(store)
|
||||||
|
|
||||||
f.setOrLoadFilerStoreSignature(store)
|
return f.setOrLoadFilerStoreSignature(store)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *Filer) setOrLoadFilerStoreSignature(store FilerStore) {
|
func (f *Filer) setOrLoadFilerStoreSignature(store FilerStore) (isFresh bool) {
|
||||||
storeIdBytes, err := store.KvGet(context.Background(), []byte(FilerStoreId))
|
storeIdBytes, err := store.KvGet(context.Background(), []byte(FilerStoreId))
|
||||||
if err == ErrKvNotFound || err == nil && len(storeIdBytes) == 0 {
|
if err == ErrKvNotFound || err == nil && len(storeIdBytes) == 0 {
|
||||||
f.Signature = util.RandomInt32()
|
f.Signature = util.RandomInt32()
|
||||||
|
@ -121,12 +141,14 @@ func (f *Filer) setOrLoadFilerStoreSignature(store FilerStore) {
|
||||||
glog.Fatalf("set %s=%d : %v", FilerStoreId, f.Signature, err)
|
glog.Fatalf("set %s=%d : %v", FilerStoreId, f.Signature, err)
|
||||||
}
|
}
|
||||||
glog.V(0).Infof("create %s to %d", FilerStoreId, f.Signature)
|
glog.V(0).Infof("create %s to %d", FilerStoreId, f.Signature)
|
||||||
|
return true
|
||||||
} else if err == nil && len(storeIdBytes) == 4 {
|
} else if err == nil && len(storeIdBytes) == 4 {
|
||||||
f.Signature = int32(util.BytesToUint32(storeIdBytes))
|
f.Signature = int32(util.BytesToUint32(storeIdBytes))
|
||||||
glog.V(0).Infof("existing %s = %d", FilerStoreId, f.Signature)
|
glog.V(0).Infof("existing %s = %d", FilerStoreId, f.Signature)
|
||||||
} else {
|
} else {
|
||||||
glog.Fatalf("read %v=%v : %v", FilerStoreId, string(storeIdBytes), err)
|
glog.Fatalf("read %v=%v : %v", FilerStoreId, string(storeIdBytes), err)
|
||||||
}
|
}
|
||||||
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *Filer) GetStore() (store FilerStore) {
|
func (f *Filer) GetStore() (store FilerStore) {
|
||||||
|
|
|
@ -48,7 +48,7 @@ func NewMetaAggregator(filer *Filer, self pb.ServerAddress, grpcDialOption grpc.
|
||||||
return t
|
return t
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ma *MetaAggregator) OnPeerUpdate(update *master_pb.ClusterNodeUpdate) {
|
func (ma *MetaAggregator) OnPeerUpdate(update *master_pb.ClusterNodeUpdate, startFrom time.Time) {
|
||||||
if update.NodeType != cluster.FilerType {
|
if update.NodeType != cluster.FilerType {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
@ -57,7 +57,7 @@ func (ma *MetaAggregator) OnPeerUpdate(update *master_pb.ClusterNodeUpdate) {
|
||||||
if update.IsAdd {
|
if update.IsAdd {
|
||||||
// every filer should subscribe to a new filer
|
// every filer should subscribe to a new filer
|
||||||
if ma.setActive(address, true) {
|
if ma.setActive(address, true) {
|
||||||
go ma.loopSubscribeToOnefiler(ma.filer, ma.self, address)
|
go ma.loopSubscribeToOnefiler(ma.filer, ma.self, address, startFrom)
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
ma.setActive(address, false)
|
ma.setActive(address, false)
|
||||||
|
@ -89,21 +89,25 @@ func (ma *MetaAggregator) isActive(address pb.ServerAddress) (isActive bool) {
|
||||||
return count > 0 && isActive
|
return count > 0 && isActive
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ma *MetaAggregator) loopSubscribeToOnefiler(f *Filer, self pb.ServerAddress, peer pb.ServerAddress) {
|
func (ma *MetaAggregator) loopSubscribeToOnefiler(f *Filer, self pb.ServerAddress, peer pb.ServerAddress, startFrom time.Time) {
|
||||||
|
lastTsNs := startFrom.UnixNano()
|
||||||
for {
|
for {
|
||||||
err := ma.doSubscribeToOneFiler(f, self, peer)
|
glog.V(0).Infof("loopSubscribeToOnefiler read %s start from %v %d", peer, time.Unix(0, lastTsNs), lastTsNs)
|
||||||
|
nextLastTsNs, err := ma.doSubscribeToOneFiler(f, self, peer, lastTsNs)
|
||||||
if !ma.isActive(peer) {
|
if !ma.isActive(peer) {
|
||||||
glog.V(0).Infof("stop subscribing remote %s meta change", peer)
|
glog.V(0).Infof("stop subscribing remote %s meta change", peer)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
glog.V(0).Infof("subscribing remote %s meta change: %v", peer, err)
|
glog.V(0).Infof("subscribing remote %s meta change: %v", peer, err)
|
||||||
|
} else if lastTsNs < nextLastTsNs {
|
||||||
|
lastTsNs = nextLastTsNs
|
||||||
}
|
}
|
||||||
time.Sleep(1733 * time.Millisecond)
|
time.Sleep(1733 * time.Millisecond)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ma *MetaAggregator) doSubscribeToOneFiler(f *Filer, self pb.ServerAddress, peer pb.ServerAddress) error {
|
func (ma *MetaAggregator) doSubscribeToOneFiler(f *Filer, self pb.ServerAddress, peer pb.ServerAddress, startFrom int64) (int64, error) {
|
||||||
|
|
||||||
/*
|
/*
|
||||||
Each filer reads the "filer.store.id", which is the store's signature when filer starts.
|
Each filer reads the "filer.store.id", which is the store's signature when filer starts.
|
||||||
|
@ -117,18 +121,15 @@ func (ma *MetaAggregator) doSubscribeToOneFiler(f *Filer, self pb.ServerAddress,
|
||||||
|
|
||||||
var maybeReplicateMetadataChange func(*filer_pb.SubscribeMetadataResponse)
|
var maybeReplicateMetadataChange func(*filer_pb.SubscribeMetadataResponse)
|
||||||
lastPersistTime := time.Now()
|
lastPersistTime := time.Now()
|
||||||
lastTsNs := time.Now().Add(-LogFlushInterval).UnixNano()
|
lastTsNs := startFrom
|
||||||
|
|
||||||
peerSignature, err := ma.readFilerStoreSignature(peer)
|
peerSignature, err := ma.readFilerStoreSignature(peer)
|
||||||
for err != nil {
|
if err != nil {
|
||||||
glog.V(0).Infof("connecting to peer filer %s: %v", peer, err)
|
return lastTsNs, fmt.Errorf("connecting to peer filer %s: %v", peer, err)
|
||||||
time.Sleep(1357 * time.Millisecond)
|
|
||||||
peerSignature, err = ma.readFilerStoreSignature(peer)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// when filer store is not shared by multiple filers
|
// when filer store is not shared by multiple filers
|
||||||
if peerSignature != f.Signature {
|
if peerSignature != f.Signature {
|
||||||
lastTsNs = 0
|
|
||||||
if prevTsNs, err := ma.readOffset(f, peer, peerSignature); err == nil {
|
if prevTsNs, err := ma.readOffset(f, peer, peerSignature); err == nil {
|
||||||
lastTsNs = prevTsNs
|
lastTsNs = prevTsNs
|
||||||
defer func(prevTsNs int64) {
|
defer func(prevTsNs int64) {
|
||||||
|
@ -215,7 +216,7 @@ func (ma *MetaAggregator) doSubscribeToOneFiler(f *Filer, self pb.ServerAddress,
|
||||||
|
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
return err
|
return lastTsNs, err
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ma *MetaAggregator) readFilerStoreSignature(peer pb.ServerAddress) (sig int32, err error) {
|
func (ma *MetaAggregator) readFilerStoreSignature(peer pb.ServerAddress) (sig int32, err error) {
|
||||||
|
@ -241,11 +242,6 @@ func (ma *MetaAggregator) readOffset(f *Filer, peer pb.ServerAddress, peerSignat
|
||||||
|
|
||||||
value, err := f.Store.KvGet(context.Background(), key)
|
value, err := f.Store.KvGet(context.Background(), key)
|
||||||
|
|
||||||
if err == ErrKvNotFound {
|
|
||||||
glog.Warningf("readOffset %s not found", peer)
|
|
||||||
return 0, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, fmt.Errorf("readOffset %s : %v", peer, err)
|
return 0, fmt.Errorf("readOffset %s : %v", peer, err)
|
||||||
}
|
}
|
||||||
|
|
|
@ -151,7 +151,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
|
||||||
// TODO deprecated, will be be removed after 2020-12-31
|
// TODO deprecated, will be be removed after 2020-12-31
|
||||||
// replaced by https://github.com/chrislusf/seaweedfs/wiki/Path-Specific-Configuration
|
// replaced by https://github.com/chrislusf/seaweedfs/wiki/Path-Specific-Configuration
|
||||||
// fs.filer.FsyncBuckets = v.GetStringSlice("filer.options.buckets_fsync")
|
// fs.filer.FsyncBuckets = v.GetStringSlice("filer.options.buckets_fsync")
|
||||||
fs.filer.LoadConfiguration(v)
|
isFresh := fs.filer.LoadConfiguration(v)
|
||||||
|
|
||||||
notification.LoadConfiguration(v, "notification.")
|
notification.LoadConfiguration(v, "notification.")
|
||||||
|
|
||||||
|
@ -165,8 +165,14 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
|
||||||
}
|
}
|
||||||
|
|
||||||
existingNodes := fs.filer.ListExistingPeerUpdates()
|
existingNodes := fs.filer.ListExistingPeerUpdates()
|
||||||
|
startFromTime := time.Now().Add(-filer.LogFlushInterval)
|
||||||
fs.filer.AggregateFromPeers(option.Host, existingNodes)
|
if isFresh {
|
||||||
|
glog.V(0).Infof("%s bootstrap from peers %+v", option.Host, existingNodes)
|
||||||
|
if err := fs.filer.MaybeBootstrapFromPeers(option.Host, existingNodes, startFromTime); err == nil {
|
||||||
|
glog.Fatalf("%s bootstrap from %+v", option.Host, existingNodes)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
fs.filer.AggregateFromPeers(option.Host, existingNodes, startFromTime)
|
||||||
|
|
||||||
fs.filer.LoadBuckets()
|
fs.filer.LoadBuckets()
|
||||||
|
|
||||||
|
|
|
@ -342,7 +342,7 @@ func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer
|
||||||
return seq
|
return seq
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ms *MasterServer) OnPeerUpdate(update *master_pb.ClusterNodeUpdate) {
|
func (ms *MasterServer) OnPeerUpdate(update *master_pb.ClusterNodeUpdate, startFrom time.Time) {
|
||||||
if update.NodeType != cluster.MasterType || ms.Topo.HashicorpRaft == nil {
|
if update.NodeType != cluster.MasterType || ms.Topo.HashicorpRaft == nil {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,7 +24,7 @@ type MasterClient struct {
|
||||||
|
|
||||||
vidMap
|
vidMap
|
||||||
|
|
||||||
OnPeerUpdate func(update *master_pb.ClusterNodeUpdate)
|
OnPeerUpdate func(update *master_pb.ClusterNodeUpdate, startFrom time.Time)
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewMasterClient(grpcDialOption grpc.DialOption, filerGroup string, clientType string, clientHost pb.ServerAddress, clientDataCenter string, masters map[string]pb.ServerAddress) *MasterClient {
|
func NewMasterClient(grpcDialOption grpc.DialOption, filerGroup string, clientType string, clientHost pb.ServerAddress, clientDataCenter string, masters map[string]pb.ServerAddress) *MasterClient {
|
||||||
|
@ -182,7 +182,7 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
|
||||||
glog.V(0).Infof("- %s.%s %s leader:%v\n", update.FilerGroup, update.NodeType, update.Address, update.IsLeader)
|
glog.V(0).Infof("- %s.%s %s leader:%v\n", update.FilerGroup, update.NodeType, update.Address, update.IsLeader)
|
||||||
}
|
}
|
||||||
stats.MasterClientConnectCounter.WithLabelValues(stats.OnPeerUpdate).Inc()
|
stats.MasterClientConnectCounter.WithLabelValues(stats.OnPeerUpdate).Inc()
|
||||||
mc.OnPeerUpdate(update)
|
mc.OnPeerUpdate(update, time.Now())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in a new issue