mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
caching ec shard locations
This commit is contained in:
parent
3a8c1055a2
commit
713596e781
|
@ -43,7 +43,7 @@ func Assign(server string, grpcDialOption grpc.DialOption, primaryRequest *Volum
|
||||||
continue
|
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{
|
req := &master_pb.AssignRequest{
|
||||||
Count: primaryRequest.Count,
|
Count: primaryRequest.Count,
|
||||||
|
|
|
@ -38,7 +38,7 @@ func toVolumeServerGrpcAddress(volumeServer string) (grpcAddress string, err err
|
||||||
return fmt.Sprintf("%s:%d", volumeServer[0:sepIndex], port+10000), nil
|
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()
|
ctx := context.Background()
|
||||||
|
|
||||||
|
|
|
@ -99,7 +99,7 @@ func LookupVolumeIds(server string, grpcDialOption grpc.DialOption, vids []strin
|
||||||
|
|
||||||
//only query unknown_vids
|
//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{
|
req := &master_pb.LookupVolumeRequest{
|
||||||
VolumeIds: unknown_vids,
|
VolumeIds: unknown_vids,
|
||||||
|
|
|
@ -9,7 +9,7 @@ import (
|
||||||
|
|
||||||
func Statistics(server string, grpcDialOption grpc.DialOption, req *master_pb.StatisticsRequest) (resp *master_pb.StatisticsResponse, err error) {
|
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)
|
grpcResponse, grpcErr := masterClient.Statistics(context.Background(), req)
|
||||||
if grpcErr != nil {
|
if grpcErr != nil {
|
||||||
|
|
|
@ -39,13 +39,13 @@ func (vs *VolumeServer) heartbeat() {
|
||||||
glog.V(0).Infof("failed to parse master grpc %v: %v", masterGrpcAddress, parseErr)
|
glog.V(0).Infof("failed to parse master grpc %v: %v", masterGrpcAddress, parseErr)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
vs.store.MasterGrpcAddress = masterGrpcAddress
|
vs.store.MasterAddress = master
|
||||||
newLeader, err = vs.doHeartbeat(context.Background(), master, masterGrpcAddress, grpcDialOption, time.Duration(vs.pulseSeconds)*time.Second)
|
newLeader, err = vs.doHeartbeat(context.Background(), master, masterGrpcAddress, grpcDialOption, time.Duration(vs.pulseSeconds)*time.Second)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
glog.V(0).Infof("heartbeat error: %v", err)
|
glog.V(0).Infof("heartbeat error: %v", err)
|
||||||
time.Sleep(time.Duration(vs.pulseSeconds) * time.Second)
|
time.Sleep(time.Duration(vs.pulseSeconds) * time.Second)
|
||||||
newLeader = ""
|
newLeader = ""
|
||||||
vs.store.MasterGrpcAddress = ""
|
vs.store.MasterAddress = ""
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -5,6 +5,8 @@ import (
|
||||||
"math"
|
"math"
|
||||||
"os"
|
"os"
|
||||||
"sort"
|
"sort"
|
||||||
|
"sync"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
|
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
|
||||||
"github.com/chrislusf/seaweedfs/weed/storage/idx"
|
"github.com/chrislusf/seaweedfs/weed/storage/idx"
|
||||||
|
@ -13,12 +15,15 @@ import (
|
||||||
)
|
)
|
||||||
|
|
||||||
type EcVolume struct {
|
type EcVolume struct {
|
||||||
Shards []*EcVolumeShard
|
VolumeId needle.VolumeId
|
||||||
VolumeId needle.VolumeId
|
Collection string
|
||||||
Collection string
|
dir string
|
||||||
dir string
|
ecxFile *os.File
|
||||||
ecxFile *os.File
|
ecxFileSize int64
|
||||||
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) {
|
func NewEcVolume(dir string, collection string, vid needle.VolumeId) (ev *EcVolume, err error) {
|
||||||
|
|
|
@ -19,7 +19,7 @@ const (
|
||||||
* A VolumeServer contains one Store
|
* A VolumeServer contains one Store
|
||||||
*/
|
*/
|
||||||
type Store struct {
|
type Store struct {
|
||||||
MasterGrpcAddress string
|
MasterAddress string
|
||||||
grpcDialOption grpc.DialOption
|
grpcDialOption grpc.DialOption
|
||||||
volumeSizeLimit uint64 //read from the master
|
volumeSizeLimit uint64 //read from the master
|
||||||
Ip string
|
Ip string
|
||||||
|
|
|
@ -4,6 +4,7 @@ import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||||
"github.com/chrislusf/seaweedfs/weed/operation"
|
"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) {
|
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 {
|
if err = s.cachedLookupEcShardLocations(ctx, ecVolume); err != nil {
|
||||||
return nil, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterGrpcAddress, err)
|
return nil, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterAddress, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
for i, interval := range intervals {
|
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
|
return nil, e
|
||||||
} else {
|
} else {
|
||||||
if i == 0 {
|
if i == 0 {
|
||||||
|
@ -138,7 +139,7 @@ func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, e
|
||||||
return
|
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)
|
shardId, actualOffset := interval.ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
|
||||||
data = make([]byte, interval.Size)
|
data = make([]byte, interval.Size)
|
||||||
if shard, found := ecVolume.FindEcVolumeShard(shardId); found {
|
if shard, found := ecVolume.FindEcVolumeShard(shardId); found {
|
||||||
|
@ -146,16 +147,33 @@ func (s *Store) readOneEcShardInterval(ctx context.Context, vid needle.VolumeId,
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
sourceDataNode := shardLocations[shardId]
|
ecVolume.ShardLocationsLock.RLock()
|
||||||
_, err = s.readOneRemoteEcShardInterval(ctx, sourceDataNode, vid, shardId, data, actualOffset)
|
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 {
|
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
|
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
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in a new issue