1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-05-09 04:50:47 +02:00

filer: add filer group

This commit is contained in:
chrislu 2022-05-01 21:59:16 -07:00
parent 4bd6bea429
commit 94635e9b5c
33 changed files with 677 additions and 590 deletions

View file

@ -15,6 +15,15 @@ const (
BrokerType = "broker"
)
type FilerGroup string
type Filers struct {
filers map[pb.ServerAddress]*ClusterNode
leaders *Leaders
}
type Leaders struct {
leaders [3]pb.ServerAddress
}
type ClusterNode struct {
Address pb.ServerAddress
Version string
@ -22,42 +31,50 @@ type ClusterNode struct {
createdTs time.Time
}
type Leaders struct {
leaders [3]pb.ServerAddress
}
type Cluster struct {
filers map[pb.ServerAddress]*ClusterNode
filersLock sync.RWMutex
filerLeaders *Leaders
brokers map[pb.ServerAddress]*ClusterNode
brokersLock sync.RWMutex
filerGroup2filers map[FilerGroup]*Filers
filersLock sync.RWMutex
brokers map[pb.ServerAddress]*ClusterNode
brokersLock sync.RWMutex
}
func NewCluster() *Cluster {
return &Cluster{
filers: make(map[pb.ServerAddress]*ClusterNode),
filerLeaders: &Leaders{},
brokers: make(map[pb.ServerAddress]*ClusterNode),
filerGroup2filers: make(map[FilerGroup]*Filers),
brokers: make(map[pb.ServerAddress]*ClusterNode),
}
}
func (cluster *Cluster) AddClusterNode(nodeType string, address pb.ServerAddress, version string) []*master_pb.KeepConnectedResponse {
func (cluster *Cluster) getFilers(filerGroup FilerGroup, createIfNotFound bool) *Filers {
cluster.filersLock.Lock()
defer cluster.filersLock.Unlock()
filers, found := cluster.filerGroup2filers[filerGroup]
if !found && createIfNotFound {
filers = &Filers{
filers: make(map[pb.ServerAddress]*ClusterNode),
leaders: &Leaders{},
}
cluster.filerGroup2filers[filerGroup] = filers
}
return filers
}
func (cluster *Cluster) AddClusterNode(ns, nodeType string, address pb.ServerAddress, version string) []*master_pb.KeepConnectedResponse {
filerGroup := FilerGroup(ns)
switch nodeType {
case FilerType:
cluster.filersLock.Lock()
defer cluster.filersLock.Unlock()
if existingNode, found := cluster.filers[address]; found {
filers := cluster.getFilers(filerGroup, true)
if existingNode, found := filers.filers[address]; found {
existingNode.counter++
return nil
}
cluster.filers[address] = &ClusterNode{
filers.filers[address] = &ClusterNode{
Address: address,
Version: version,
counter: 1,
createdTs: time.Now(),
}
return cluster.ensureFilerLeaders(true, nodeType, address)
return cluster.ensureFilerLeaders(filers, true, filerGroup, nodeType, address)
case BrokerType:
cluster.brokersLock.Lock()
defer cluster.brokersLock.Unlock()
@ -94,18 +111,21 @@ func (cluster *Cluster) AddClusterNode(nodeType string, address pb.ServerAddress
return nil
}
func (cluster *Cluster) RemoveClusterNode(nodeType string, address pb.ServerAddress) []*master_pb.KeepConnectedResponse {
func (cluster *Cluster) RemoveClusterNode(ns string, nodeType string, address pb.ServerAddress) []*master_pb.KeepConnectedResponse {
filerGroup := FilerGroup(ns)
switch nodeType {
case FilerType:
cluster.filersLock.Lock()
defer cluster.filersLock.Unlock()
if existingNode, found := cluster.filers[address]; !found {
filers := cluster.getFilers(filerGroup, false)
if filers == nil {
return nil
}
if existingNode, found := filers.filers[address]; !found {
return nil
} else {
existingNode.counter--
if existingNode.counter <= 0 {
delete(cluster.filers, address)
return cluster.ensureFilerLeaders(false, nodeType, address)
delete(filers.filers, address)
return cluster.ensureFilerLeaders(filers, false, filerGroup, nodeType, address)
}
}
case BrokerType:
@ -142,12 +162,16 @@ func (cluster *Cluster) RemoveClusterNode(nodeType string, address pb.ServerAddr
return nil
}
func (cluster *Cluster) ListClusterNode(nodeType string) (nodes []*ClusterNode) {
func (cluster *Cluster) ListClusterNode(filerGroup FilerGroup, nodeType string) (nodes []*ClusterNode) {
switch nodeType {
case FilerType:
filers := cluster.getFilers(filerGroup, false)
if filers == nil {
return
}
cluster.filersLock.RLock()
defer cluster.filersLock.RUnlock()
for _, node := range cluster.filers {
for _, node := range filers.filers {
nodes = append(nodes, node)
}
case BrokerType:
@ -161,41 +185,48 @@ func (cluster *Cluster) ListClusterNode(nodeType string) (nodes []*ClusterNode)
return
}
func (cluster *Cluster) IsOneLeader(address pb.ServerAddress) bool {
return cluster.filerLeaders.isOneLeader(address)
func (cluster *Cluster) IsOneLeader(filerGroup FilerGroup, address pb.ServerAddress) bool {
filers := cluster.getFilers(filerGroup, false)
if filers == nil {
return false
}
return filers.leaders.isOneLeader(address)
}
func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address pb.ServerAddress) (result []*master_pb.KeepConnectedResponse) {
func (cluster *Cluster) ensureFilerLeaders(filers *Filers, isAdd bool, filerGroup FilerGroup, nodeType string, address pb.ServerAddress) (result []*master_pb.KeepConnectedResponse) {
if isAdd {
if cluster.filerLeaders.addLeaderIfVacant(address) {
if filers.leaders.addLeaderIfVacant(address) {
// has added the address as one leader
result = append(result, &master_pb.KeepConnectedResponse{
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
NodeType: nodeType,
Address: string(address),
IsLeader: true,
IsAdd: true,
FilerGroup: string(filerGroup),
NodeType: nodeType,
Address: string(address),
IsLeader: true,
IsAdd: true,
},
})
} else {
result = append(result, &master_pb.KeepConnectedResponse{
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
NodeType: nodeType,
Address: string(address),
IsLeader: false,
IsAdd: true,
FilerGroup: string(filerGroup),
NodeType: nodeType,
Address: string(address),
IsLeader: false,
IsAdd: true,
},
})
}
} else {
if cluster.filerLeaders.removeLeaderIfExists(address) {
if filers.leaders.removeLeaderIfExists(address) {
result = append(result, &master_pb.KeepConnectedResponse{
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
NodeType: nodeType,
Address: string(address),
IsLeader: true,
IsAdd: false,
FilerGroup: string(filerGroup),
NodeType: nodeType,
Address: string(address),
IsLeader: true,
IsAdd: false,
},
})
@ -203,8 +234,8 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
var shortestDuration int64 = math.MaxInt64
now := time.Now()
var candidateAddress pb.ServerAddress
for _, node := range cluster.filers {
if cluster.filerLeaders.isOneLeader(node.Address) {
for _, node := range filers.filers {
if filers.leaders.isOneLeader(node.Address) {
continue
}
duration := now.Sub(node.createdTs).Nanoseconds()
@ -214,7 +245,7 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
}
}
if candidateAddress != "" {
cluster.filerLeaders.addLeaderIfVacant(candidateAddress)
filers.leaders.addLeaderIfVacant(candidateAddress)
// added a new leader
result = append(result, &master_pb.KeepConnectedResponse{
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
@ -228,10 +259,11 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
} else {
result = append(result, &master_pb.KeepConnectedResponse{
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
NodeType: nodeType,
Address: string(address),
IsLeader: false,
IsAdd: false,
FilerGroup: string(filerGroup),
NodeType: nodeType,
Address: string(address),
IsLeader: false,
IsAdd: false,
},
})
}

View file

@ -129,7 +129,7 @@ func runBenchmark(cmd *Command, args []string) bool {
defer pprof.StopCPUProfile()
}
b.masterClient = wdclient.NewMasterClient(b.grpcDialOption, "client", "", "", pb.ServerAddresses(*b.masters).ToAddressMap())
b.masterClient = wdclient.NewMasterClient(b.grpcDialOption, "", "client", "", "", pb.ServerAddresses(*b.masters).ToAddressMap())
go b.masterClient.KeepConnectedToMaster()
b.masterClient.WaitUntilConnected()

View file

@ -37,6 +37,7 @@ type FilerOptions struct {
port *int
portGrpc *int
publicPort *int
filerGroup *string
collection *string
defaultReplicaPlacement *string
disableDirListing *bool
@ -59,6 +60,7 @@ type FilerOptions struct {
func init() {
cmdFiler.Run = runFiler // break init cycle
f.mastersString = cmdFiler.Flag.String("master", "localhost:9333", "comma-separated master servers")
f.filerGroup = cmdFiler.Flag.String("filerGroup", "", "share metadata with other filers in the same filerGroup")
f.collection = cmdFiler.Flag.String("collection", "", "all data will be stored in this default collection")
f.ip = cmdFiler.Flag.String("ip", util.DetectedHostAddress(), "filer server http listen ip address")
f.bindIp = cmdFiler.Flag.String("ip.bind", "", "ip address to bind to. If empty, default to same as -ip option.")
@ -201,6 +203,7 @@ func (fo *FilerOptions) startFiler() {
fs, nfs_err := weed_server.NewFilerServer(defaultMux, publicVolumeMux, &weed_server.FilerOption{
Masters: fo.masters,
FilerGroup: *fo.filerGroup,
Collection: *fo.collection,
DefaultReplication: *fo.defaultReplicaPlacement,
DisableDirListing: *fo.disableDirListing,

View file

@ -101,6 +101,7 @@ func init() {
masterOptions.heartbeatInterval = cmdServer.Flag.Duration("master.heartbeatInterval", 300*time.Millisecond, "heartbeat interval of master servers, and will be randomly multiplied by [1, 1.25)")
masterOptions.electionTimeout = cmdServer.Flag.Duration("master.electionTimeout", 10*time.Second, "election timeout of master servers")
filerOptions.filerGroup = cmdServer.Flag.String("filer.filerGroup", "", "share metadata with other filers in the same filerGroup")
filerOptions.collection = cmdServer.Flag.String("filer.collection", "", "all data will be stored in this collection")
filerOptions.port = cmdServer.Flag.Int("filer.port", 8888, "filer server http listen port")
filerOptions.portGrpc = cmdServer.Flag.Int("filer.port.grpc", 0, "filer server grpc listen port")

View file

@ -18,6 +18,7 @@ var (
func init() {
cmdShell.Run = runShell // break init cycle
shellOptions.Masters = cmdShell.Flag.String("master", "", "comma-separated master servers, e.g. localhost:9333")
shellOptions.FilerGroup = cmdShell.Flag.String("filerGroup", "", "filerGroup for the filers")
shellInitialFiler = cmdShell.Flag.String("filer", "", "filer host and port, e.g. localhost:8888")
shellCluster = cmdShell.Flag.String("cluster", "", "cluster defined in shell.toml")
}

View file

@ -49,10 +49,10 @@ type Filer struct {
UniqueFileId uint32
}
func NewFiler(masters map[string]pb.ServerAddress, grpcDialOption grpc.DialOption,
filerHost pb.ServerAddress, collection string, replication string, dataCenter string, notifyFn func()) *Filer {
func NewFiler(masters map[string]pb.ServerAddress, grpcDialOption grpc.DialOption, filerHost pb.ServerAddress,
filerGroup string, collection string, replication string, dataCenter string, notifyFn func()) *Filer {
f := &Filer{
MasterClient: wdclient.NewMasterClient(grpcDialOption, cluster.FilerType, filerHost, dataCenter, masters),
MasterClient: wdclient.NewMasterClient(grpcDialOption, filerGroup, cluster.FilerType, filerHost, dataCenter, masters),
fileIdDeletionQueue: util.NewUnboundedQueue(),
GrpcDialOption: grpcDialOption,
FilerConf: NewFilerConf(),
@ -84,6 +84,7 @@ func (f *Filer) ListExistingPeerUpdates() (existingNodes []*master_pb.ClusterNod
if grpcErr := pb.WithMasterClient(false, f.MasterClient.GetMaster(), f.GrpcDialOption, func(client master_pb.SeaweedClient) error {
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
ClientType: cluster.FilerType,
FilerGroup: f.MasterClient.FilerGroup,
})
glog.V(0).Infof("the cluster has %d filers\n", len(resp.ClusterNodes))

View file

@ -12,7 +12,7 @@ import (
)
func TestCreateAndFind(t *testing.T) {
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
dir := t.TempDir()
store := &LevelDBStore{}
store.initialize(dir)
@ -65,7 +65,7 @@ func TestCreateAndFind(t *testing.T) {
}
func TestEmptyRoot(t *testing.T) {
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
dir := t.TempDir()
store := &LevelDBStore{}
store.initialize(dir)
@ -87,7 +87,7 @@ func TestEmptyRoot(t *testing.T) {
}
func BenchmarkInsertEntry(b *testing.B) {
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
dir := b.TempDir()
store := &LevelDBStore{}
store.initialize(dir)

View file

@ -9,7 +9,7 @@ import (
)
func TestCreateAndFind(t *testing.T) {
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
dir := t.TempDir()
store := &LevelDB2Store{}
store.initialize(dir, 2)
@ -62,7 +62,7 @@ func TestCreateAndFind(t *testing.T) {
}
func TestEmptyRoot(t *testing.T) {
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
dir := t.TempDir()
store := &LevelDB2Store{}
store.initialize(dir, 2)

View file

@ -9,7 +9,7 @@ import (
)
func TestCreateAndFind(t *testing.T) {
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
dir := t.TempDir()
store := &LevelDB3Store{}
store.initialize(dir)
@ -62,7 +62,7 @@ func TestCreateAndFind(t *testing.T) {
}
func TestEmptyRoot(t *testing.T) {
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
dir := t.TempDir()
store := &LevelDB3Store{}
store.initialize(dir)

View file

@ -49,7 +49,7 @@ var s3ApiConfigure IamS3ApiConfig
func NewIamApiServer(router *mux.Router, option *IamServerOption) (iamApiServer *IamApiServer, err error) {
s3ApiConfigure = IamS3ApiConfigure{
option: option,
masterClient: wdclient.NewMasterClient(option.GrpcDialOption, "iam", "", "", option.Masters),
masterClient: wdclient.NewMasterClient(option.GrpcDialOption, "", "iam", "", "", option.Masters),
}
s3Option := s3api.S3ApiServerOption{Filer: option.Filer}
iamApiServer = &IamApiServer{

View file

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.28.0
// protoc v3.19.4
// protoc-gen-go v1.26.0
// protoc v3.17.3
// source: filer.proto
package filer_pb

View file

@ -11,6 +11,7 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// SeaweedFilerClient is the client API for SeaweedFiler service.

View file

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.28.0
// protoc v3.19.4
// protoc-gen-go v1.26.0
// protoc v3.17.3
// source: iam.proto
package iam_pb

View file

@ -8,6 +8,7 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// SeaweedIdentityAccessManagementClient is the client API for SeaweedIdentityAccessManagement service.

View file

@ -137,6 +137,7 @@ message KeepConnectedRequest {
string client_type = 1;
string client_address = 3;
string version = 4;
string filer_group = 5;
}
message VolumeLocation {
@ -156,6 +157,7 @@ message ClusterNodeUpdate {
string address = 2;
bool is_leader = 3;
bool is_add = 4;
string filer_group = 5;
}
message KeepConnectedResponse {
@ -310,6 +312,7 @@ message GetMasterConfigurationResponse {
message ListClusterNodesRequest {
string client_type = 1;
string filer_group = 2;
}
message ListClusterNodesResponse {
message ClusterNode {

File diff suppressed because it is too large Load diff

View file

@ -11,6 +11,7 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// SeaweedClient is the client API for Seaweed service.

View file

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.28.0
// protoc v3.19.4
// protoc-gen-go v1.26.0
// protoc v3.17.3
// source: messaging.proto
package messaging_pb

View file

@ -11,6 +11,7 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// SeaweedMessagingClient is the client API for SeaweedMessaging service.

View file

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.28.0
// protoc v3.19.4
// protoc-gen-go v1.26.0
// protoc v3.17.3
// source: mount.proto
package mount_pb

View file

@ -11,6 +11,7 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// SeaweedMountClient is the client API for SeaweedMount service.

View file

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.28.0
// protoc v3.19.4
// protoc-gen-go v1.26.0
// protoc v3.17.3
// source: remote.proto
package remote_pb

View file

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.28.0
// protoc v3.19.4
// protoc-gen-go v1.26.0
// protoc v3.17.3
// source: volume_server.proto
package volume_server_pb

View file

@ -11,6 +11,7 @@ import (
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
// Requires gRPC-Go v1.32.0 or later.
const _ = grpc.SupportPackageIsVersion7
// VolumeServerClient is the client API for VolumeServer service.

View file

@ -50,6 +50,7 @@ import (
type FilerOption struct {
Masters map[string]pb.ServerAddress
FilerGroup string
Collection string
DefaultReplication string
DisableDirListing bool
@ -118,7 +119,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
glog.Fatal("master list is required!")
}
fs.filer = filer.NewFiler(option.Masters, fs.grpcDialOption, option.Host, option.Collection, option.DefaultReplication, option.DataCenter, func() {
fs.filer = filer.NewFiler(option.Masters, fs.grpcDialOption, option.Host, option.FilerGroup, option.Collection, option.DefaultReplication, option.DataCenter, func() {
fs.listenersCond.Broadcast()
})
fs.filer.Cipher = option.Cipher

View file

@ -201,13 +201,13 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ
// buffer by 1 so we don't end up getting stuck writing to stopChan forever
stopChan := make(chan bool, 1)
clientName, messageChan := ms.addClient(req.ClientType, peerAddress)
for _, update := range ms.Cluster.AddClusterNode(req.ClientType, peerAddress, req.Version) {
clientName, messageChan := ms.addClient(req.FilerGroup, req.ClientType, peerAddress)
for _, update := range ms.Cluster.AddClusterNode(req.FilerGroup, req.ClientType, peerAddress, req.Version) {
ms.broadcastToClients(update)
}
defer func() {
for _, update := range ms.Cluster.RemoveClusterNode(req.ClientType, peerAddress) {
for _, update := range ms.Cluster.RemoveClusterNode(req.FilerGroup, req.ClientType, peerAddress) {
ms.broadcastToClients(update)
}
ms.deleteClient(clientName)
@ -276,8 +276,8 @@ func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedSe
return nil
}
func (ms *MasterServer) addClient(clientType string, clientAddress pb.ServerAddress) (clientName string, messageChan chan *master_pb.KeepConnectedResponse) {
clientName = clientType + "@" + string(clientAddress)
func (ms *MasterServer) addClient(filerGroup, clientType string, clientAddress pb.ServerAddress) (clientName string, messageChan chan *master_pb.KeepConnectedResponse) {
clientName = filerGroup + "." + clientType + "@" + string(clientAddress)
glog.V(0).Infof("+ client %v", clientName)
// we buffer this because otherwise we end up in a potential deadlock where

View file

@ -10,26 +10,26 @@ import (
func (ms *MasterServer) ListClusterNodes(ctx context.Context, req *master_pb.ListClusterNodesRequest) (*master_pb.ListClusterNodesResponse, error) {
resp := &master_pb.ListClusterNodesResponse{}
clusterNodes := ms.Cluster.ListClusterNode(req.ClientType)
filerGroup := cluster.FilerGroup(req.FilerGroup)
clusterNodes := ms.Cluster.ListClusterNode(filerGroup, req.ClientType)
for _, node := range clusterNodes {
resp.ClusterNodes = append(resp.ClusterNodes, &master_pb.ListClusterNodesResponse_ClusterNode{
Address: string(node.Address),
Version: node.Version,
IsLeader: ms.Cluster.IsOneLeader(node.Address),
IsLeader: ms.Cluster.IsOneLeader(filerGroup, node.Address),
})
}
return resp, nil
}
func (ms *MasterServer) GetOneFiler() pb.ServerAddress {
func (ms *MasterServer) GetOneFiler(filerGroup cluster.FilerGroup) pb.ServerAddress {
clusterNodes := ms.Cluster.ListClusterNode(cluster.FilerType)
clusterNodes := ms.Cluster.ListClusterNode(filerGroup, cluster.FilerType)
var filers []pb.ServerAddress
for _, node := range clusterNodes {
if ms.Cluster.IsOneLeader(node.Address) {
if ms.Cluster.IsOneLeader(filerGroup, node.Address) {
filers = append(filers, node.Address)
}
}

View file

@ -113,7 +113,7 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers map[string]pb.Se
vgCh: make(chan *topology.VolumeGrowRequest, 1<<6),
clientChans: make(map[string]chan *master_pb.KeepConnectedResponse),
grpcDialOption: grpcDialOption,
MasterClient: wdclient.NewMasterClient(grpcDialOption, cluster.MasterType, option.Master, "", peers),
MasterClient: wdclient.NewMasterClient(grpcDialOption, "", cluster.MasterType, option.Master, "", peers),
adminLocks: NewAdminLocks(),
Cluster: cluster.NewCluster(),
}
@ -285,7 +285,7 @@ func (ms *MasterServer) startAdminScripts() {
for {
time.Sleep(time.Duration(sleepMinutes) * time.Minute)
if ms.Topo.IsLeader() {
shellOptions.FilerAddress = ms.GetOneFiler()
shellOptions.FilerAddress = ms.GetOneFiler(cluster.FilerGroup(*shellOptions.FilerGroup))
if shellOptions.FilerAddress == "" {
continue
}

View file

@ -59,6 +59,7 @@ func (c *commandClusterCheck) Do(args []string, commandEnv *CommandEnv, writer i
err = commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
ClientType: cluster.FilerType,
FilerGroup: *commandEnv.option.FilerGroup,
})
for _, node := range resp.ClusterNodes {

View file

@ -39,6 +39,7 @@ func (c *commandClusterPs) Do(args []string, commandEnv *CommandEnv, writer io.W
err = commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
ClientType: cluster.FilerType,
FilerGroup: *commandEnv.option.FilerGroup,
})
fmt.Fprintf(writer, "the cluster has %d filers\n", len(resp.ClusterNodes))

View file

@ -22,6 +22,7 @@ type ShellOptions struct {
// shell transient context
FilerHost string
FilerPort int64
FilerGroup *string
FilerAddress pb.ServerAddress
Directory string
}
@ -46,7 +47,7 @@ var (
func NewCommandEnv(options *ShellOptions) *CommandEnv {
ce := &CommandEnv{
env: make(map[string]string),
MasterClient: wdclient.NewMasterClient(options.GrpcDialOption, pb.AdminShellClient, "", "", pb.ServerAddresses(*options.Masters).ToAddressMap()),
MasterClient: wdclient.NewMasterClient(options.GrpcDialOption, *options.FilerGroup, pb.AdminShellClient, "", "", pb.ServerAddresses(*options.Masters).ToAddressMap()),
option: options,
}
ce.locker = exclusive_locks.NewExclusiveLocker(ce.MasterClient, "admin")

View file

@ -54,6 +54,7 @@ func RunShell(options ShellOptions) {
commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
ClientType: cluster.FilerType,
FilerGroup: *options.FilerGroup,
})
if err != nil {
return err

View file

@ -15,6 +15,7 @@ import (
)
type MasterClient struct {
FilerGroup string
clientType string
clientHost pb.ServerAddress
currentMaster pb.ServerAddress
@ -26,8 +27,9 @@ type MasterClient struct {
OnPeerUpdate func(update *master_pb.ClusterNodeUpdate)
}
func NewMasterClient(grpcDialOption grpc.DialOption, 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 {
return &MasterClient{
FilerGroup: filerGroup,
clientType: clientType,
clientHost: clientHost,
masters: masters,
@ -53,7 +55,7 @@ func (mc *MasterClient) WaitUntilConnected() {
}
func (mc *MasterClient) KeepConnectedToMaster() {
glog.V(1).Infof("%s masterClient bootstraps with masters %v", mc.clientType, mc.masters)
glog.V(1).Infof("%s.%s masterClient bootstraps with masters %v", mc.FilerGroup, mc.clientType, mc.masters)
for {
mc.tryAllMasters()
time.Sleep(time.Second)
@ -101,7 +103,7 @@ func (mc *MasterClient) tryAllMasters() {
}
func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedLeader pb.ServerAddress) {
glog.V(1).Infof("%s masterClient Connecting to master %v", mc.clientType, master)
glog.V(1).Infof("%s.%s masterClient Connecting to master %v", mc.FilerGroup, mc.clientType, master)
stats.MasterClientConnectCounter.WithLabelValues("total").Inc()
gprcErr := pb.WithMasterClient(true, master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
ctx, cancel := context.WithCancel(context.Background())
@ -109,28 +111,29 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
stream, err := client.KeepConnected(ctx)
if err != nil {
glog.V(1).Infof("%s masterClient failed to keep connected to %s: %v", mc.clientType, master, err)
glog.V(1).Infof("%s.%s masterClient failed to keep connected to %s: %v", mc.FilerGroup, mc.clientType, master, err)
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToKeepConnected).Inc()
return err
}
if err = stream.Send(&master_pb.KeepConnectedRequest{
FilerGroup: mc.FilerGroup,
ClientType: mc.clientType,
ClientAddress: string(mc.clientHost),
Version: util.Version(),
}); err != nil {
glog.V(0).Infof("%s masterClient failed to send to %s: %v", mc.clientType, master, err)
glog.V(0).Infof("%s.%s masterClient failed to send to %s: %v", mc.FilerGroup, mc.clientType, master, err)
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToSend).Inc()
return err
}
glog.V(1).Infof("%s masterClient Connected to %v", mc.clientType, master)
glog.V(1).Infof("%s.%s masterClient Connected to %v", mc.FilerGroup, mc.clientType, master)
mc.currentMaster = master
for {
resp, err := stream.Recv()
if err != nil {
glog.V(0).Infof("%s masterClient failed to receive from %s: %v", mc.clientType, master, err)
glog.V(0).Infof("%s.%s masterClient failed to receive from %s: %v", mc.FilerGroup, mc.clientType, master, err)
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToReceive).Inc()
return err
}
@ -152,19 +155,19 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
GrpcPort: int(resp.VolumeLocation.GrpcPort),
}
for _, newVid := range resp.VolumeLocation.NewVids {
glog.V(1).Infof("%s: %s masterClient adds volume %d", mc.clientType, loc.Url, newVid)
glog.V(1).Infof("%s.%s: %s masterClient adds volume %d", mc.FilerGroup, mc.clientType, loc.Url, newVid)
mc.addLocation(newVid, loc)
}
for _, deletedVid := range resp.VolumeLocation.DeletedVids {
glog.V(1).Infof("%s: %s masterClient removes volume %d", mc.clientType, loc.Url, deletedVid)
glog.V(1).Infof("%s.%s: %s masterClient removes volume %d", mc.FilerGroup, mc.clientType, loc.Url, deletedVid)
mc.deleteLocation(deletedVid, loc)
}
for _, newEcVid := range resp.VolumeLocation.NewEcVids {
glog.V(1).Infof("%s: %s masterClient adds ec volume %d", mc.clientType, loc.Url, newEcVid)
glog.V(1).Infof("%s.%s: %s masterClient adds ec volume %d", mc.FilerGroup, mc.clientType, loc.Url, newEcVid)
mc.addEcLocation(newEcVid, loc)
}
for _, deletedEcVid := range resp.VolumeLocation.DeletedEcVids {
glog.V(1).Infof("%s: %s masterClient removes ec volume %d", mc.clientType, loc.Url, deletedEcVid)
glog.V(1).Infof("%s.%s: %s masterClient removes ec volume %d", mc.FilerGroup, mc.clientType, loc.Url, deletedEcVid)
mc.deleteEcLocation(deletedEcVid, loc)
}
}
@ -172,13 +175,15 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
if resp.ClusterNodeUpdate != nil {
update := resp.ClusterNodeUpdate
if mc.OnPeerUpdate != nil {
if update.IsAdd {
glog.V(0).Infof("+ %s %s leader:%v\n", update.NodeType, update.Address, update.IsLeader)
} else {
glog.V(0).Infof("- %s %s leader:%v\n", update.NodeType, update.Address, update.IsLeader)
if update.FilerGroup == mc.FilerGroup {
if update.IsAdd {
glog.V(0).Infof("+ %s.%s %s leader:%v\n", update.FilerGroup, update.NodeType, update.Address, update.IsLeader)
} else {
glog.V(0).Infof("- %s.%s %s leader:%v\n", update.FilerGroup, update.NodeType, update.Address, update.IsLeader)
}
stats.MasterClientConnectCounter.WithLabelValues(stats.OnPeerUpdate).Inc()
mc.OnPeerUpdate(update)
}
stats.MasterClientConnectCounter.WithLabelValues(stats.OnPeerUpdate).Inc()
mc.OnPeerUpdate(update)
}
}
@ -187,7 +192,7 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
})
if gprcErr != nil {
stats.MasterClientConnectCounter.WithLabelValues(stats.Failed).Inc()
glog.V(1).Infof("%s masterClient failed to connect with master %v: %v", mc.clientType, master, gprcErr)
glog.V(1).Infof("%s.%s masterClient failed to connect with master %v: %v", mc.FilerGroup, mc.clientType, master, gprcErr)
}
return
}