mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
filer: add filer group
This commit is contained in:
parent
4bd6bea429
commit
94635e9b5c
|
@ -15,6 +15,15 @@ const (
|
||||||
BrokerType = "broker"
|
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 {
|
type ClusterNode struct {
|
||||||
Address pb.ServerAddress
|
Address pb.ServerAddress
|
||||||
Version string
|
Version string
|
||||||
|
@ -22,42 +31,50 @@ type ClusterNode struct {
|
||||||
createdTs time.Time
|
createdTs time.Time
|
||||||
}
|
}
|
||||||
|
|
||||||
type Leaders struct {
|
|
||||||
leaders [3]pb.ServerAddress
|
|
||||||
}
|
|
||||||
|
|
||||||
type Cluster struct {
|
type Cluster struct {
|
||||||
filers map[pb.ServerAddress]*ClusterNode
|
filerGroup2filers map[FilerGroup]*Filers
|
||||||
filersLock sync.RWMutex
|
filersLock sync.RWMutex
|
||||||
filerLeaders *Leaders
|
|
||||||
brokers map[pb.ServerAddress]*ClusterNode
|
brokers map[pb.ServerAddress]*ClusterNode
|
||||||
brokersLock sync.RWMutex
|
brokersLock sync.RWMutex
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewCluster() *Cluster {
|
func NewCluster() *Cluster {
|
||||||
return &Cluster{
|
return &Cluster{
|
||||||
filers: make(map[pb.ServerAddress]*ClusterNode),
|
filerGroup2filers: make(map[FilerGroup]*Filers),
|
||||||
filerLeaders: &Leaders{},
|
|
||||||
brokers: make(map[pb.ServerAddress]*ClusterNode),
|
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 {
|
||||||
switch nodeType {
|
|
||||||
case FilerType:
|
|
||||||
cluster.filersLock.Lock()
|
cluster.filersLock.Lock()
|
||||||
defer cluster.filersLock.Unlock()
|
defer cluster.filersLock.Unlock()
|
||||||
if existingNode, found := cluster.filers[address]; found {
|
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:
|
||||||
|
filers := cluster.getFilers(filerGroup, true)
|
||||||
|
if existingNode, found := filers.filers[address]; found {
|
||||||
existingNode.counter++
|
existingNode.counter++
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
cluster.filers[address] = &ClusterNode{
|
filers.filers[address] = &ClusterNode{
|
||||||
Address: address,
|
Address: address,
|
||||||
Version: version,
|
Version: version,
|
||||||
counter: 1,
|
counter: 1,
|
||||||
createdTs: time.Now(),
|
createdTs: time.Now(),
|
||||||
}
|
}
|
||||||
return cluster.ensureFilerLeaders(true, nodeType, address)
|
return cluster.ensureFilerLeaders(filers, true, filerGroup, nodeType, address)
|
||||||
case BrokerType:
|
case BrokerType:
|
||||||
cluster.brokersLock.Lock()
|
cluster.brokersLock.Lock()
|
||||||
defer cluster.brokersLock.Unlock()
|
defer cluster.brokersLock.Unlock()
|
||||||
|
@ -94,18 +111,21 @@ func (cluster *Cluster) AddClusterNode(nodeType string, address pb.ServerAddress
|
||||||
return nil
|
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 {
|
switch nodeType {
|
||||||
case FilerType:
|
case FilerType:
|
||||||
cluster.filersLock.Lock()
|
filers := cluster.getFilers(filerGroup, false)
|
||||||
defer cluster.filersLock.Unlock()
|
if filers == nil {
|
||||||
if existingNode, found := cluster.filers[address]; !found {
|
return nil
|
||||||
|
}
|
||||||
|
if existingNode, found := filers.filers[address]; !found {
|
||||||
return nil
|
return nil
|
||||||
} else {
|
} else {
|
||||||
existingNode.counter--
|
existingNode.counter--
|
||||||
if existingNode.counter <= 0 {
|
if existingNode.counter <= 0 {
|
||||||
delete(cluster.filers, address)
|
delete(filers.filers, address)
|
||||||
return cluster.ensureFilerLeaders(false, nodeType, address)
|
return cluster.ensureFilerLeaders(filers, false, filerGroup, nodeType, address)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
case BrokerType:
|
case BrokerType:
|
||||||
|
@ -142,12 +162,16 @@ func (cluster *Cluster) RemoveClusterNode(nodeType string, address pb.ServerAddr
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (cluster *Cluster) ListClusterNode(nodeType string) (nodes []*ClusterNode) {
|
func (cluster *Cluster) ListClusterNode(filerGroup FilerGroup, nodeType string) (nodes []*ClusterNode) {
|
||||||
switch nodeType {
|
switch nodeType {
|
||||||
case FilerType:
|
case FilerType:
|
||||||
|
filers := cluster.getFilers(filerGroup, false)
|
||||||
|
if filers == nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
cluster.filersLock.RLock()
|
cluster.filersLock.RLock()
|
||||||
defer cluster.filersLock.RUnlock()
|
defer cluster.filersLock.RUnlock()
|
||||||
for _, node := range cluster.filers {
|
for _, node := range filers.filers {
|
||||||
nodes = append(nodes, node)
|
nodes = append(nodes, node)
|
||||||
}
|
}
|
||||||
case BrokerType:
|
case BrokerType:
|
||||||
|
@ -161,16 +185,21 @@ func (cluster *Cluster) ListClusterNode(nodeType string) (nodes []*ClusterNode)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
func (cluster *Cluster) IsOneLeader(address pb.ServerAddress) bool {
|
func (cluster *Cluster) IsOneLeader(filerGroup FilerGroup, address pb.ServerAddress) bool {
|
||||||
return cluster.filerLeaders.isOneLeader(address)
|
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 isAdd {
|
||||||
if cluster.filerLeaders.addLeaderIfVacant(address) {
|
if filers.leaders.addLeaderIfVacant(address) {
|
||||||
// has added the address as one leader
|
// has added the address as one leader
|
||||||
result = append(result, &master_pb.KeepConnectedResponse{
|
result = append(result, &master_pb.KeepConnectedResponse{
|
||||||
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
||||||
|
FilerGroup: string(filerGroup),
|
||||||
NodeType: nodeType,
|
NodeType: nodeType,
|
||||||
Address: string(address),
|
Address: string(address),
|
||||||
IsLeader: true,
|
IsLeader: true,
|
||||||
|
@ -180,6 +209,7 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
|
||||||
} else {
|
} else {
|
||||||
result = append(result, &master_pb.KeepConnectedResponse{
|
result = append(result, &master_pb.KeepConnectedResponse{
|
||||||
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
||||||
|
FilerGroup: string(filerGroup),
|
||||||
NodeType: nodeType,
|
NodeType: nodeType,
|
||||||
Address: string(address),
|
Address: string(address),
|
||||||
IsLeader: false,
|
IsLeader: false,
|
||||||
|
@ -188,10 +218,11 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if cluster.filerLeaders.removeLeaderIfExists(address) {
|
if filers.leaders.removeLeaderIfExists(address) {
|
||||||
|
|
||||||
result = append(result, &master_pb.KeepConnectedResponse{
|
result = append(result, &master_pb.KeepConnectedResponse{
|
||||||
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
||||||
|
FilerGroup: string(filerGroup),
|
||||||
NodeType: nodeType,
|
NodeType: nodeType,
|
||||||
Address: string(address),
|
Address: string(address),
|
||||||
IsLeader: true,
|
IsLeader: true,
|
||||||
|
@ -203,8 +234,8 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
|
||||||
var shortestDuration int64 = math.MaxInt64
|
var shortestDuration int64 = math.MaxInt64
|
||||||
now := time.Now()
|
now := time.Now()
|
||||||
var candidateAddress pb.ServerAddress
|
var candidateAddress pb.ServerAddress
|
||||||
for _, node := range cluster.filers {
|
for _, node := range filers.filers {
|
||||||
if cluster.filerLeaders.isOneLeader(node.Address) {
|
if filers.leaders.isOneLeader(node.Address) {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
duration := now.Sub(node.createdTs).Nanoseconds()
|
duration := now.Sub(node.createdTs).Nanoseconds()
|
||||||
|
@ -214,7 +245,7 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if candidateAddress != "" {
|
if candidateAddress != "" {
|
||||||
cluster.filerLeaders.addLeaderIfVacant(candidateAddress)
|
filers.leaders.addLeaderIfVacant(candidateAddress)
|
||||||
// added a new leader
|
// added a new leader
|
||||||
result = append(result, &master_pb.KeepConnectedResponse{
|
result = append(result, &master_pb.KeepConnectedResponse{
|
||||||
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
||||||
|
@ -228,6 +259,7 @@ func (cluster *Cluster) ensureFilerLeaders(isAdd bool, nodeType string, address
|
||||||
} else {
|
} else {
|
||||||
result = append(result, &master_pb.KeepConnectedResponse{
|
result = append(result, &master_pb.KeepConnectedResponse{
|
||||||
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
ClusterNodeUpdate: &master_pb.ClusterNodeUpdate{
|
||||||
|
FilerGroup: string(filerGroup),
|
||||||
NodeType: nodeType,
|
NodeType: nodeType,
|
||||||
Address: string(address),
|
Address: string(address),
|
||||||
IsLeader: false,
|
IsLeader: false,
|
||||||
|
|
|
@ -129,7 +129,7 @@ func runBenchmark(cmd *Command, args []string) bool {
|
||||||
defer pprof.StopCPUProfile()
|
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()
|
go b.masterClient.KeepConnectedToMaster()
|
||||||
b.masterClient.WaitUntilConnected()
|
b.masterClient.WaitUntilConnected()
|
||||||
|
|
||||||
|
|
|
@ -37,6 +37,7 @@ type FilerOptions struct {
|
||||||
port *int
|
port *int
|
||||||
portGrpc *int
|
portGrpc *int
|
||||||
publicPort *int
|
publicPort *int
|
||||||
|
filerGroup *string
|
||||||
collection *string
|
collection *string
|
||||||
defaultReplicaPlacement *string
|
defaultReplicaPlacement *string
|
||||||
disableDirListing *bool
|
disableDirListing *bool
|
||||||
|
@ -59,6 +60,7 @@ type FilerOptions struct {
|
||||||
func init() {
|
func init() {
|
||||||
cmdFiler.Run = runFiler // break init cycle
|
cmdFiler.Run = runFiler // break init cycle
|
||||||
f.mastersString = cmdFiler.Flag.String("master", "localhost:9333", "comma-separated master servers")
|
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.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.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.")
|
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{
|
fs, nfs_err := weed_server.NewFilerServer(defaultMux, publicVolumeMux, &weed_server.FilerOption{
|
||||||
Masters: fo.masters,
|
Masters: fo.masters,
|
||||||
|
FilerGroup: *fo.filerGroup,
|
||||||
Collection: *fo.collection,
|
Collection: *fo.collection,
|
||||||
DefaultReplication: *fo.defaultReplicaPlacement,
|
DefaultReplication: *fo.defaultReplicaPlacement,
|
||||||
DisableDirListing: *fo.disableDirListing,
|
DisableDirListing: *fo.disableDirListing,
|
||||||
|
|
|
@ -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.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")
|
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.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.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")
|
filerOptions.portGrpc = cmdServer.Flag.Int("filer.port.grpc", 0, "filer server grpc listen port")
|
||||||
|
|
|
@ -18,6 +18,7 @@ var (
|
||||||
func init() {
|
func init() {
|
||||||
cmdShell.Run = runShell // break init cycle
|
cmdShell.Run = runShell // break init cycle
|
||||||
shellOptions.Masters = cmdShell.Flag.String("master", "", "comma-separated master servers, e.g. localhost:9333")
|
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")
|
shellInitialFiler = cmdShell.Flag.String("filer", "", "filer host and port, e.g. localhost:8888")
|
||||||
shellCluster = cmdShell.Flag.String("cluster", "", "cluster defined in shell.toml")
|
shellCluster = cmdShell.Flag.String("cluster", "", "cluster defined in shell.toml")
|
||||||
}
|
}
|
||||||
|
|
|
@ -49,10 +49,10 @@ type Filer struct {
|
||||||
UniqueFileId uint32
|
UniqueFileId uint32
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewFiler(masters map[string]pb.ServerAddress, grpcDialOption grpc.DialOption,
|
func NewFiler(masters map[string]pb.ServerAddress, grpcDialOption grpc.DialOption, filerHost pb.ServerAddress,
|
||||||
filerHost pb.ServerAddress, collection string, replication string, dataCenter string, notifyFn func()) *Filer {
|
filerGroup string, collection string, replication string, dataCenter string, notifyFn func()) *Filer {
|
||||||
f := &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(),
|
fileIdDeletionQueue: util.NewUnboundedQueue(),
|
||||||
GrpcDialOption: grpcDialOption,
|
GrpcDialOption: grpcDialOption,
|
||||||
FilerConf: NewFilerConf(),
|
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 {
|
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{
|
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
|
||||||
ClientType: cluster.FilerType,
|
ClientType: cluster.FilerType,
|
||||||
|
FilerGroup: f.MasterClient.FilerGroup,
|
||||||
})
|
})
|
||||||
|
|
||||||
glog.V(0).Infof("the cluster has %d filers\n", len(resp.ClusterNodes))
|
glog.V(0).Infof("the cluster has %d filers\n", len(resp.ClusterNodes))
|
||||||
|
|
|
@ -12,7 +12,7 @@ import (
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestCreateAndFind(t *testing.T) {
|
func TestCreateAndFind(t *testing.T) {
|
||||||
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
|
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
|
||||||
dir := t.TempDir()
|
dir := t.TempDir()
|
||||||
store := &LevelDBStore{}
|
store := &LevelDBStore{}
|
||||||
store.initialize(dir)
|
store.initialize(dir)
|
||||||
|
@ -65,7 +65,7 @@ func TestCreateAndFind(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestEmptyRoot(t *testing.T) {
|
func TestEmptyRoot(t *testing.T) {
|
||||||
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
|
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
|
||||||
dir := t.TempDir()
|
dir := t.TempDir()
|
||||||
store := &LevelDBStore{}
|
store := &LevelDBStore{}
|
||||||
store.initialize(dir)
|
store.initialize(dir)
|
||||||
|
@ -87,7 +87,7 @@ func TestEmptyRoot(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func BenchmarkInsertEntry(b *testing.B) {
|
func BenchmarkInsertEntry(b *testing.B) {
|
||||||
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
|
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
|
||||||
dir := b.TempDir()
|
dir := b.TempDir()
|
||||||
store := &LevelDBStore{}
|
store := &LevelDBStore{}
|
||||||
store.initialize(dir)
|
store.initialize(dir)
|
||||||
|
|
|
@ -9,7 +9,7 @@ import (
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestCreateAndFind(t *testing.T) {
|
func TestCreateAndFind(t *testing.T) {
|
||||||
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
|
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
|
||||||
dir := t.TempDir()
|
dir := t.TempDir()
|
||||||
store := &LevelDB2Store{}
|
store := &LevelDB2Store{}
|
||||||
store.initialize(dir, 2)
|
store.initialize(dir, 2)
|
||||||
|
@ -62,7 +62,7 @@ func TestCreateAndFind(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestEmptyRoot(t *testing.T) {
|
func TestEmptyRoot(t *testing.T) {
|
||||||
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
|
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
|
||||||
dir := t.TempDir()
|
dir := t.TempDir()
|
||||||
store := &LevelDB2Store{}
|
store := &LevelDB2Store{}
|
||||||
store.initialize(dir, 2)
|
store.initialize(dir, 2)
|
||||||
|
|
|
@ -9,7 +9,7 @@ import (
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestCreateAndFind(t *testing.T) {
|
func TestCreateAndFind(t *testing.T) {
|
||||||
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
|
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
|
||||||
dir := t.TempDir()
|
dir := t.TempDir()
|
||||||
store := &LevelDB3Store{}
|
store := &LevelDB3Store{}
|
||||||
store.initialize(dir)
|
store.initialize(dir)
|
||||||
|
@ -62,7 +62,7 @@ func TestCreateAndFind(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestEmptyRoot(t *testing.T) {
|
func TestEmptyRoot(t *testing.T) {
|
||||||
testFiler := filer.NewFiler(nil, nil, "", "", "", "", nil)
|
testFiler := filer.NewFiler(nil, nil, "", "", "", "", "", nil)
|
||||||
dir := t.TempDir()
|
dir := t.TempDir()
|
||||||
store := &LevelDB3Store{}
|
store := &LevelDB3Store{}
|
||||||
store.initialize(dir)
|
store.initialize(dir)
|
||||||
|
|
|
@ -49,7 +49,7 @@ var s3ApiConfigure IamS3ApiConfig
|
||||||
func NewIamApiServer(router *mux.Router, option *IamServerOption) (iamApiServer *IamApiServer, err error) {
|
func NewIamApiServer(router *mux.Router, option *IamServerOption) (iamApiServer *IamApiServer, err error) {
|
||||||
s3ApiConfigure = IamS3ApiConfigure{
|
s3ApiConfigure = IamS3ApiConfigure{
|
||||||
option: option,
|
option: option,
|
||||||
masterClient: wdclient.NewMasterClient(option.GrpcDialOption, "iam", "", "", option.Masters),
|
masterClient: wdclient.NewMasterClient(option.GrpcDialOption, "", "iam", "", "", option.Masters),
|
||||||
}
|
}
|
||||||
s3Option := s3api.S3ApiServerOption{Filer: option.Filer}
|
s3Option := s3api.S3ApiServerOption{Filer: option.Filer}
|
||||||
iamApiServer = &IamApiServer{
|
iamApiServer = &IamApiServer{
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.28.0
|
// protoc-gen-go v1.26.0
|
||||||
// protoc v3.19.4
|
// protoc v3.17.3
|
||||||
// source: filer.proto
|
// source: filer.proto
|
||||||
|
|
||||||
package filer_pb
|
package filer_pb
|
||||||
|
|
|
@ -11,6 +11,7 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.32.0 or later.
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// SeaweedFilerClient is the client API for SeaweedFiler service.
|
// SeaweedFilerClient is the client API for SeaweedFiler service.
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.28.0
|
// protoc-gen-go v1.26.0
|
||||||
// protoc v3.19.4
|
// protoc v3.17.3
|
||||||
// source: iam.proto
|
// source: iam.proto
|
||||||
|
|
||||||
package iam_pb
|
package iam_pb
|
||||||
|
|
|
@ -8,6 +8,7 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.32.0 or later.
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// SeaweedIdentityAccessManagementClient is the client API for SeaweedIdentityAccessManagement service.
|
// SeaweedIdentityAccessManagementClient is the client API for SeaweedIdentityAccessManagement service.
|
||||||
|
|
|
@ -137,6 +137,7 @@ message KeepConnectedRequest {
|
||||||
string client_type = 1;
|
string client_type = 1;
|
||||||
string client_address = 3;
|
string client_address = 3;
|
||||||
string version = 4;
|
string version = 4;
|
||||||
|
string filer_group = 5;
|
||||||
}
|
}
|
||||||
|
|
||||||
message VolumeLocation {
|
message VolumeLocation {
|
||||||
|
@ -156,6 +157,7 @@ message ClusterNodeUpdate {
|
||||||
string address = 2;
|
string address = 2;
|
||||||
bool is_leader = 3;
|
bool is_leader = 3;
|
||||||
bool is_add = 4;
|
bool is_add = 4;
|
||||||
|
string filer_group = 5;
|
||||||
}
|
}
|
||||||
|
|
||||||
message KeepConnectedResponse {
|
message KeepConnectedResponse {
|
||||||
|
@ -310,6 +312,7 @@ message GetMasterConfigurationResponse {
|
||||||
|
|
||||||
message ListClusterNodesRequest {
|
message ListClusterNodesRequest {
|
||||||
string client_type = 1;
|
string client_type = 1;
|
||||||
|
string filer_group = 2;
|
||||||
}
|
}
|
||||||
message ListClusterNodesResponse {
|
message ListClusterNodesResponse {
|
||||||
message ClusterNode {
|
message ClusterNode {
|
||||||
|
|
File diff suppressed because it is too large
Load diff
|
@ -11,6 +11,7 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.32.0 or later.
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// SeaweedClient is the client API for Seaweed service.
|
// SeaweedClient is the client API for Seaweed service.
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.28.0
|
// protoc-gen-go v1.26.0
|
||||||
// protoc v3.19.4
|
// protoc v3.17.3
|
||||||
// source: messaging.proto
|
// source: messaging.proto
|
||||||
|
|
||||||
package messaging_pb
|
package messaging_pb
|
||||||
|
|
|
@ -11,6 +11,7 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.32.0 or later.
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// SeaweedMessagingClient is the client API for SeaweedMessaging service.
|
// SeaweedMessagingClient is the client API for SeaweedMessaging service.
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.28.0
|
// protoc-gen-go v1.26.0
|
||||||
// protoc v3.19.4
|
// protoc v3.17.3
|
||||||
// source: mount.proto
|
// source: mount.proto
|
||||||
|
|
||||||
package mount_pb
|
package mount_pb
|
||||||
|
|
|
@ -11,6 +11,7 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.32.0 or later.
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// SeaweedMountClient is the client API for SeaweedMount service.
|
// SeaweedMountClient is the client API for SeaweedMount service.
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.28.0
|
// protoc-gen-go v1.26.0
|
||||||
// protoc v3.19.4
|
// protoc v3.17.3
|
||||||
// source: remote.proto
|
// source: remote.proto
|
||||||
|
|
||||||
package remote_pb
|
package remote_pb
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.28.0
|
// protoc-gen-go v1.26.0
|
||||||
// protoc v3.19.4
|
// protoc v3.17.3
|
||||||
// source: volume_server.proto
|
// source: volume_server.proto
|
||||||
|
|
||||||
package volume_server_pb
|
package volume_server_pb
|
||||||
|
|
|
@ -11,6 +11,7 @@ import (
|
||||||
|
|
||||||
// This is a compile-time assertion to ensure that this generated file
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
// is compatible with the grpc package it is being compiled against.
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.32.0 or later.
|
||||||
const _ = grpc.SupportPackageIsVersion7
|
const _ = grpc.SupportPackageIsVersion7
|
||||||
|
|
||||||
// VolumeServerClient is the client API for VolumeServer service.
|
// VolumeServerClient is the client API for VolumeServer service.
|
||||||
|
|
|
@ -50,6 +50,7 @@ import (
|
||||||
|
|
||||||
type FilerOption struct {
|
type FilerOption struct {
|
||||||
Masters map[string]pb.ServerAddress
|
Masters map[string]pb.ServerAddress
|
||||||
|
FilerGroup string
|
||||||
Collection string
|
Collection string
|
||||||
DefaultReplication string
|
DefaultReplication string
|
||||||
DisableDirListing bool
|
DisableDirListing bool
|
||||||
|
@ -118,7 +119,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
|
||||||
glog.Fatal("master list is required!")
|
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.listenersCond.Broadcast()
|
||||||
})
|
})
|
||||||
fs.filer.Cipher = option.Cipher
|
fs.filer.Cipher = option.Cipher
|
||||||
|
|
|
@ -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
|
// buffer by 1 so we don't end up getting stuck writing to stopChan forever
|
||||||
stopChan := make(chan bool, 1)
|
stopChan := make(chan bool, 1)
|
||||||
|
|
||||||
clientName, messageChan := ms.addClient(req.ClientType, peerAddress)
|
clientName, messageChan := ms.addClient(req.FilerGroup, req.ClientType, peerAddress)
|
||||||
for _, update := range ms.Cluster.AddClusterNode(req.ClientType, peerAddress, req.Version) {
|
for _, update := range ms.Cluster.AddClusterNode(req.FilerGroup, req.ClientType, peerAddress, req.Version) {
|
||||||
ms.broadcastToClients(update)
|
ms.broadcastToClients(update)
|
||||||
}
|
}
|
||||||
|
|
||||||
defer func() {
|
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.broadcastToClients(update)
|
||||||
}
|
}
|
||||||
ms.deleteClient(clientName)
|
ms.deleteClient(clientName)
|
||||||
|
@ -276,8 +276,8 @@ func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedSe
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ms *MasterServer) addClient(clientType string, clientAddress pb.ServerAddress) (clientName string, messageChan chan *master_pb.KeepConnectedResponse) {
|
func (ms *MasterServer) addClient(filerGroup, clientType string, clientAddress pb.ServerAddress) (clientName string, messageChan chan *master_pb.KeepConnectedResponse) {
|
||||||
clientName = clientType + "@" + string(clientAddress)
|
clientName = filerGroup + "." + clientType + "@" + string(clientAddress)
|
||||||
glog.V(0).Infof("+ client %v", clientName)
|
glog.V(0).Infof("+ client %v", clientName)
|
||||||
|
|
||||||
// we buffer this because otherwise we end up in a potential deadlock where
|
// we buffer this because otherwise we end up in a potential deadlock where
|
||||||
|
|
|
@ -10,26 +10,26 @@ import (
|
||||||
|
|
||||||
func (ms *MasterServer) ListClusterNodes(ctx context.Context, req *master_pb.ListClusterNodesRequest) (*master_pb.ListClusterNodesResponse, error) {
|
func (ms *MasterServer) ListClusterNodes(ctx context.Context, req *master_pb.ListClusterNodesRequest) (*master_pb.ListClusterNodesResponse, error) {
|
||||||
resp := &master_pb.ListClusterNodesResponse{}
|
resp := &master_pb.ListClusterNodesResponse{}
|
||||||
|
filerGroup := cluster.FilerGroup(req.FilerGroup)
|
||||||
clusterNodes := ms.Cluster.ListClusterNode(req.ClientType)
|
clusterNodes := ms.Cluster.ListClusterNode(filerGroup, req.ClientType)
|
||||||
|
|
||||||
for _, node := range clusterNodes {
|
for _, node := range clusterNodes {
|
||||||
resp.ClusterNodes = append(resp.ClusterNodes, &master_pb.ListClusterNodesResponse_ClusterNode{
|
resp.ClusterNodes = append(resp.ClusterNodes, &master_pb.ListClusterNodesResponse_ClusterNode{
|
||||||
Address: string(node.Address),
|
Address: string(node.Address),
|
||||||
Version: node.Version,
|
Version: node.Version,
|
||||||
IsLeader: ms.Cluster.IsOneLeader(node.Address),
|
IsLeader: ms.Cluster.IsOneLeader(filerGroup, node.Address),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
return resp, nil
|
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
|
var filers []pb.ServerAddress
|
||||||
for _, node := range clusterNodes {
|
for _, node := range clusterNodes {
|
||||||
if ms.Cluster.IsOneLeader(node.Address) {
|
if ms.Cluster.IsOneLeader(filerGroup, node.Address) {
|
||||||
filers = append(filers, node.Address)
|
filers = append(filers, node.Address)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -113,7 +113,7 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers map[string]pb.Se
|
||||||
vgCh: make(chan *topology.VolumeGrowRequest, 1<<6),
|
vgCh: make(chan *topology.VolumeGrowRequest, 1<<6),
|
||||||
clientChans: make(map[string]chan *master_pb.KeepConnectedResponse),
|
clientChans: make(map[string]chan *master_pb.KeepConnectedResponse),
|
||||||
grpcDialOption: grpcDialOption,
|
grpcDialOption: grpcDialOption,
|
||||||
MasterClient: wdclient.NewMasterClient(grpcDialOption, cluster.MasterType, option.Master, "", peers),
|
MasterClient: wdclient.NewMasterClient(grpcDialOption, "", cluster.MasterType, option.Master, "", peers),
|
||||||
adminLocks: NewAdminLocks(),
|
adminLocks: NewAdminLocks(),
|
||||||
Cluster: cluster.NewCluster(),
|
Cluster: cluster.NewCluster(),
|
||||||
}
|
}
|
||||||
|
@ -285,7 +285,7 @@ func (ms *MasterServer) startAdminScripts() {
|
||||||
for {
|
for {
|
||||||
time.Sleep(time.Duration(sleepMinutes) * time.Minute)
|
time.Sleep(time.Duration(sleepMinutes) * time.Minute)
|
||||||
if ms.Topo.IsLeader() {
|
if ms.Topo.IsLeader() {
|
||||||
shellOptions.FilerAddress = ms.GetOneFiler()
|
shellOptions.FilerAddress = ms.GetOneFiler(cluster.FilerGroup(*shellOptions.FilerGroup))
|
||||||
if shellOptions.FilerAddress == "" {
|
if shellOptions.FilerAddress == "" {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
|
@ -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 {
|
err = commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
|
||||||
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
|
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
|
||||||
ClientType: cluster.FilerType,
|
ClientType: cluster.FilerType,
|
||||||
|
FilerGroup: *commandEnv.option.FilerGroup,
|
||||||
})
|
})
|
||||||
|
|
||||||
for _, node := range resp.ClusterNodes {
|
for _, node := range resp.ClusterNodes {
|
||||||
|
|
|
@ -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 {
|
err = commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
|
||||||
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
|
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
|
||||||
ClientType: cluster.FilerType,
|
ClientType: cluster.FilerType,
|
||||||
|
FilerGroup: *commandEnv.option.FilerGroup,
|
||||||
})
|
})
|
||||||
|
|
||||||
fmt.Fprintf(writer, "the cluster has %d filers\n", len(resp.ClusterNodes))
|
fmt.Fprintf(writer, "the cluster has %d filers\n", len(resp.ClusterNodes))
|
||||||
|
|
|
@ -22,6 +22,7 @@ type ShellOptions struct {
|
||||||
// shell transient context
|
// shell transient context
|
||||||
FilerHost string
|
FilerHost string
|
||||||
FilerPort int64
|
FilerPort int64
|
||||||
|
FilerGroup *string
|
||||||
FilerAddress pb.ServerAddress
|
FilerAddress pb.ServerAddress
|
||||||
Directory string
|
Directory string
|
||||||
}
|
}
|
||||||
|
@ -46,7 +47,7 @@ var (
|
||||||
func NewCommandEnv(options *ShellOptions) *CommandEnv {
|
func NewCommandEnv(options *ShellOptions) *CommandEnv {
|
||||||
ce := &CommandEnv{
|
ce := &CommandEnv{
|
||||||
env: make(map[string]string),
|
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,
|
option: options,
|
||||||
}
|
}
|
||||||
ce.locker = exclusive_locks.NewExclusiveLocker(ce.MasterClient, "admin")
|
ce.locker = exclusive_locks.NewExclusiveLocker(ce.MasterClient, "admin")
|
||||||
|
|
|
@ -54,6 +54,7 @@ func RunShell(options ShellOptions) {
|
||||||
commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
|
commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
|
||||||
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
|
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
|
||||||
ClientType: cluster.FilerType,
|
ClientType: cluster.FilerType,
|
||||||
|
FilerGroup: *options.FilerGroup,
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
|
|
|
@ -15,6 +15,7 @@ import (
|
||||||
)
|
)
|
||||||
|
|
||||||
type MasterClient struct {
|
type MasterClient struct {
|
||||||
|
FilerGroup string
|
||||||
clientType string
|
clientType string
|
||||||
clientHost pb.ServerAddress
|
clientHost pb.ServerAddress
|
||||||
currentMaster pb.ServerAddress
|
currentMaster pb.ServerAddress
|
||||||
|
@ -26,8 +27,9 @@ type MasterClient struct {
|
||||||
OnPeerUpdate func(update *master_pb.ClusterNodeUpdate)
|
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{
|
return &MasterClient{
|
||||||
|
FilerGroup: filerGroup,
|
||||||
clientType: clientType,
|
clientType: clientType,
|
||||||
clientHost: clientHost,
|
clientHost: clientHost,
|
||||||
masters: masters,
|
masters: masters,
|
||||||
|
@ -53,7 +55,7 @@ func (mc *MasterClient) WaitUntilConnected() {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (mc *MasterClient) KeepConnectedToMaster() {
|
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 {
|
for {
|
||||||
mc.tryAllMasters()
|
mc.tryAllMasters()
|
||||||
time.Sleep(time.Second)
|
time.Sleep(time.Second)
|
||||||
|
@ -101,7 +103,7 @@ func (mc *MasterClient) tryAllMasters() {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedLeader pb.ServerAddress) {
|
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()
|
stats.MasterClientConnectCounter.WithLabelValues("total").Inc()
|
||||||
gprcErr := pb.WithMasterClient(true, master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
|
gprcErr := pb.WithMasterClient(true, master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
@ -109,28 +111,29 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
|
||||||
|
|
||||||
stream, err := client.KeepConnected(ctx)
|
stream, err := client.KeepConnected(ctx)
|
||||||
if err != nil {
|
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()
|
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToKeepConnected).Inc()
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
if err = stream.Send(&master_pb.KeepConnectedRequest{
|
if err = stream.Send(&master_pb.KeepConnectedRequest{
|
||||||
|
FilerGroup: mc.FilerGroup,
|
||||||
ClientType: mc.clientType,
|
ClientType: mc.clientType,
|
||||||
ClientAddress: string(mc.clientHost),
|
ClientAddress: string(mc.clientHost),
|
||||||
Version: util.Version(),
|
Version: util.Version(),
|
||||||
}); err != nil {
|
}); 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()
|
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToSend).Inc()
|
||||||
return err
|
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
|
mc.currentMaster = master
|
||||||
|
|
||||||
for {
|
for {
|
||||||
resp, err := stream.Recv()
|
resp, err := stream.Recv()
|
||||||
if err != nil {
|
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()
|
stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToReceive).Inc()
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -152,19 +155,19 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
|
||||||
GrpcPort: int(resp.VolumeLocation.GrpcPort),
|
GrpcPort: int(resp.VolumeLocation.GrpcPort),
|
||||||
}
|
}
|
||||||
for _, newVid := range resp.VolumeLocation.NewVids {
|
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)
|
mc.addLocation(newVid, loc)
|
||||||
}
|
}
|
||||||
for _, deletedVid := range resp.VolumeLocation.DeletedVids {
|
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)
|
mc.deleteLocation(deletedVid, loc)
|
||||||
}
|
}
|
||||||
for _, newEcVid := range resp.VolumeLocation.NewEcVids {
|
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)
|
mc.addEcLocation(newEcVid, loc)
|
||||||
}
|
}
|
||||||
for _, deletedEcVid := range resp.VolumeLocation.DeletedEcVids {
|
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)
|
mc.deleteEcLocation(deletedEcVid, loc)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -172,22 +175,24 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
|
||||||
if resp.ClusterNodeUpdate != nil {
|
if resp.ClusterNodeUpdate != nil {
|
||||||
update := resp.ClusterNodeUpdate
|
update := resp.ClusterNodeUpdate
|
||||||
if mc.OnPeerUpdate != nil {
|
if mc.OnPeerUpdate != nil {
|
||||||
|
if update.FilerGroup == mc.FilerGroup {
|
||||||
if update.IsAdd {
|
if update.IsAdd {
|
||||||
glog.V(0).Infof("+ %s %s leader:%v\n", update.NodeType, update.Address, update.IsLeader)
|
glog.V(0).Infof("+ %s.%s %s leader:%v\n", update.FilerGroup, update.NodeType, update.Address, update.IsLeader)
|
||||||
} else {
|
} else {
|
||||||
glog.V(0).Infof("- %s %s leader:%v\n", 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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
})
|
})
|
||||||
if gprcErr != nil {
|
if gprcErr != nil {
|
||||||
stats.MasterClientConnectCounter.WithLabelValues(stats.Failed).Inc()
|
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
|
return
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in a new issue