aboutsummaryrefslogtreecommitdiff
path: root/weed/wdclient
diff options
context:
space:
mode:
Diffstat (limited to 'weed/wdclient')
-rw-r--r--weed/wdclient/exclusive_locks/exclusive_locker.go111
-rw-r--r--weed/wdclient/masterclient.go140
-rw-r--r--weed/wdclient/vid_map.go59
-rw-r--r--weed/wdclient/vid_map_test.go76
-rw-r--r--weed/wdclient/wdclient.go15
5 files changed, 311 insertions, 90 deletions
diff --git a/weed/wdclient/exclusive_locks/exclusive_locker.go b/weed/wdclient/exclusive_locks/exclusive_locker.go
new file mode 100644
index 000000000..1ecfe6ce2
--- /dev/null
+++ b/weed/wdclient/exclusive_locks/exclusive_locker.go
@@ -0,0 +1,111 @@
+package exclusive_locks
+
+import (
+ "context"
+ "sync/atomic"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/wdclient"
+)
+
+const (
+ RenewInteval = 4 * time.Second
+ SafeRenewInteval = 3 * time.Second
+ InitLockInteval = 1 * time.Second
+ AdminLockName = "admin"
+)
+
+type ExclusiveLocker struct {
+ masterClient *wdclient.MasterClient
+ token int64
+ lockTsNs int64
+ isLocking bool
+}
+
+func NewExclusiveLocker(masterClient *wdclient.MasterClient) *ExclusiveLocker {
+ return &ExclusiveLocker{
+ masterClient: masterClient,
+ }
+}
+func (l *ExclusiveLocker) IsLocking() bool {
+ return l.isLocking
+}
+
+func (l *ExclusiveLocker) GetToken() (token int64, lockTsNs int64) {
+ for time.Unix(0, atomic.LoadInt64(&l.lockTsNs)).Add(SafeRenewInteval).Before(time.Now()) {
+ // wait until now is within the safe lock period, no immediate renewal to change the token
+ time.Sleep(100 * time.Millisecond)
+ }
+ return atomic.LoadInt64(&l.token), atomic.LoadInt64(&l.lockTsNs)
+}
+
+func (l *ExclusiveLocker) RequestLock() {
+ if l.isLocking {
+ return
+ }
+
+ // retry to get the lease
+ for {
+ if err := l.masterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err := client.LeaseAdminToken(context.Background(), &master_pb.LeaseAdminTokenRequest{
+ PreviousToken: atomic.LoadInt64(&l.token),
+ PreviousLockTime: atomic.LoadInt64(&l.lockTsNs),
+ LockName: AdminLockName,
+ })
+ if err == nil {
+ atomic.StoreInt64(&l.token, resp.Token)
+ atomic.StoreInt64(&l.lockTsNs, resp.LockTsNs)
+ }
+ return err
+ }); err != nil {
+ // println("leasing problem", err.Error())
+ time.Sleep(InitLockInteval)
+ } else {
+ break
+ }
+ }
+
+ l.isLocking = true
+
+ // start a goroutine to renew the lease
+ go func() {
+ for l.isLocking {
+ if err := l.masterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err := client.LeaseAdminToken(context.Background(), &master_pb.LeaseAdminTokenRequest{
+ PreviousToken: atomic.LoadInt64(&l.token),
+ PreviousLockTime: atomic.LoadInt64(&l.lockTsNs),
+ LockName: AdminLockName,
+ })
+ if err == nil {
+ atomic.StoreInt64(&l.token, resp.Token)
+ atomic.StoreInt64(&l.lockTsNs, resp.LockTsNs)
+ // println("ts", l.lockTsNs, "token", l.token)
+ }
+ return err
+ }); err != nil {
+ glog.Errorf("failed to renew lock: %v", err)
+ return
+ } else {
+ time.Sleep(RenewInteval)
+ }
+
+ }
+ }()
+
+}
+
+func (l *ExclusiveLocker) ReleaseLock() {
+ l.isLocking = false
+ l.masterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ client.ReleaseAdminToken(context.Background(), &master_pb.ReleaseAdminTokenRequest{
+ PreviousToken: atomic.LoadInt64(&l.token),
+ PreviousLockTime: atomic.LoadInt64(&l.lockTsNs),
+ LockName: AdminLockName,
+ })
+ return nil
+ })
+ atomic.StoreInt64(&l.token, 0)
+ atomic.StoreInt64(&l.lockTsNs, 0)
+}
diff --git a/weed/wdclient/masterclient.go b/weed/wdclient/masterclient.go
index f58c28504..4c066d535 100644
--- a/weed/wdclient/masterclient.go
+++ b/weed/wdclient/masterclient.go
@@ -2,30 +2,35 @@ package wdclient
import (
"context"
- "fmt"
+ "math/rand"
"time"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
- "github.com/chrislusf/seaweedfs/weed/util"
- "math/rand"
)
type MasterClient struct {
- ctx context.Context
- name string
- currentMaster string
- masters []string
+ clientType string
+ clientHost string
+ grpcPort uint32
+ currentMaster string
+ masters []string
+ grpcDialOption grpc.DialOption
vidMap
}
-func NewMasterClient(ctx context.Context, clientName string, masters []string) *MasterClient {
+func NewMasterClient(grpcDialOption grpc.DialOption, clientType string, clientHost string, clientGrpcPort uint32, masters []string) *MasterClient {
return &MasterClient{
- ctx: ctx,
- name: clientName,
- masters: masters,
- vidMap: newVidMap(),
+ clientType: clientType,
+ clientHost: clientHost,
+ grpcPort: clientGrpcPort,
+ masters: masters,
+ grpcDialOption: grpcDialOption,
+ vidMap: newVidMap(),
}
}
@@ -40,7 +45,7 @@ func (mc *MasterClient) WaitUntilConnected() {
}
func (mc *MasterClient) KeepConnectedToMaster() {
- glog.V(0).Infof("%s bootstraps with masters %v", mc.name, mc.masters)
+ glog.V(1).Infof("%s masterClient bootstraps with masters %v", mc.clientType, mc.masters)
for {
mc.tryAllMasters()
time.Sleep(time.Second)
@@ -48,69 +53,78 @@ func (mc *MasterClient) KeepConnectedToMaster() {
}
func (mc *MasterClient) tryAllMasters() {
+ nextHintedLeader := ""
for _, master := range mc.masters {
- glog.V(0).Infof("Connecting to master %v", master)
- gprcErr := withMasterClient(master, func(client master_pb.SeaweedClient) error {
- stream, err := client.KeepConnected(context.Background())
+ nextHintedLeader = mc.tryConnectToMaster(master)
+ for nextHintedLeader != "" {
+ nextHintedLeader = mc.tryConnectToMaster(nextHintedLeader)
+ }
+
+ mc.currentMaster = ""
+ mc.vidMap = newVidMap()
+ }
+}
+
+func (mc *MasterClient) tryConnectToMaster(master string) (nextHintedLeader string) {
+ glog.V(1).Infof("%s masterClient Connecting to master %v", mc.clientType, master)
+ gprcErr := pb.WithMasterClient(master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
+
+ stream, err := client.KeepConnected(context.Background())
+ if err != nil {
+ glog.V(0).Infof("%s masterClient failed to keep connected to %s: %v", mc.clientType, master, err)
+ return err
+ }
+
+ if err = stream.Send(&master_pb.KeepConnectedRequest{Name: mc.clientType, GrpcPort: mc.grpcPort}); err != nil {
+ glog.V(0).Infof("%s masterClient failed to send to %s: %v", mc.clientType, master, err)
+ return err
+ }
+
+ glog.V(1).Infof("%s masterClient Connected to %v", mc.clientType, master)
+ mc.currentMaster = master
+
+ for {
+ volumeLocation, err := stream.Recv()
if err != nil {
- glog.V(0).Infof("failed to keep connected to %s: %v", master, err)
+ glog.V(0).Infof("%s masterClient failed to receive from %s: %v", mc.clientType, master, err)
return err
}
- if err = stream.Send(&master_pb.ClientListenRequest{Name: mc.name}); err != nil {
- glog.V(0).Infof("failed to send to %s: %v", master, err)
- return err
+ // maybe the leader is changed
+ if volumeLocation.Leader != "" {
+ glog.V(0).Infof("redirected to leader %v", volumeLocation.Leader)
+ nextHintedLeader = volumeLocation.Leader
+ return nil
}
- for {
- if volumeLocation, err := stream.Recv(); err != nil {
- glog.V(0).Infof("failed to receive from %s: %v", master, err)
- return err
- } else {
- loc := Location{
- Url: volumeLocation.Url,
- PublicUrl: volumeLocation.PublicUrl,
- }
- for _, newVid := range volumeLocation.NewVids {
- mc.addLocation(newVid, loc)
- }
- for _, deletedVid := range volumeLocation.DeletedVids {
- mc.deleteLocation(deletedVid, loc)
- }
-
- if mc.currentMaster == "" {
- glog.V(0).Infof("Connected to %v", master)
- mc.currentMaster = master
- }
-
- }
+ // process new volume location
+ loc := Location{
+ Url: volumeLocation.Url,
+ PublicUrl: volumeLocation.PublicUrl,
+ }
+ for _, newVid := range volumeLocation.NewVids {
+ glog.V(1).Infof("%s: %s masterClient adds volume %d", mc.clientType, loc.Url, newVid)
+ mc.addLocation(newVid, loc)
+ }
+ for _, deletedVid := range volumeLocation.DeletedVids {
+ glog.V(1).Infof("%s: %s masterClient removes volume %d", mc.clientType, loc.Url, deletedVid)
+ mc.deleteLocation(deletedVid, loc)
}
-
- })
-
- if gprcErr != nil {
- glog.V(0).Infof("%s failed to connect with master %v: %v", mc.name, master, gprcErr)
}
- mc.currentMaster = ""
+ })
+ if gprcErr != nil {
+ glog.V(0).Infof("%s masterClient failed to connect with master %v: %v", mc.clientType, master, gprcErr)
}
+ return
}
-func withMasterClient(master string, fn func(client master_pb.SeaweedClient) error) error {
-
- masterGrpcAddress, parseErr := util.ParseServerToGrpcAddress(master, 0)
- if parseErr != nil {
- return fmt.Errorf("failed to parse master grpc %v", master)
- }
-
- grpcConnection, err := util.GrpcDial(masterGrpcAddress)
- if err != nil {
- return fmt.Errorf("fail to dial %s: %v", master, err)
+func (mc *MasterClient) WithClient(fn func(client master_pb.SeaweedClient) error) error {
+ for mc.currentMaster == "" {
+ time.Sleep(3 * time.Second)
}
- defer grpcConnection.Close()
-
- client := master_pb.NewSeaweedClient(grpcConnection)
-
- return fn(client)
+ return pb.WithMasterClient(mc.currentMaster, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
+ return fn(client)
+ })
}
diff --git a/weed/wdclient/vid_map.go b/weed/wdclient/vid_map.go
index aef29f56f..97df49cb6 100644
--- a/weed/wdclient/vid_map.go
+++ b/weed/wdclient/vid_map.go
@@ -3,14 +3,18 @@ package wdclient
import (
"errors"
"fmt"
- "math/rand"
"strconv"
"strings"
"sync"
+ "sync/atomic"
"github.com/chrislusf/seaweedfs/weed/glog"
)
+const (
+ maxCursorIndex = 4096
+)
+
type Location struct {
Url string `json:"url,omitempty"`
PublicUrl string `json:"publicUrl,omitempty"`
@@ -19,14 +23,27 @@ type Location struct {
type vidMap struct {
sync.RWMutex
vid2Locations map[uint32][]Location
+
+ cursor int32
}
func newVidMap() vidMap {
return vidMap{
vid2Locations: make(map[uint32][]Location),
+ cursor: -1,
}
}
+func (vc *vidMap) getLocationIndex(length int) (int, error) {
+ if length <= 0 {
+ return 0, fmt.Errorf("invalid length: %d", length)
+ }
+ if atomic.LoadInt32(&vc.cursor) == maxCursorIndex {
+ atomic.CompareAndSwapInt32(&vc.cursor, maxCursorIndex, -1)
+ }
+ return int(atomic.AddInt32(&vc.cursor, 1)) % length, nil
+}
+
func (vc *vidMap) LookupVolumeServerUrl(vid string) (serverUrl string, err error) {
id, err := strconv.Atoi(vid)
if err != nil {
@@ -34,12 +51,7 @@ func (vc *vidMap) LookupVolumeServerUrl(vid string) (serverUrl string, err error
return "", err
}
- locations := vc.GetLocations(uint32(id))
- if len(locations) == 0 {
- return "", fmt.Errorf("volume %d not found", id)
- }
-
- return locations[rand.Intn(len(locations))].Url, nil
+ return vc.GetRandomLocation(uint32(id))
}
func (vc *vidMap) LookupFileId(fileId string) (fullUrl string, err error) {
@@ -66,20 +78,42 @@ func (vc *vidMap) LookupVolumeServer(fileId string) (volumeServer string, err er
return serverUrl, nil
}
-func (vc *vidMap) GetVidLocations(vid string) (locations []Location) {
+func (vc *vidMap) GetVidLocations(vid string) (locations []Location, err error) {
id, err := strconv.Atoi(vid)
if err != nil {
glog.V(1).Infof("Unknown volume id %s", vid)
- return nil
+ return nil, fmt.Errorf("Unknown volume id %s", vid)
+ }
+ foundLocations, found := vc.GetLocations(uint32(id))
+ if found {
+ return foundLocations, nil
}
- return vc.GetLocations(uint32(id))
+ return nil, fmt.Errorf("volume id %s not found", vid)
}
-func (vc *vidMap) GetLocations(vid uint32) (locations []Location) {
+func (vc *vidMap) GetLocations(vid uint32) (locations []Location, found bool) {
vc.RLock()
defer vc.RUnlock()
- return vc.vid2Locations[vid]
+ locations, found = vc.vid2Locations[vid]
+ return
+}
+
+func (vc *vidMap) GetRandomLocation(vid uint32) (serverUrl string, err error) {
+ vc.RLock()
+ defer vc.RUnlock()
+
+ locations := vc.vid2Locations[vid]
+ if len(locations) == 0 {
+ return "", fmt.Errorf("volume %d not found", vid)
+ }
+
+ index, err := vc.getLocationIndex(len(locations))
+ if err != nil {
+ return "", fmt.Errorf("volume %d: %v", vid, err)
+ }
+
+ return locations[index].Url, nil
}
func (vc *vidMap) addLocation(vid uint32, location Location) {
@@ -114,6 +148,7 @@ func (vc *vidMap) deleteLocation(vid uint32, location Location) {
for i, loc := range locations {
if loc.Url == location.Url {
vc.vid2Locations[vid] = append(locations[0:i], locations[i+1:]...)
+ break
}
}
diff --git a/weed/wdclient/vid_map_test.go b/weed/wdclient/vid_map_test.go
new file mode 100644
index 000000000..87be2fc25
--- /dev/null
+++ b/weed/wdclient/vid_map_test.go
@@ -0,0 +1,76 @@
+package wdclient
+
+import (
+ "fmt"
+ "testing"
+)
+
+func TestLocationIndex(t *testing.T) {
+ vm := vidMap{}
+ // test must be failed
+ mustFailed := func(length int) {
+ _, err := vm.getLocationIndex(length)
+ if err == nil {
+ t.Errorf("length %d must be failed", length)
+ }
+ if err.Error() != fmt.Sprintf("invalid length: %d", length) {
+ t.Errorf("length %d must be failed. error: %v", length, err)
+ }
+ }
+
+ mustFailed(-1)
+ mustFailed(0)
+
+ mustOk := func(length, cursor, expect int) {
+ if length <= 0 {
+ t.Fatal("please don't do this")
+ }
+ vm.cursor = int32(cursor)
+ got, err := vm.getLocationIndex(length)
+ if err != nil {
+ t.Errorf("length: %d, why? %v\n", length, err)
+ return
+ }
+ if got != expect {
+ t.Errorf("cursor: %d, length: %d, expect: %d, got: %d\n", cursor, length, expect, got)
+ return
+ }
+ }
+
+ for i := -1; i < 100; i++ {
+ mustOk(7, i, (i+1)%7)
+ }
+
+ // when cursor reaches MaxInt64
+ mustOk(7, maxCursorIndex, 0)
+
+ // test with constructor
+ vm = newVidMap()
+ length := 7
+ for i := 0; i < 100; i++ {
+ got, err := vm.getLocationIndex(length)
+ if err != nil {
+ t.Errorf("length: %d, why? %v\n", length, err)
+ return
+ }
+ if got != i%length {
+ t.Errorf("length: %d, i: %d, got: %d\n", length, i, got)
+ }
+ }
+}
+
+func BenchmarkLocationIndex(b *testing.B) {
+ b.SetParallelism(8)
+ vm := vidMap{
+ cursor: maxCursorIndex - 4000,
+ }
+ b.ResetTimer()
+ b.RunParallel(func(pb *testing.PB) {
+ for pb.Next() {
+ _, err := vm.getLocationIndex(3)
+ if err != nil {
+ b.Error(err)
+ }
+ }
+ })
+}
diff --git a/weed/wdclient/wdclient.go b/weed/wdclient/wdclient.go
deleted file mode 100644
index 722f4d061..000000000
--- a/weed/wdclient/wdclient.go
+++ /dev/null
@@ -1,15 +0,0 @@
-package wdclient
-
-import (
- "context"
-)
-
-type SeaweedClient struct {
- *MasterClient
-}
-
-func NewSeaweedClient(ctx context.Context, clientName string, masters []string) *SeaweedClient {
- return &SeaweedClient{
- MasterClient: NewMasterClient(ctx, clientName, masters),
- }
-}