convert needle id to ec intervals to read from

This commit is contained in:
Chris Lu 2019-05-27 01:29:46 -07:00
parent a463759edf
commit a4f3d82c57
9 changed files with 215 additions and 105 deletions

View file

@ -66,9 +66,6 @@ func (vs *VolumeServer) doHeartbeat(ctx context.Context, masterNode, masterGrpcA
glog.V(0).Infof("Heartbeat to: %v", masterNode) glog.V(0).Infof("Heartbeat to: %v", masterNode)
vs.currentMaster = masterNode vs.currentMaster = masterNode
vs.store.Client = stream
defer func() { vs.store.Client = nil }()
doneChan := make(chan error, 1) doneChan := make(chan error, 1)
go func() { go func() {

View file

@ -40,7 +40,9 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
} }
glog.V(4).Infoln("volume", volumeId, "reading", n) glog.V(4).Infoln("volume", volumeId, "reading", n)
if !vs.store.HasVolume(volumeId) { hasVolume := vs.store.HasVolume(volumeId)
_, hasEcShard := vs.store.HasEcShard(volumeId)
if !hasVolume && !hasEcShard {
if !vs.ReadRedirect { if !vs.ReadRedirect {
glog.V(2).Infoln("volume is not local:", err, r.URL.Path) glog.V(2).Infoln("volume is not local:", err, r.URL.Path)
w.WriteHeader(http.StatusNotFound) w.WriteHeader(http.StatusNotFound)
@ -65,10 +67,15 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
return return
} }
cookie := n.Cookie cookie := n.Cookie
count, e := vs.store.ReadVolumeNeedle(volumeId, n) var count int
glog.V(4).Infoln("read bytes", count, "error", e) if hasVolume {
if e != nil || count < 0 { count, err = vs.store.ReadVolumeNeedle(volumeId, n)
glog.V(0).Infof("read %s error: %v", r.URL.Path, e) } else if hasEcShard {
count, err = vs.store.ReadEcShardNeedle(volumeId, n)
}
glog.V(4).Infoln("read bytes", count, "error", err)
if err != nil || count < 0 {
glog.V(0).Infof("read %s error: %v", r.URL.Path, err)
w.WriteHeader(http.StatusNotFound) w.WriteHeader(http.StatusNotFound)
return return
} }

View file

@ -16,6 +16,17 @@ var (
re = regexp.MustCompile("\\.ec[0-9][0-9]") re = regexp.MustCompile("\\.ec[0-9][0-9]")
) )
func (l *DiskLocation) HasEcShard(vid needle.VolumeId) (erasure_coding.EcVolumeShards, bool) {
l.ecShardsLock.RLock()
defer l.ecShardsLock.RUnlock()
ecShards, ok := l.ecShards[vid]
if ok {
return ecShards, true
}
return nil, false
}
func (l *DiskLocation) FindEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (*erasure_coding.EcVolumeShard, bool) { func (l *DiskLocation) FindEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (*erasure_coding.EcVolumeShard, bool) {
l.ecShardsLock.RLock() l.ecShardsLock.RLock()
defer l.ecShardsLock.RUnlock() defer l.ecShardsLock.RUnlock()

View file

@ -1,22 +1,25 @@
package erasure_coding package erasure_coding
type Interval struct { type Interval struct {
blockIndex int BlockIndex int
innerBlockOffset int64 InnerBlockOffset int64
size uint32 Size uint32
isLargeBlock bool IsLargeBlock bool
LargeBlockRowsCount int
} }
func locateData(largeBlockLength, smallBlockLength int64, datSize int64, offset int64, size uint32) (intervals []Interval) { func LocateData(largeBlockLength, smallBlockLength int64, datSize int64, offset int64, size uint32) (intervals []Interval) {
blockIndex, isLargeBlock, innerBlockOffset := locateOffset(largeBlockLength, smallBlockLength, datSize, offset) blockIndex, isLargeBlock, innerBlockOffset := locateOffset(largeBlockLength, smallBlockLength, datSize, offset)
nLargeBlockRows := int(datSize / (largeBlockLength * DataShardsCount)) // adding DataShardsCount*smallBlockLength to ensure we can derive the number of large block size from a shard size
nLargeBlockRows := int((datSize + DataShardsCount*smallBlockLength) / (largeBlockLength * DataShardsCount))
for size > 0 { for size > 0 {
interval := Interval{ interval := Interval{
blockIndex: blockIndex, BlockIndex: blockIndex,
innerBlockOffset: innerBlockOffset, InnerBlockOffset: innerBlockOffset,
isLargeBlock: isLargeBlock, IsLargeBlock: isLargeBlock,
LargeBlockRowsCount: nLargeBlockRows,
} }
blockRemaining := largeBlockLength - innerBlockOffset blockRemaining := largeBlockLength - innerBlockOffset
@ -25,14 +28,14 @@ func locateData(largeBlockLength, smallBlockLength int64, datSize int64, offset
} }
if int64(size) <= blockRemaining { if int64(size) <= blockRemaining {
interval.size = size interval.Size = size
intervals = append(intervals, interval) intervals = append(intervals, interval)
return return
} }
interval.size = uint32(blockRemaining) interval.Size = uint32(blockRemaining)
intervals = append(intervals, interval) intervals = append(intervals, interval)
size -= interval.size size -= interval.Size
blockIndex += 1 blockIndex += 1
if isLargeBlock && blockIndex == nLargeBlockRows*DataShardsCount { if isLargeBlock && blockIndex == nLargeBlockRows*DataShardsCount {
isLargeBlock = false isLargeBlock = false

View file

@ -0,0 +1,107 @@
package erasure_coding
import (
"fmt"
"os"
"path"
"strconv"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
type ShardId uint8
type EcVolumeShard struct {
VolumeId needle.VolumeId
ShardId ShardId
Collection string
dir string
ecdFile *os.File
ecdFileSize int64
ecxFile *os.File
ecxFileSize int64
}
func NewEcVolumeShard(dirname string, collection string, id needle.VolumeId, shardId ShardId) (v *EcVolumeShard, e error) {
v = &EcVolumeShard{dir: dirname, Collection: collection, VolumeId: id, ShardId: shardId}
baseFileName := v.FileName()
// open ecx file
if v.ecxFile, e = os.OpenFile(baseFileName+".ecx", os.O_RDONLY, 0644); e != nil {
return nil, fmt.Errorf("cannot read ec volume index %s.ecx: %v", baseFileName, e)
}
ecxFi, statErr := v.ecxFile.Stat()
if statErr != nil {
return nil, fmt.Errorf("can not stat ec volume index %s.ecx: %v", baseFileName, statErr)
}
v.ecxFileSize = ecxFi.Size()
// open ecd file
if v.ecdFile, e = os.OpenFile(baseFileName+ToExt(int(shardId)), os.O_RDONLY, 0644); e != nil {
return nil, fmt.Errorf("cannot read ec volume shard %s.%s: %v", baseFileName, ToExt(int(shardId)), e)
}
ecdFi, statErr := v.ecdFile.Stat()
if statErr != nil {
return nil, fmt.Errorf("can not stat ec volume shard %s.%s: %v", baseFileName, ToExt(int(shardId)), statErr)
}
v.ecdFileSize = ecdFi.Size()
return
}
func (shard *EcVolumeShard) String() string {
return fmt.Sprintf("ec shard %v:%v, dir:%s, Collection:%s", shard.VolumeId, shard.ShardId, shard.dir, shard.Collection)
}
func (shard *EcVolumeShard) FileName() (fileName string) {
return EcShardFileName(shard.Collection, shard.dir, int(shard.VolumeId))
}
func EcShardFileName(collection string, dir string, id int) (fileName string) {
idString := strconv.Itoa(id)
if collection == "" {
fileName = path.Join(dir, idString)
} else {
fileName = path.Join(dir, collection+"_"+idString)
}
return
}
func (shard *EcVolumeShard) Close() {
if shard.ecdFile != nil {
_ = shard.ecdFile.Close()
shard.ecdFile = nil
}
if shard.ecxFile != nil {
_ = shard.ecxFile.Close()
shard.ecxFile = nil
}
}
func (shard *EcVolumeShard) findNeedleFromEcx(needleId types.NeedleId) (offset types.Offset, size uint32, err error) {
var key types.NeedleId
buf := make([]byte, types.NeedleMapEntrySize)
l, h := int64(0), shard.ecxFileSize/types.NeedleMapEntrySize
for l < h {
m := (l + h) / 2
if _, err := shard.ecxFile.ReadAt(buf, m*types.NeedleMapEntrySize); err != nil {
return types.Offset{}, 0, err
}
key, offset, size = idx.IdxFileEntry(buf)
if key == needleId {
return
}
if key < needleId {
l = m + 1
} else {
h = m
}
}
err = fmt.Errorf("needle id %d not found", needleId)
return
}

View file

@ -103,7 +103,7 @@ func readDatFile(datFile *os.File, offset types.Offset, size uint32) ([]byte, er
func readEcFile(datSize int64, ecFiles []*os.File, offset types.Offset, size uint32) (data []byte, err error) { func readEcFile(datSize int64, ecFiles []*os.File, offset types.Offset, size uint32) (data []byte, err error) {
intervals := locateData(largeBlockSize, smallBlockSize, datSize, offset.ToAcutalOffset(), size) intervals := LocateData(largeBlockSize, smallBlockSize, datSize, offset.ToAcutalOffset(), size)
nLargeBlockRows := int(datSize / (largeBlockSize * DataShardsCount)) nLargeBlockRows := int(datSize / (largeBlockSize * DataShardsCount))
@ -123,20 +123,20 @@ func readEcFile(datSize int64, ecFiles []*os.File, offset types.Offset, size uin
} }
func readOneInterval(interval Interval, ecFiles []*os.File, nLargeBlockRows int) (data []byte, err error) { func readOneInterval(interval Interval, ecFiles []*os.File, nLargeBlockRows int) (data []byte, err error) {
ecFileOffset := interval.innerBlockOffset ecFileOffset := interval.InnerBlockOffset
rowIndex := interval.blockIndex / DataShardsCount rowIndex := interval.BlockIndex / DataShardsCount
if interval.isLargeBlock { if interval.IsLargeBlock {
ecFileOffset += int64(rowIndex) * largeBlockSize ecFileOffset += int64(rowIndex) * largeBlockSize
} else { } else {
ecFileOffset += int64(nLargeBlockRows)*largeBlockSize + int64(rowIndex)*smallBlockSize ecFileOffset += int64(nLargeBlockRows)*largeBlockSize + int64(rowIndex)*smallBlockSize
} }
ecFileIndex := interval.blockIndex % DataShardsCount ecFileIndex := interval.BlockIndex % DataShardsCount
data = make([]byte, interval.size) data = make([]byte, interval.Size)
err = readFromFile(ecFiles[ecFileIndex], data, ecFileOffset) err = readFromFile(ecFiles[ecFileIndex], data, ecFileOffset)
{ // do some ec testing { // do some ec testing
ecData, err := readFromOtherEcFiles(ecFiles, ecFileIndex, ecFileOffset, interval.size) ecData, err := readFromOtherEcFiles(ecFiles, ecFileIndex, ecFileOffset, interval.Size)
if err != nil { if err != nil {
return nil, fmt.Errorf("ec reconstruct error: %v", err) return nil, fmt.Errorf("ec reconstruct error: %v", err)
} }
@ -194,7 +194,7 @@ func removeGeneratedFiles(baseFileName string) {
} }
func TestLocateData(t *testing.T) { func TestLocateData(t *testing.T) {
intervals := locateData(largeBlockSize, smallBlockSize, DataShardsCount*largeBlockSize+1, DataShardsCount*largeBlockSize, 1) intervals := LocateData(largeBlockSize, smallBlockSize, DataShardsCount*largeBlockSize+1, DataShardsCount*largeBlockSize, 1)
if len(intervals) != 1 { if len(intervals) != 1 {
t.Errorf("unexpected interval size %d", len(intervals)) t.Errorf("unexpected interval size %d", len(intervals))
} }
@ -202,13 +202,13 @@ func TestLocateData(t *testing.T) {
t.Errorf("unexpected interval %+v", intervals[0]) t.Errorf("unexpected interval %+v", intervals[0])
} }
intervals = locateData(largeBlockSize, smallBlockSize, DataShardsCount*largeBlockSize+1, DataShardsCount*largeBlockSize/2+100, DataShardsCount*largeBlockSize+1-DataShardsCount*largeBlockSize/2-100) intervals = LocateData(largeBlockSize, smallBlockSize, DataShardsCount*largeBlockSize+1, DataShardsCount*largeBlockSize/2+100, DataShardsCount*largeBlockSize+1-DataShardsCount*largeBlockSize/2-100)
fmt.Printf("%+v\n", intervals) fmt.Printf("%+v\n", intervals)
} }
func (this Interval) sameAs(that Interval) bool { func (this Interval) sameAs(that Interval) bool {
return this.isLargeBlock == that.isLargeBlock && return this.IsLargeBlock == that.IsLargeBlock &&
this.innerBlockOffset == that.innerBlockOffset && this.InnerBlockOffset == that.InnerBlockOffset &&
this.blockIndex == that.blockIndex && this.BlockIndex == that.BlockIndex &&
this.size == that.size this.Size == that.Size
} }

View file

@ -1,44 +1,15 @@
package erasure_coding package erasure_coding
import ( import (
"fmt"
"math" "math"
"os"
"path"
"sort" "sort"
"strconv"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb" "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle" "github.com/chrislusf/seaweedfs/weed/storage/needle"
) )
type ShardId uint8
type EcVolumeShard struct {
VolumeId needle.VolumeId
ShardId ShardId
Collection string
dir string
ecdFile *os.File
ecxFile *os.File
}
type EcVolumeShards []*EcVolumeShard type EcVolumeShards []*EcVolumeShard
func NewEcVolumeShard(dirname string, collection string, id needle.VolumeId, shardId ShardId) (v *EcVolumeShard, e error) {
v = &EcVolumeShard{dir: dirname, Collection: collection, VolumeId: id, ShardId: shardId}
baseFileName := v.FileName()
if v.ecxFile, e = os.OpenFile(baseFileName+".ecx", os.O_RDONLY, 0644); e != nil {
return nil, fmt.Errorf("cannot read ec volume index %s.ecx: %v", baseFileName, e)
}
if v.ecdFile, e = os.OpenFile(baseFileName+ToExt(int(shardId)), os.O_RDONLY, 0644); e != nil {
return nil, fmt.Errorf("cannot read ec volume shard %s.%s: %v", baseFileName, ToExt(int(shardId)), e)
}
return
}
func (shards *EcVolumeShards) AddEcVolumeShard(ecVolumeShard *EcVolumeShard) bool { func (shards *EcVolumeShards) AddEcVolumeShard(ecVolumeShard *EcVolumeShard) bool {
for _, s := range *shards { for _, s := range *shards {
if s.ShardId == ecVolumeShard.ShardId { if s.ShardId == ecVolumeShard.ShardId {
@ -68,6 +39,15 @@ func (shards *EcVolumeShards) DeleteEcVolumeShard(ecVolumeShard *EcVolumeShard)
return true return true
} }
func (shards *EcVolumeShards) FindEcVolumeShard(shardId ShardId) (ecVolumeShard *EcVolumeShard, found bool) {
for _, s := range *shards {
if s.ShardId == shardId {
return s, true
}
}
return nil, false
}
func (shards *EcVolumeShards) Close() { func (shards *EcVolumeShards) Close() {
for _, s := range *shards { for _, s := range *shards {
s.Close() s.Close()
@ -91,31 +71,19 @@ func (shards *EcVolumeShards) ToVolumeEcShardInformationMessage() (messages []*m
return return
} }
func (v *EcVolumeShard) String() string { func (shards *EcVolumeShards) ReadEcShardNeedle(n *needle.Needle) (int, error) {
return fmt.Sprintf("ec shard %v:%v, dir:%s, Collection:%s", v.VolumeId, v.ShardId, v.dir, v.Collection)
}
func (v *EcVolumeShard) FileName() (fileName string) { shard := (*shards)[0]
return EcShardFileName(v.Collection, v.dir, int(v.VolumeId)) // find the needle from ecx file
} offset, size, err := shard.findNeedleFromEcx(n.Id)
if err != nil {
return 0, err
}
func EcShardFileName(collection string, dir string, id int) (fileName string) { // calculate the locations in the ec shards
idString := strconv.Itoa(id) intervals := LocateData(ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize, shard.ecxFileSize, offset.ToAcutalOffset(), size)
if collection == "" {
fileName = path.Join(dir, idString)
} else {
fileName = path.Join(dir, collection+"_"+idString)
}
return
}
func (v *EcVolumeShard) Close() { // TODO read the intervals
if v.ecdFile != nil {
_ = v.ecdFile.Close() return len(intervals), nil
v.ecdFile = nil
}
if v.ecxFile != nil {
_ = v.ecxFile.Close()
v.ecxFile = nil
}
} }

View file

@ -18,18 +18,17 @@ const (
* A VolumeServer contains one Store * A VolumeServer contains one Store
*/ */
type Store struct { type Store struct {
volumeSizeLimit uint64 //read from the master volumeSizeLimit uint64 //read from the master
Ip string Ip string
Port int Port int
PublicUrl string PublicUrl string
Locations []*DiskLocation Locations []*DiskLocation
dataCenter string //optional informaton, overwriting master setting if exists dataCenter string //optional informaton, overwriting master setting if exists
rack string //optional information, overwriting master setting if exists rack string //optional information, overwriting master setting if exists
connected bool connected bool
Client master_pb.Seaweed_SendHeartbeatClient NeedleMapType NeedleMapType
NeedleMapType NeedleMapType NewVolumesChan chan master_pb.VolumeShortInformationMessage
NewVolumesChan chan master_pb.VolumeShortInformationMessage DeletedVolumesChan chan master_pb.VolumeShortInformationMessage
DeletedVolumesChan chan master_pb.VolumeShortInformationMessage
NewEcShardsChan chan master_pb.VolumeEcShardInformationMessage NewEcShardsChan chan master_pb.VolumeEcShardInformationMessage
DeletedEcShardsChan chan master_pb.VolumeEcShardInformationMessage DeletedEcShardsChan chan master_pb.VolumeEcShardInformationMessage
} }

View file

@ -33,9 +33,9 @@ func (s *Store) MountEcShards(collection string, vid needle.VolumeId, shardId er
var shardBits erasure_coding.ShardBits var shardBits erasure_coding.ShardBits
s.NewEcShardsChan <- master_pb.VolumeEcShardInformationMessage{ s.NewEcShardsChan <- master_pb.VolumeEcShardInformationMessage{
Id: uint32(vid), Id: uint32(vid),
Collection: collection, Collection: collection,
EcIndexBits: uint32(shardBits.AddShardId(shardId)), EcIndexBits: uint32(shardBits.AddShardId(shardId)),
} }
return nil return nil
} }
@ -53,9 +53,9 @@ func (s *Store) UnmountEcShards(vid needle.VolumeId, shardId erasure_coding.Shar
var shardBits erasure_coding.ShardBits var shardBits erasure_coding.ShardBits
message := master_pb.VolumeEcShardInformationMessage{ message := master_pb.VolumeEcShardInformationMessage{
Id: uint32(vid), Id: uint32(vid),
Collection: ecShard.Collection, Collection: ecShard.Collection,
EcIndexBits: uint32(shardBits.AddShardId(shardId)), EcIndexBits: uint32(shardBits.AddShardId(shardId)),
} }
for _, location := range s.Locations { for _, location := range s.Locations {
@ -69,7 +69,7 @@ func (s *Store) UnmountEcShards(vid needle.VolumeId, shardId erasure_coding.Shar
return fmt.Errorf("UnmountEcShards %d.%d not found on disk", vid, shardId) return fmt.Errorf("UnmountEcShards %d.%d not found on disk", vid, shardId)
} }
func (s *Store) findEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (*erasure_coding.EcVolumeShard, bool) { func (s *Store) findEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (*erasure_coding.EcVolumeShard, bool) {
for _, location := range s.Locations { for _, location := range s.Locations {
if v, found := location.FindEcShard(vid, shardId); found { if v, found := location.FindEcShard(vid, shardId); found {
return v, found return v, found
@ -77,3 +77,21 @@ func (s *Store) findEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId)
} }
return nil, false return nil, false
} }
func (s *Store) HasEcShard(vid needle.VolumeId) (erasure_coding.EcVolumeShards, bool) {
for _, location := range s.Locations {
if s, found := location.HasEcShard(vid); found {
return s, true
}
}
return nil, false
}
func (s *Store) ReadEcShardNeedle(vid needle.VolumeId, n *needle.Needle) (int, error) {
for _, location := range s.Locations {
if ecShards, found := location.HasEcShard(vid); found {
return ecShards.ReadEcShardNeedle(n)
}
}
return 0, fmt.Errorf("ec shard %d not found", vid)
}