1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-07-04 08:06:53 +02:00

caching ec shard locations

This commit is contained in:
Chris Lu 2019-05-27 22:54:58 -07:00
parent 3a8c1055a2
commit 713596e781
8 changed files with 45 additions and 22 deletions

View file

@ -43,7 +43,7 @@ func Assign(server string, grpcDialOption grpc.DialOption, primaryRequest *Volum
continue
}
lastError = withMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
lastError = WithMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
req := &master_pb.AssignRequest{
Count: primaryRequest.Count,

View file

@ -38,7 +38,7 @@ func toVolumeServerGrpcAddress(volumeServer string) (grpcAddress string, err err
return fmt.Sprintf("%s:%d", volumeServer[0:sepIndex], port+10000), nil
}
func withMasterServerClient(masterServer string, grpcDialOption grpc.DialOption, fn func(masterClient master_pb.SeaweedClient) error) error {
func WithMasterServerClient(masterServer string, grpcDialOption grpc.DialOption, fn func(masterClient master_pb.SeaweedClient) error) error {
ctx := context.Background()

View file

@ -99,7 +99,7 @@ func LookupVolumeIds(server string, grpcDialOption grpc.DialOption, vids []strin
//only query unknown_vids
err := withMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
err := WithMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
req := &master_pb.LookupVolumeRequest{
VolumeIds: unknown_vids,

View file

@ -9,7 +9,7 @@ import (
func Statistics(server string, grpcDialOption grpc.DialOption, req *master_pb.StatisticsRequest) (resp *master_pb.StatisticsResponse, err error) {
err = withMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
err = WithMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
grpcResponse, grpcErr := masterClient.Statistics(context.Background(), req)
if grpcErr != nil {

View file

@ -39,13 +39,13 @@ func (vs *VolumeServer) heartbeat() {
glog.V(0).Infof("failed to parse master grpc %v: %v", masterGrpcAddress, parseErr)
continue
}
vs.store.MasterGrpcAddress = masterGrpcAddress
vs.store.MasterAddress = master
newLeader, err = vs.doHeartbeat(context.Background(), master, masterGrpcAddress, grpcDialOption, time.Duration(vs.pulseSeconds)*time.Second)
if err != nil {
glog.V(0).Infof("heartbeat error: %v", err)
time.Sleep(time.Duration(vs.pulseSeconds) * time.Second)
newLeader = ""
vs.store.MasterGrpcAddress = ""
vs.store.MasterAddress = ""
}
}
}

View file

@ -5,6 +5,8 @@ import (
"math"
"os"
"sort"
"sync"
"time"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
@ -13,12 +15,15 @@ import (
)
type EcVolume struct {
Shards []*EcVolumeShard
VolumeId needle.VolumeId
Collection string
dir string
ecxFile *os.File
ecxFileSize int64
VolumeId needle.VolumeId
Collection string
dir string
ecxFile *os.File
ecxFileSize int64
Shards []*EcVolumeShard
ShardLocations map[ShardId][]string
ShardLocationsRefreshTime time.Time
ShardLocationsLock sync.RWMutex
}
func NewEcVolume(dir string, collection string, vid needle.VolumeId) (ev *EcVolume, err error) {

View file

@ -19,7 +19,7 @@ const (
* A VolumeServer contains one Store
*/
type Store struct {
MasterGrpcAddress string
MasterAddress string
grpcDialOption grpc.DialOption
volumeSizeLimit uint64 //read from the master
Ip string

View file

@ -4,6 +4,7 @@ import (
"context"
"fmt"
"io"
"time"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
@ -119,13 +120,13 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
}
func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, err error) {
shardLocations, err := s.cachedLookupEcShardLocations(ctx, vid)
if err != nil {
return nil, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterGrpcAddress, err)
if err = s.cachedLookupEcShardLocations(ctx, ecVolume); err != nil {
return nil, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterAddress, err)
}
for i, interval := range intervals {
if d, e := s.readOneEcShardInterval(ctx, vid, ecVolume, shardLocations, interval); e != nil {
if d, e := s.readOneEcShardInterval(ctx, ecVolume, interval); e != nil {
return nil, e
} else {
if i == 0 {
@ -138,7 +139,7 @@ func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, e
return
}
func (s *Store) readOneEcShardInterval(ctx context.Context, vid needle.VolumeId, ecVolume *erasure_coding.EcVolume, shardLocations map[erasure_coding.ShardId]string, interval erasure_coding.Interval) (data []byte, err error) {
func (s *Store) readOneEcShardInterval(ctx context.Context, ecVolume *erasure_coding.EcVolume, interval erasure_coding.Interval) (data []byte, err error) {
shardId, actualOffset := interval.ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
data = make([]byte, interval.Size)
if shard, found := ecVolume.FindEcVolumeShard(shardId); found {
@ -146,16 +147,33 @@ func (s *Store) readOneEcShardInterval(ctx context.Context, vid needle.VolumeId,
return
}
} else {
sourceDataNode := shardLocations[shardId]
_, err = s.readOneRemoteEcShardInterval(ctx, sourceDataNode, vid, shardId, data, actualOffset)
ecVolume.ShardLocationsLock.RLock()
sourceDataNodes, found := ecVolume.ShardLocations[shardId]
ecVolume.ShardLocationsLock.RUnlock()
if !found || len(sourceDataNodes) == 0 {
return nil, fmt.Errorf("failed to find ec shard %d.%d", ecVolume.VolumeId, shardId)
}
_, err = s.readOneRemoteEcShardInterval(ctx, sourceDataNodes[0], ecVolume.VolumeId, shardId, data, actualOffset)
if err != nil {
glog.V(1).Infof("failed to read from %s for ec shard %d.%d : %v", sourceDataNode, vid, shardId, err)
glog.V(1).Infof("failed to read from %s for ec shard %d.%d : %v", sourceDataNodes[0], ecVolume.VolumeId, shardId, err)
}
}
return
}
func (s *Store) cachedLookupEcShardLocations(ctx context.Context, vid needle.VolumeId) (shardLocations map[erasure_coding.ShardId]string, err error) {
func (s *Store) cachedLookupEcShardLocations(ctx context.Context, ecVolume *erasure_coding.EcVolume) (err error) {
if ecVolume.ShardLocationsRefreshTime.Add(10 * time.Minute).After(time.Now()) {
// still fresh
return nil
}
ecVolume.ShardLocationsLock.Lock()
defer ecVolume.ShardLocationsLock.Unlock()
err = operation.WithMasterServerClient(s.MasterAddress, s.grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
return nil
})
return
}