filer: dynamically create bucket under /buckets folder

This commit is contained in:
Chris Lu 2020-02-24 22:28:45 -08:00
parent 5bcb44eda9
commit 6ab7368ef2
21 changed files with 436 additions and 183 deletions

View file

@ -165,6 +165,7 @@ message AssignVolumeRequest {
string replication = 3;
int32 ttl_sec = 4;
string data_center = 5;
string parent_path = 6;
}
message AssignVolumeResponse {
@ -173,6 +174,8 @@ message AssignVolumeResponse {
string public_url = 3;
int32 count = 4;
string auth = 5;
string collection = 6;
string replication = 7;
}
message LookupVolumeRequest {

View file

@ -125,10 +125,6 @@ func runCopy(cmd *Command, args []string) bool {
}
copy.masters = masters
copy.masterClient = wdclient.NewMasterClient(ctx, copy.grpcDialOption, "client", copy.masters)
go copy.masterClient.KeepConnectedToMaster()
copy.masterClient.WaitUntilConnected()
if *cmdCopy.IsDebug {
util.SetupProfiling("filer.copy.cpu.pprof", "filer.copy.mem.pprof")
}
@ -274,23 +270,35 @@ func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopy
mimeType := detectMimeType(f)
var chunks []*filer_pb.FileChunk
var assignResult *filer_pb.AssignVolumeResponse
var assignError error
if task.fileSize > 0 {
// assign a volume
assignResult, err := operation.Assign(worker.options.masterClient.GetMaster(), worker.options.grpcDialOption, &operation.VolumeAssignRequest{
Count: 1,
Replication: *worker.options.replication,
Collection: *worker.options.collection,
Ttl: *worker.options.ttl,
err := withFilerClient(ctx, worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
Replication: *worker.options.replication,
Collection: *worker.options.collection,
TtlSec: int32(util.ParseInt(*worker.options.ttl, 0)),
ParentPath: task.destinationUrlPath,
}
assignResult, assignError = client.AssignVolume(ctx, request)
if assignError != nil {
return fmt.Errorf("assign volume failure %v: %v", request, assignError)
}
return nil
})
if err != nil {
fmt.Printf("Failed to assign from %v: %v\n", worker.options.masters, err)
}
targetUrl := "http://" + assignResult.Url + "/" + assignResult.Fid
targetUrl := "http://" + assignResult.Url + "/" + assignResult.FileId
uploadResult, err := operation.UploadWithLocalCompressionLevel(targetUrl, fileName, f, false, mimeType, nil, assignResult.Auth, *worker.options.compressionLevel)
uploadResult, err := operation.UploadWithLocalCompressionLevel(targetUrl, fileName, f, false, mimeType, nil, security.EncodedJwt(assignResult.Auth), *worker.options.compressionLevel)
if err != nil {
return fmt.Errorf("upload data %v to %s: %v\n", fileName, targetUrl, err)
}
@ -300,7 +308,7 @@ func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopy
fmt.Printf("uploaded %s to %s\n", fileName, targetUrl)
chunks = append(chunks, &filer_pb.FileChunk{
FileId: assignResult.Fid,
FileId: assignResult.FileId,
Offset: 0,
Size: uint64(uploadResult.Size),
Mtime: time.Now().UnixNano(),
@ -352,6 +360,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
concurrentChunks := make(chan struct{}, *worker.options.concurrenctChunks)
var wg sync.WaitGroup
var uploadError error
var collection, replication string
fmt.Printf("uploading %s in %d chunks ...\n", fileName, chunkCount)
for i := int64(0); i < int64(chunkCount) && uploadError == nil; i++ {
@ -363,22 +372,42 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
<-concurrentChunks
}()
// assign a volume
assignResult, err := operation.Assign(worker.options.masterClient.GetMaster(), worker.options.grpcDialOption, &operation.VolumeAssignRequest{
Count: 1,
Replication: *worker.options.replication,
Collection: *worker.options.collection,
Ttl: *worker.options.ttl,
var assignResult *filer_pb.AssignVolumeResponse
var assignError error
err := withFilerClient(ctx, worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
Replication: *worker.options.replication,
Collection: *worker.options.collection,
TtlSec: int32(util.ParseInt(*worker.options.ttl, 0)),
ParentPath: task.destinationUrlPath,
}
assignResult, assignError = client.AssignVolume(ctx, request)
if assignError != nil {
return fmt.Errorf("assign volume failure %v: %v", request, assignError)
}
return nil
})
if err != nil {
fmt.Printf("Failed to assign from %v: %v\n", worker.options.masters, err)
}
if err != nil {
fmt.Printf("Failed to assign from %v: %v\n", worker.options.masters, err)
}
targetUrl := "http://" + assignResult.Url + "/" + assignResult.Fid
targetUrl := "http://" + assignResult.Url + "/" + assignResult.FileId
if collection == "" {
collection = assignResult.Collection
}
if replication == "" {
replication = assignResult.Replication
}
uploadResult, err := operation.Upload(targetUrl,
fileName+"-"+strconv.FormatInt(i+1, 10),
io.NewSectionReader(f, i*chunkSize, chunkSize),
false, "", nil, assignResult.Auth)
false, "", nil, security.EncodedJwt(assignResult.Auth))
if err != nil {
uploadError = fmt.Errorf("upload data %v to %s: %v\n", fileName, targetUrl, err)
return
@ -388,7 +417,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
return
}
chunksChan <- &filer_pb.FileChunk{
FileId: assignResult.Fid,
FileId: assignResult.FileId,
Offset: i * chunkSize,
Size: uint64(uploadResult.Size),
Mtime: time.Now().UnixNano(),
@ -410,7 +439,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
for _, chunk := range chunks {
fileIds = append(fileIds, chunk.FileId)
}
operation.DeleteFiles(worker.options.masterClient.GetMaster(), worker.options.grpcDialOption, fileIds)
operation.DeleteFiles(copy.masters[0], worker.options.grpcDialOption, fileIds)
return uploadError
}
@ -427,8 +456,8 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
FileSize: uint64(task.fileSize),
FileMode: uint32(task.fileMode),
Mime: mimeType,
Replication: *worker.options.replication,
Collection: *worker.options.collection,
Replication: replication,
Collection: collection,
TtlSec: int32(util.ParseInt(*worker.options.ttl, 0)),
},
Chunks: chunks,

View file

@ -99,7 +99,7 @@ func RunMount(filer, filerMountRootPath, dir, collection, replication, dataCente
fuse.VolumeName(mountName),
fuse.FSName(filer + ":" + filerMountRootPath),
fuse.Subtype("seaweedfs"),
fuse.NoAppleDouble(),
// fuse.NoAppleDouble(), // include .DS_Store, otherwise can not delete non-empty folders
fuse.NoAppleXattr(),
fuse.NoBrowse(),
fuse.AutoXattr(),

View file

@ -134,6 +134,7 @@ func (s3opt *S3Options) startS3Server() bool {
return fmt.Errorf("get filer %s configuration: %v", filerGrpcAddress, err)
}
filerBucketsPath = resp.DirBuckets
glog.V(0).Infof("S3 read filer buckets dir: %s", filerBucketsPath)
return nil
})
if err != nil {

View file

@ -30,9 +30,11 @@ type Filer struct {
MasterClient *wdclient.MasterClient
fileIdDeletionQueue *util.UnboundedQueue
GrpcDialOption grpc.DialOption
DirBucketsPath string
buckets *FilerBuckets
}
func NewFiler(masters []string, grpcDialOption grpc.DialOption) *Filer {
func NewFiler(masters []string, grpcDialOption grpc.DialOption, bucketFolder string) *Filer {
f := &Filer{
directoryCache: ccache.New(ccache.Configure().MaxSize(1000).ItemsToPrune(100)),
MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "filer", masters),
@ -109,11 +111,13 @@ func (f *Filer) CreateEntry(ctx context.Context, entry *Entry, o_excl bool) erro
dirEntry = &Entry{
FullPath: FullPath(dirPath),
Attr: Attr{
Mtime: now,
Crtime: now,
Mode: os.ModeDir | 0770,
Uid: entry.Uid,
Gid: entry.Gid,
Mtime: now,
Crtime: now,
Mode: os.ModeDir | 0770,
Uid: entry.Uid,
Gid: entry.Gid,
Collection: entry.Collection,
Replication: entry.Replication,
},
}
@ -125,6 +129,7 @@ func (f *Filer) CreateEntry(ctx context.Context, entry *Entry, o_excl bool) erro
return fmt.Errorf("mkdir %s: %v", dirPath, mkdirErr)
}
} else {
f.maybeAddBucket(dirEntry)
f.NotifyUpdateEvent(nil, dirEntry, false)
}
@ -175,6 +180,7 @@ func (f *Filer) CreateEntry(ctx context.Context, entry *Entry, o_excl bool) erro
}
}
f.maybeAddBucket(entry)
f.NotifyUpdateEvent(oldEntry, entry, true)
f.deleteChunksIfNotNew(oldEntry, entry)

View file

@ -0,0 +1,113 @@
package filer2
import (
"context"
"math"
"sync"
"github.com/chrislusf/seaweedfs/weed/glog"
)
type BucketName string
type BucketOption struct {
Name BucketName
Replication string
}
type FilerBuckets struct {
dirBucketsPath string
buckets map[BucketName]*BucketOption
sync.RWMutex
}
func (f *Filer) LoadBuckets(dirBucketsPath string) {
f.buckets = &FilerBuckets{
buckets: make(map[BucketName]*BucketOption),
}
f.DirBucketsPath = dirBucketsPath
limit := math.MaxInt32
entries, err := f.ListDirectoryEntries(context.Background(), FullPath(dirBucketsPath), "", false, limit)
if err != nil {
glog.V(1).Infof("no buckets found: %v", err)
return
}
glog.V(1).Infof("buckets found: %d", len(entries))
f.buckets.Lock()
for _, entry := range entries {
f.buckets.buckets[BucketName(entry.Name())] = &BucketOption{
Name: BucketName(entry.Name()),
Replication: entry.Replication,
}
}
f.buckets.Unlock()
}
func (f *Filer) ReadBucketOption(buketName string) (replication string) {
f.buckets.RLock()
defer f.buckets.RUnlock()
option, found := f.buckets.buckets[BucketName(buketName)]
if !found {
return ""
}
return option.Replication
}
func (f *Filer) isBucket(entry *Entry) bool {
if !entry.IsDirectory() {
return false
}
parent, dirName := entry.FullPath.DirAndName()
if parent != f.DirBucketsPath {
return false
}
f.buckets.RLock()
defer f.buckets.RUnlock()
_, found := f.buckets.buckets[BucketName(dirName)]
return found
}
func (f *Filer) maybeAddBucket(entry *Entry) {
if !entry.IsDirectory() {
return
}
parent, dirName := entry.FullPath.DirAndName()
if parent != f.DirBucketsPath {
return
}
f.addBucket(dirName, &BucketOption{
Name: BucketName(dirName),
Replication: entry.Replication,
})
}
func (f *Filer) addBucket(buketName string, bucketOption *BucketOption) {
f.buckets.Lock()
defer f.buckets.Unlock()
f.buckets.buckets[BucketName(buketName)] = bucketOption
}
func (f *Filer) deleteBucket(buketName string) {
f.buckets.Lock()
defer f.buckets.Unlock()
delete(f.buckets.buckets, BucketName(buketName))
}

View file

@ -6,6 +6,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
)
func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p FullPath, isRecursive bool, ignoreRecursiveError, shouldDeleteChunks bool) (err error) {
@ -18,27 +19,35 @@ func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p FullPath, isRecurs
return findErr
}
isCollection := f.isBucket(entry)
var chunks []*filer_pb.FileChunk
chunks = append(chunks, entry.Chunks...)
if entry.IsDirectory() {
// delete the folder children, not including the folder itself
var dirChunks []*filer_pb.FileChunk
dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, entry, isRecursive, ignoreRecursiveError, shouldDeleteChunks)
dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, entry, isRecursive, ignoreRecursiveError, shouldDeleteChunks && !isCollection)
if err != nil {
glog.V(0).Infof("delete directory %s: %v", p, err)
return fmt.Errorf("delete directory %s: %v", p, err)
}
chunks = append(chunks, dirChunks...)
f.cacheDelDirectory(string(p))
}
// delete the file or folder
err = f.doDeleteEntryMetaAndData(ctx, entry, shouldDeleteChunks)
if err != nil {
return fmt.Errorf("delete file %s: %v", p, err)
}
if shouldDeleteChunks {
if shouldDeleteChunks && !isCollection {
go f.DeleteChunks(chunks)
}
if isCollection {
collectionName := entry.Name()
f.doDeleteCollection(ctx, collectionName)
f.deleteBucket(collectionName)
}
return nil
}
@ -55,6 +64,9 @@ func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry
}
if lastFileName == "" && !isRecursive && len(entries) > 0 {
// only for first iteration in the loop
for _, child := range entries {
println("existing children", child.Name())
}
return nil, fmt.Errorf("fail to delete non-empty folder: %s", entry.FullPath)
}
@ -100,3 +112,17 @@ func (f *Filer) doDeleteEntryMetaAndData(ctx context.Context, entry *Entry, shou
return nil
}
func (f *Filer) doDeleteCollection(ctx context.Context, collectionName string) (err error) {
return f.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
_, err := client.CollectionDelete(ctx, &master_pb.CollectionDeleteRequest{
Name: collectionName,
})
if err != nil {
glog.Infof("delete collection %s: %v", collectionName, err)
}
return err
})
}

View file

@ -9,7 +9,7 @@ import (
)
func TestCreateAndFind(t *testing.T) {
filer := filer2.NewFiler(nil, nil)
filer := filer2.NewFiler(nil, nil, "")
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test")
defer os.RemoveAll(dir)
store := &LevelDBStore{}
@ -64,7 +64,7 @@ func TestCreateAndFind(t *testing.T) {
}
func TestEmptyRoot(t *testing.T) {
filer := filer2.NewFiler(nil, nil)
filer := filer2.NewFiler(nil, nil, "")
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test2")
defer os.RemoveAll(dir)
store := &LevelDBStore{}

View file

@ -9,7 +9,7 @@ import (
)
func TestCreateAndFind(t *testing.T) {
filer := filer2.NewFiler(nil, nil)
filer := filer2.NewFiler(nil, nil, "")
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test")
defer os.RemoveAll(dir)
store := &LevelDB2Store{}
@ -64,7 +64,7 @@ func TestCreateAndFind(t *testing.T) {
}
func TestEmptyRoot(t *testing.T) {
filer := filer2.NewFiler(nil, nil)
filer := filer2.NewFiler(nil, nil, "")
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test2")
defer os.RemoveAll(dir)
store := &LevelDB2Store{}

View file

@ -15,9 +15,11 @@ import (
)
type ContinuousDirtyPages struct {
intervals *ContinuousIntervals
f *File
lock sync.Mutex
intervals *ContinuousIntervals
f *File
lock sync.Mutex
collection string
replication string
}
func newDirtyPages(file *File) *ContinuousDirtyPages {
@ -140,6 +142,8 @@ func (pages *ContinuousDirtyPages) saveToStorage(ctx context.Context, reader io.
var fileId, host string
var auth security.EncodedJwt
dir, _ := pages.f.fullpath().DirAndName()
if err := pages.f.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
@ -148,6 +152,7 @@ func (pages *ContinuousDirtyPages) saveToStorage(ctx context.Context, reader io.
Collection: pages.f.wfs.option.Collection,
TtlSec: pages.f.wfs.option.TtlSec,
DataCenter: pages.f.wfs.option.DataCenter,
ParentPath: dir,
}
resp, err := client.AssignVolume(ctx, request)
@ -157,6 +162,7 @@ func (pages *ContinuousDirtyPages) saveToStorage(ctx context.Context, reader io.
}
fileId, host, auth = resp.FileId, resp.Url, security.EncodedJwt(resp.Auth)
pages.collection, pages.replication = resp.Collection, resp.Replication
return nil
}); err != nil {

View file

@ -178,6 +178,8 @@ func (fh *FileHandle) Flush(ctx context.Context, req *fuse.FlushRequest) error {
fh.f.entry.Attributes.Mtime = time.Now().Unix()
fh.f.entry.Attributes.Crtime = time.Now().Unix()
fh.f.entry.Attributes.FileMode = uint32(0777 &^ fh.f.wfs.option.Umask)
fh.f.entry.Attributes.Collection = fh.dirtyPages.collection
fh.f.entry.Attributes.Replication = fh.dirtyPages.replication
}
request := &filer_pb.CreateEntryRequest{

View file

@ -3,11 +3,13 @@ package operation
import (
"context"
"fmt"
"strings"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/util"
"google.golang.org/grpc"
"strings"
)
type VolumeAssignRequest struct {

View file

@ -165,6 +165,7 @@ message AssignVolumeRequest {
string replication = 3;
int32 ttl_sec = 4;
string data_center = 5;
string parent_path = 6;
}
message AssignVolumeResponse {
@ -173,6 +174,8 @@ message AssignVolumeResponse {
string public_url = 3;
int32 count = 4;
string auth = 5;
string collection = 6;
string replication = 7;
}
message LookupVolumeRequest {

View file

@ -685,6 +685,7 @@ type AssignVolumeRequest struct {
Replication string `protobuf:"bytes,3,opt,name=replication" json:"replication,omitempty"`
TtlSec int32 `protobuf:"varint,4,opt,name=ttl_sec,json=ttlSec" json:"ttl_sec,omitempty"`
DataCenter string `protobuf:"bytes,5,opt,name=data_center,json=dataCenter" json:"data_center,omitempty"`
ParentPath string `protobuf:"bytes,6,opt,name=parent_path,json=parentPath" json:"parent_path,omitempty"`
}
func (m *AssignVolumeRequest) Reset() { *m = AssignVolumeRequest{} }
@ -727,12 +728,21 @@ func (m *AssignVolumeRequest) GetDataCenter() string {
return ""
}
func (m *AssignVolumeRequest) GetParentPath() string {
if m != nil {
return m.ParentPath
}
return ""
}
type AssignVolumeResponse struct {
FileId string `protobuf:"bytes,1,opt,name=file_id,json=fileId" json:"file_id,omitempty"`
Url string `protobuf:"bytes,2,opt,name=url" json:"url,omitempty"`
PublicUrl string `protobuf:"bytes,3,opt,name=public_url,json=publicUrl" json:"public_url,omitempty"`
Count int32 `protobuf:"varint,4,opt,name=count" json:"count,omitempty"`
Auth string `protobuf:"bytes,5,opt,name=auth" json:"auth,omitempty"`
FileId string `protobuf:"bytes,1,opt,name=file_id,json=fileId" json:"file_id,omitempty"`
Url string `protobuf:"bytes,2,opt,name=url" json:"url,omitempty"`
PublicUrl string `protobuf:"bytes,3,opt,name=public_url,json=publicUrl" json:"public_url,omitempty"`
Count int32 `protobuf:"varint,4,opt,name=count" json:"count,omitempty"`
Auth string `protobuf:"bytes,5,opt,name=auth" json:"auth,omitempty"`
Collection string `protobuf:"bytes,6,opt,name=collection" json:"collection,omitempty"`
Replication string `protobuf:"bytes,7,opt,name=replication" json:"replication,omitempty"`
}
func (m *AssignVolumeResponse) Reset() { *m = AssignVolumeResponse{} }
@ -775,6 +785,20 @@ func (m *AssignVolumeResponse) GetAuth() string {
return ""
}
func (m *AssignVolumeResponse) GetCollection() string {
if m != nil {
return m.Collection
}
return ""
}
func (m *AssignVolumeResponse) GetReplication() string {
if m != nil {
return m.Replication
}
return ""
}
type LookupVolumeRequest struct {
VolumeIds []string `protobuf:"bytes,1,rep,name=volume_ids,json=volumeIds" json:"volume_ids,omitempty"`
}
@ -1481,109 +1505,110 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("filer.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{
// 1655 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0x4b, 0x6f, 0x1b, 0x47,
0x12, 0xd6, 0xf0, 0x3d, 0x45, 0xd2, 0x96, 0x9a, 0x92, 0x4d, 0x53, 0x8f, 0x95, 0x47, 0x6b, 0xaf,
0x16, 0x36, 0xb4, 0x86, 0xd6, 0x07, 0x7b, 0xbd, 0x7b, 0xb0, 0xf5, 0x58, 0x08, 0x2b, 0x3f, 0x30,
0xb2, 0x17, 0xbb, 0x08, 0x90, 0xc1, 0x68, 0xa6, 0x49, 0x75, 0x34, 0x9c, 0x66, 0xba, 0x7b, 0x24,
0x39, 0x3f, 0x21, 0x3f, 0x21, 0x40, 0xce, 0xf9, 0x01, 0xb9, 0x06, 0xb9, 0x04, 0x41, 0xfe, 0x4d,
0x8e, 0x39, 0x07, 0xdd, 0x3d, 0x33, 0xec, 0x21, 0x29, 0xc9, 0x46, 0xe0, 0xdb, 0x74, 0xbd, 0xba,
0xaa, 0xba, 0xea, 0xab, 0x22, 0xa1, 0xd9, 0x27, 0x11, 0x66, 0x5b, 0x23, 0x46, 0x05, 0x45, 0x0d,
0x75, 0xf0, 0x46, 0xc7, 0xce, 0x6b, 0x58, 0x3e, 0xa4, 0xf4, 0x34, 0x19, 0xed, 0x12, 0x86, 0x03,
0x41, 0xd9, 0xfb, 0xbd, 0x58, 0xb0, 0xf7, 0x2e, 0xfe, 0x32, 0xc1, 0x5c, 0xa0, 0x15, 0xb0, 0xc3,
0x8c, 0xd1, 0xb5, 0xd6, 0xad, 0x4d, 0xdb, 0x1d, 0x13, 0x10, 0x82, 0x4a, 0xec, 0x0f, 0x71, 0xb7,
0xa4, 0x18, 0xea, 0xdb, 0xd9, 0x83, 0x95, 0xd9, 0x06, 0xf9, 0x88, 0xc6, 0x1c, 0xa3, 0x7b, 0x50,
0xc5, 0x92, 0xa0, 0xac, 0x35, 0xb7, 0x6f, 0x6e, 0x65, 0xae, 0x6c, 0x69, 0x39, 0xcd, 0x75, 0x7e,
0xb4, 0x00, 0x1d, 0x12, 0x2e, 0x24, 0x91, 0x60, 0xfe, 0x61, 0xfe, 0xdc, 0x82, 0xda, 0x88, 0xe1,
0x3e, 0xb9, 0x48, 0x3d, 0x4a, 0x4f, 0xe8, 0x21, 0x2c, 0x70, 0xe1, 0x33, 0xb1, 0xcf, 0xe8, 0x70,
0x9f, 0x44, 0xf8, 0x95, 0x74, 0xba, 0xac, 0x44, 0xa6, 0x19, 0x68, 0x0b, 0x10, 0x89, 0x83, 0x28,
0xe1, 0xe4, 0x0c, 0x1f, 0x65, 0xdc, 0x6e, 0x65, 0xdd, 0xda, 0x6c, 0xb8, 0x33, 0x38, 0x68, 0x11,
0xaa, 0x11, 0x19, 0x12, 0xd1, 0xad, 0xae, 0x5b, 0x9b, 0x6d, 0x57, 0x1f, 0x9c, 0x7f, 0x42, 0xa7,
0xe0, 0xff, 0xc7, 0x85, 0xff, 0x6d, 0x09, 0xaa, 0x8a, 0x90, 0xe7, 0xd8, 0x1a, 0xe7, 0x18, 0xdd,
0x85, 0x16, 0xe1, 0xde, 0x38, 0x11, 0x25, 0xe5, 0x5b, 0x93, 0xf0, 0x3c, 0xe7, 0xe8, 0x01, 0xd4,
0x82, 0x93, 0x24, 0x3e, 0xe5, 0xdd, 0xf2, 0x7a, 0x79, 0xb3, 0xb9, 0xdd, 0x19, 0x5f, 0x24, 0x03,
0xdd, 0x91, 0x3c, 0x37, 0x15, 0x41, 0x4f, 0x00, 0x7c, 0x21, 0x18, 0x39, 0x4e, 0x04, 0xe6, 0x2a,
0xd2, 0xe6, 0x76, 0xd7, 0x50, 0x48, 0x38, 0x7e, 0x9e, 0xf3, 0x5d, 0x43, 0x16, 0x3d, 0x85, 0x06,
0xbe, 0x10, 0x38, 0x0e, 0x71, 0xd8, 0xad, 0xaa, 0x8b, 0x56, 0x27, 0x22, 0xda, 0xda, 0x4b, 0xf9,
0x3a, 0xbe, 0x5c, 0xbc, 0xf7, 0x0c, 0xda, 0x05, 0x16, 0x9a, 0x87, 0xf2, 0x29, 0xce, 0x5e, 0x55,
0x7e, 0xca, 0xcc, 0x9e, 0xf9, 0x51, 0xa2, 0x0b, 0xac, 0xe5, 0xea, 0xc3, 0x3f, 0x4a, 0x4f, 0x2c,
0x67, 0x17, 0xec, 0xfd, 0x24, 0x8a, 0x72, 0xc5, 0x90, 0xb0, 0x4c, 0x31, 0x24, 0x6c, 0x9c, 0xe5,
0xd2, 0x95, 0x59, 0xfe, 0xc1, 0x82, 0x85, 0xbd, 0x33, 0x1c, 0x8b, 0x57, 0x54, 0x90, 0x3e, 0x09,
0x7c, 0x41, 0x68, 0x8c, 0x1e, 0x82, 0x4d, 0xa3, 0xd0, 0xbb, 0xf2, 0x99, 0x1a, 0x34, 0x4a, 0xbd,
0x7e, 0x08, 0x76, 0x8c, 0xcf, 0xbd, 0x2b, 0xaf, 0x6b, 0xc4, 0xf8, 0x5c, 0x4b, 0x6f, 0x40, 0x3b,
0xc4, 0x11, 0x16, 0xd8, 0xcb, 0x5f, 0x47, 0x3e, 0x5d, 0x4b, 0x13, 0x77, 0xf4, 0x73, 0xdc, 0x87,
0x9b, 0xd2, 0xe4, 0xc8, 0x67, 0x38, 0x16, 0xde, 0xc8, 0x17, 0x27, 0xea, 0x4d, 0x6c, 0xb7, 0x1d,
0xe3, 0xf3, 0x37, 0x8a, 0xfa, 0xc6, 0x17, 0x27, 0xce, 0x6f, 0x16, 0xd8, 0xf9, 0x63, 0xa2, 0xdb,
0x50, 0x97, 0xd7, 0x7a, 0x24, 0x4c, 0x33, 0x51, 0x93, 0xc7, 0x83, 0x50, 0x76, 0x05, 0xed, 0xf7,
0x39, 0x16, 0xca, 0xbd, 0xb2, 0x9b, 0x9e, 0x64, 0x65, 0x71, 0xf2, 0x95, 0x6e, 0x84, 0x8a, 0xab,
0xbe, 0x65, 0xc6, 0x87, 0x82, 0x0c, 0xb1, 0xba, 0xb0, 0xec, 0xea, 0x03, 0xea, 0x40, 0x15, 0x7b,
0xc2, 0x1f, 0xa8, 0x0a, 0xb7, 0xdd, 0x0a, 0x7e, 0xeb, 0x0f, 0xd0, 0x9f, 0xe1, 0x06, 0xa7, 0x09,
0x0b, 0xb0, 0x97, 0x5d, 0x5b, 0x53, 0xdc, 0x96, 0xa6, 0xee, 0xeb, 0xcb, 0x1d, 0x28, 0xf7, 0x49,
0xd8, 0xad, 0xab, 0xc4, 0xcc, 0x17, 0x8b, 0xf0, 0x20, 0x74, 0x25, 0x13, 0xfd, 0x0d, 0x20, 0xb7,
0x14, 0x76, 0x1b, 0x97, 0x88, 0xda, 0x99, 0xdd, 0xd0, 0xf9, 0x1f, 0xd4, 0x52, 0xf3, 0xcb, 0x60,
0x9f, 0xd1, 0x28, 0x19, 0xe6, 0x61, 0xb7, 0xdd, 0x86, 0x26, 0x1c, 0x84, 0xe8, 0x0e, 0x28, 0x9c,
0xf3, 0x64, 0x55, 0x95, 0x54, 0x90, 0x2a, 0x43, 0xff, 0xc1, 0x0a, 0x29, 0x02, 0x4a, 0x4f, 0x89,
0x8e, 0xbe, 0xee, 0xa6, 0x27, 0xe7, 0xd7, 0x12, 0xdc, 0x28, 0x96, 0xbb, 0xbc, 0x42, 0x59, 0x51,
0xb9, 0xb2, 0x94, 0x19, 0x65, 0xf6, 0xa8, 0x90, 0xaf, 0x92, 0x99, 0xaf, 0x4c, 0x65, 0x48, 0x43,
0x7d, 0x41, 0x5b, 0xab, 0xbc, 0xa4, 0x21, 0x96, 0xd5, 0x9a, 0x90, 0x50, 0x25, 0xb8, 0xed, 0xca,
0x4f, 0x49, 0x19, 0x90, 0x30, 0x85, 0x0f, 0xf9, 0xa9, 0xdc, 0x63, 0xca, 0x6e, 0x4d, 0x3f, 0x99,
0x3e, 0xc9, 0x27, 0x1b, 0x4a, 0x6a, 0x5d, 0xbf, 0x83, 0xfc, 0x46, 0xeb, 0xd0, 0x64, 0x78, 0x14,
0xa5, 0xd5, 0xab, 0xd2, 0x67, 0xbb, 0x26, 0x09, 0xad, 0x01, 0x04, 0x34, 0x8a, 0x70, 0xa0, 0x04,
0x6c, 0x25, 0x60, 0x50, 0x64, 0xe5, 0x08, 0x11, 0x79, 0x1c, 0x07, 0x5d, 0x58, 0xb7, 0x36, 0xab,
0x6e, 0x4d, 0x88, 0xe8, 0x08, 0x07, 0x32, 0x8e, 0x84, 0x63, 0xe6, 0x29, 0x00, 0x6a, 0x2a, 0xbd,
0x86, 0x24, 0x28, 0x98, 0x5c, 0x05, 0x18, 0x30, 0x9a, 0x8c, 0x34, 0xb7, 0xb5, 0x5e, 0x96, 0x58,
0xac, 0x28, 0x8a, 0x7d, 0x0f, 0x6e, 0xf0, 0xf7, 0xc3, 0x88, 0xc4, 0xa7, 0x9e, 0xf0, 0xd9, 0x00,
0x8b, 0x6e, 0x5b, 0xd7, 0x70, 0x4a, 0x7d, 0xab, 0x88, 0xce, 0x08, 0xd0, 0x0e, 0xc3, 0xbe, 0xc0,
0x1f, 0x31, 0x76, 0x3e, 0xac, 0xbb, 0xd1, 0x12, 0xd4, 0xa8, 0x87, 0x2f, 0x82, 0x28, 0x6d, 0xb2,
0x2a, 0xdd, 0xbb, 0x08, 0x22, 0xe7, 0x01, 0x74, 0x0a, 0x37, 0xa6, 0xc0, 0xbc, 0x08, 0x55, 0xcc,
0x18, 0xcd, 0x60, 0x44, 0x1f, 0x9c, 0xff, 0x03, 0x7a, 0x37, 0x0a, 0x3f, 0x85, 0x7b, 0xce, 0x12,
0x74, 0x0a, 0xa6, 0xb5, 0x1f, 0xce, 0xcf, 0x16, 0xa0, 0x5d, 0x85, 0x06, 0x7f, 0x6c, 0x10, 0xcb,
0xfe, 0x94, 0x43, 0x42, 0xa3, 0x4d, 0xe8, 0x0b, 0x3f, 0x1d, 0x61, 0x2d, 0xc2, 0xb5, 0xfd, 0x5d,
0x5f, 0xf8, 0xe9, 0x28, 0x61, 0x38, 0x48, 0x98, 0x9c, 0x6a, 0xaa, 0x08, 0xd5, 0x28, 0x71, 0x33,
0x12, 0x7a, 0x0c, 0xb7, 0xc8, 0x20, 0xa6, 0x0c, 0x8f, 0xc5, 0x3c, 0x9d, 0xaa, 0x9a, 0x12, 0x5e,
0xd4, 0xdc, 0x5c, 0x61, 0x4f, 0x65, 0x6e, 0x09, 0x3a, 0x85, 0x30, 0xd2, 0xf0, 0xbe, 0xb1, 0xa0,
0xfb, 0x5c, 0xd0, 0x21, 0x09, 0x5c, 0x2c, 0xdd, 0x2c, 0x04, 0xb9, 0x01, 0x6d, 0x89, 0xbc, 0x93,
0x81, 0xb6, 0x68, 0x14, 0x8e, 0x27, 0xdb, 0x1d, 0x90, 0xe0, 0xeb, 0x19, 0xf1, 0xd6, 0x69, 0x14,
0xaa, 0x9a, 0xdb, 0x00, 0x89, 0x90, 0x86, 0xbe, 0x9e, 0xf1, 0xad, 0x18, 0x9f, 0x17, 0xf4, 0xa5,
0x90, 0xd2, 0xd7, 0xb0, 0x5a, 0x8f, 0xf1, 0xb9, 0xd4, 0x77, 0x96, 0xe1, 0xce, 0x0c, 0xdf, 0x52,
0xcf, 0xbf, 0xb3, 0xa0, 0xf3, 0x9c, 0x73, 0x32, 0x88, 0xff, 0xab, 0x00, 0x26, 0x73, 0x7a, 0x11,
0xaa, 0x01, 0x4d, 0x62, 0xa1, 0x9c, 0xad, 0xba, 0xfa, 0x30, 0xd1, 0x73, 0xa5, 0xa9, 0x9e, 0x9b,
0xe8, 0xda, 0xf2, 0x74, 0xd7, 0x1a, 0x5d, 0x59, 0x29, 0x74, 0xe5, 0x9f, 0xa0, 0x29, 0x9f, 0xd3,
0x0b, 0x70, 0x2c, 0x30, 0x4b, 0x31, 0x19, 0x24, 0x69, 0x47, 0x51, 0x9c, 0xaf, 0x2d, 0x58, 0x2c,
0x7a, 0x9a, 0xd6, 0xf8, 0xa5, 0x23, 0x42, 0x62, 0x12, 0x8b, 0x52, 0x37, 0xe5, 0xa7, 0xec, 0xee,
0x51, 0x72, 0x1c, 0x91, 0xc0, 0x93, 0x0c, 0xed, 0x9e, 0xad, 0x29, 0xef, 0x58, 0x34, 0x0e, 0xba,
0x62, 0x06, 0x8d, 0xa0, 0xe2, 0x27, 0xe2, 0x24, 0x1b, 0x13, 0xf2, 0xdb, 0x79, 0x0c, 0x1d, 0xbd,
0x0f, 0x16, 0xb3, 0xb6, 0x0a, 0x90, 0x03, 0x37, 0xef, 0x5a, 0x1a, 0x3d, 0x32, 0xe4, 0xe6, 0xce,
0xbf, 0xc0, 0x3e, 0xa4, 0x3a, 0x11, 0x1c, 0x3d, 0x02, 0x3b, 0xca, 0x0e, 0x4a, 0xb4, 0xb9, 0x8d,
0xc6, 0x4d, 0x95, 0xc9, 0xb9, 0x63, 0x21, 0xe7, 0x19, 0x34, 0x32, 0x72, 0x16, 0x9b, 0x75, 0x59,
0x6c, 0xa5, 0x89, 0xd8, 0x9c, 0x9f, 0x2c, 0x58, 0x2c, 0xba, 0x9c, 0xa6, 0xef, 0x1d, 0xb4, 0xf3,
0x2b, 0xbc, 0xa1, 0x3f, 0x4a, 0x7d, 0x79, 0x64, 0xfa, 0x32, 0xad, 0x96, 0x3b, 0xc8, 0x5f, 0xfa,
0x23, 0x5d, 0x52, 0xad, 0xc8, 0x20, 0xf5, 0xde, 0xc2, 0xc2, 0x94, 0xc8, 0x8c, 0x65, 0xe8, 0xaf,
0xe6, 0x32, 0x54, 0x58, 0xe8, 0x72, 0x6d, 0x73, 0x43, 0x7a, 0x0a, 0xb7, 0x75, 0xff, 0xed, 0xe4,
0x45, 0x97, 0xe5, 0xbe, 0x58, 0x9b, 0xd6, 0x64, 0x6d, 0x3a, 0x3d, 0xe8, 0x4e, 0xab, 0xa6, 0x5d,
0x30, 0x80, 0x85, 0x23, 0xe1, 0x0b, 0xc2, 0x05, 0x09, 0xf2, 0xad, 0x7c, 0xa2, 0x98, 0xad, 0xeb,
0x46, 0xd0, 0x74, 0x3b, 0xcc, 0x43, 0x59, 0x88, 0xac, 0xce, 0xe4, 0xa7, 0x7c, 0x05, 0x64, 0xde,
0x94, 0xbe, 0xc1, 0x27, 0xb8, 0x4a, 0xd6, 0x83, 0xa0, 0xc2, 0x8f, 0xf4, 0x88, 0xaf, 0xa8, 0x11,
0x6f, 0x2b, 0x8a, 0x9a, 0xf1, 0x7a, 0x0a, 0x86, 0x9a, 0x5b, 0xd5, 0x0b, 0x80, 0x24, 0x28, 0xe6,
0x2a, 0x80, 0x6a, 0x29, 0xdd, 0x0d, 0x35, 0xad, 0x2b, 0x29, 0x3b, 0x92, 0xe0, 0xac, 0xc1, 0xca,
0xbf, 0xb1, 0x90, 0xcb, 0x0a, 0xdb, 0xa1, 0x71, 0x9f, 0x0c, 0x12, 0xe6, 0x1b, 0x4f, 0xe1, 0x7c,
0x6f, 0xc1, 0xea, 0x25, 0x02, 0x69, 0xc0, 0x5d, 0xa8, 0x0f, 0x7d, 0x2e, 0x30, 0xcb, 0xba, 0x24,
0x3b, 0x4e, 0xa6, 0xa2, 0x74, 0x5d, 0x2a, 0xca, 0x53, 0xa9, 0x58, 0x82, 0xda, 0xd0, 0xbf, 0xf0,
0x86, 0xc7, 0xe9, 0x36, 0x52, 0x1d, 0xfa, 0x17, 0x2f, 0x8f, 0x15, 0xc0, 0x10, 0xe6, 0x1d, 0x27,
0xc1, 0x29, 0x16, 0x3c, 0x07, 0x18, 0xc2, 0x5e, 0x68, 0xca, 0xf6, 0x2f, 0x75, 0x68, 0x1d, 0x61,
0xff, 0x1c, 0xe3, 0x50, 0x79, 0x8e, 0x06, 0x59, 0xc7, 0x14, 0x7f, 0xf4, 0xa1, 0x7b, 0x93, 0xad,
0x31, 0xf3, 0x57, 0x66, 0xef, 0xfe, 0x75, 0x62, 0x69, 0xf1, 0xcd, 0xa1, 0x57, 0xd0, 0x34, 0x7e,
0x55, 0xa1, 0x15, 0x43, 0x71, 0xea, 0xc7, 0x62, 0x6f, 0xf5, 0x12, 0x6e, 0x66, 0xed, 0x91, 0x85,
0x0e, 0xa1, 0x69, 0x2c, 0x03, 0xa6, 0xbd, 0xe9, 0xad, 0xc4, 0xb4, 0x37, 0x63, 0x83, 0x70, 0xe6,
0xa4, 0x35, 0x63, 0xa4, 0x9b, 0xd6, 0xa6, 0x97, 0x08, 0xd3, 0xda, 0xac, 0x3d, 0x40, 0x59, 0x33,
0x26, 0xa8, 0x69, 0x6d, 0x7a, 0x3f, 0x30, 0xad, 0xcd, 0x1a, 0xbb, 0x73, 0xe8, 0x73, 0x58, 0x98,
0x9a, 0x6d, 0xc8, 0x19, 0x6b, 0x5d, 0x36, 0x94, 0x7b, 0x1b, 0x57, 0xca, 0xe4, 0xf6, 0x5f, 0x43,
0xcb, 0x9c, 0x39, 0xc8, 0x70, 0x68, 0xc6, 0xd4, 0xec, 0xad, 0x5d, 0xc6, 0x36, 0x0d, 0x9a, 0x70,
0x6a, 0x1a, 0x9c, 0x31, 0x50, 0x4c, 0x83, 0xb3, 0x50, 0xd8, 0x99, 0x43, 0x9f, 0xc1, 0xfc, 0x24,
0xac, 0xa1, 0xbb, 0x93, 0x69, 0x9b, 0x42, 0xcb, 0x9e, 0x73, 0x95, 0x48, 0x6e, 0xfc, 0x00, 0x60,
0x8c, 0x56, 0x68, 0x79, 0xac, 0x33, 0x85, 0x96, 0xbd, 0x95, 0xd9, 0xcc, 0xdc, 0xd4, 0x17, 0xb0,
0x34, 0x13, 0x12, 0x90, 0xd1, 0x26, 0x57, 0x81, 0x4a, 0xef, 0x2f, 0xd7, 0xca, 0x65, 0x77, 0xbd,
0x58, 0x83, 0x79, 0xae, 0x1b, 0xb9, 0xcf, 0xb7, 0x82, 0x88, 0xe0, 0x58, 0xbc, 0x00, 0xa5, 0xf1,
0x86, 0x51, 0x41, 0x8f, 0x6b, 0xea, 0xff, 0xa2, 0xbf, 0xff, 0x1e, 0x00, 0x00, 0xff, 0xff, 0x7f,
0xd8, 0x6f, 0xb2, 0x3e, 0x12, 0x00, 0x00,
// 1671 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0xcb, 0x6f, 0xdb, 0x46,
0x1a, 0x37, 0xf5, 0xe6, 0x27, 0x29, 0xb1, 0x47, 0x76, 0xa2, 0xc8, 0x8f, 0x75, 0xe8, 0x4d, 0xd6,
0x8b, 0x04, 0xde, 0xc0, 0x9b, 0x43, 0xb2, 0xd9, 0x3d, 0x24, 0x7e, 0x2c, 0x8c, 0x75, 0x1e, 0xa0,
0x93, 0xc5, 0x2e, 0x0a, 0x94, 0xa0, 0xc9, 0x91, 0x3c, 0x35, 0xc5, 0x61, 0x87, 0x43, 0xdb, 0xe9,
0x9f, 0x52, 0xa0, 0x7f, 0x45, 0xaf, 0x45, 0x2f, 0x45, 0xd1, 0x1e, 0xfa, 0xb7, 0xf4, 0xd8, 0x73,
0x31, 0x33, 0x24, 0x35, 0x14, 0x65, 0x3b, 0x41, 0x91, 0x1b, 0xe7, 0x7b, 0xcd, 0x37, 0xbf, 0xef,
0x29, 0x41, 0x7b, 0x48, 0x02, 0xcc, 0xb6, 0x22, 0x46, 0x39, 0x45, 0x2d, 0x79, 0x70, 0xa2, 0x63,
0xeb, 0x35, 0x2c, 0x1f, 0x52, 0x7a, 0x9a, 0x44, 0xbb, 0x84, 0x61, 0x8f, 0x53, 0xf6, 0x7e, 0x2f,
0xe4, 0xec, 0xbd, 0x8d, 0xbf, 0x4c, 0x70, 0xcc, 0xd1, 0x0a, 0x98, 0x7e, 0xc6, 0xe8, 0x1b, 0xeb,
0xc6, 0xa6, 0x69, 0x4f, 0x08, 0x08, 0x41, 0x2d, 0x74, 0xc7, 0xb8, 0x5f, 0x91, 0x0c, 0xf9, 0x6d,
0xed, 0xc1, 0xca, 0x6c, 0x83, 0x71, 0x44, 0xc3, 0x18, 0xa3, 0x7b, 0x50, 0xc7, 0x82, 0x20, 0xad,
0xb5, 0xb7, 0x6f, 0x6e, 0x65, 0xae, 0x6c, 0x29, 0x39, 0xc5, 0xb5, 0xbe, 0x37, 0x00, 0x1d, 0x92,
0x98, 0x0b, 0x22, 0xc1, 0xf1, 0x87, 0xf9, 0x73, 0x0b, 0x1a, 0x11, 0xc3, 0x43, 0x72, 0x91, 0x7a,
0x94, 0x9e, 0xd0, 0x43, 0x58, 0x88, 0xb9, 0xcb, 0xf8, 0x3e, 0xa3, 0xe3, 0x7d, 0x12, 0xe0, 0x57,
0xc2, 0xe9, 0xaa, 0x14, 0x29, 0x33, 0xd0, 0x16, 0x20, 0x12, 0x7a, 0x41, 0x12, 0x93, 0x33, 0x7c,
0x94, 0x71, 0xfb, 0xb5, 0x75, 0x63, 0xb3, 0x65, 0xcf, 0xe0, 0xa0, 0x45, 0xa8, 0x07, 0x64, 0x4c,
0x78, 0xbf, 0xbe, 0x6e, 0x6c, 0x76, 0x6d, 0x75, 0xb0, 0xfe, 0x09, 0xbd, 0x82, 0xff, 0x1f, 0xf7,
0xfc, 0x6f, 0x2a, 0x50, 0x97, 0x84, 0x1c, 0x63, 0x63, 0x82, 0x31, 0xba, 0x0b, 0x1d, 0x12, 0x3b,
0x13, 0x20, 0x2a, 0xd2, 0xb7, 0x36, 0x89, 0x73, 0xcc, 0xd1, 0x03, 0x68, 0x78, 0x27, 0x49, 0x78,
0x1a, 0xf7, 0xab, 0xeb, 0xd5, 0xcd, 0xf6, 0x76, 0x6f, 0x72, 0x91, 0x78, 0xe8, 0x8e, 0xe0, 0xd9,
0xa9, 0x08, 0x7a, 0x02, 0xe0, 0x72, 0xce, 0xc8, 0x71, 0xc2, 0x71, 0x2c, 0x5f, 0xda, 0xde, 0xee,
0x6b, 0x0a, 0x49, 0x8c, 0x9f, 0xe7, 0x7c, 0x5b, 0x93, 0x45, 0x4f, 0xa1, 0x85, 0x2f, 0x38, 0x0e,
0x7d, 0xec, 0xf7, 0xeb, 0xf2, 0xa2, 0xd5, 0xa9, 0x17, 0x6d, 0xed, 0xa5, 0x7c, 0xf5, 0xbe, 0x5c,
0x7c, 0xf0, 0x0c, 0xba, 0x05, 0x16, 0x9a, 0x87, 0xea, 0x29, 0xce, 0xa2, 0x2a, 0x3e, 0x05, 0xb2,
0x67, 0x6e, 0x90, 0xa8, 0x04, 0xeb, 0xd8, 0xea, 0xf0, 0x8f, 0xca, 0x13, 0xc3, 0xda, 0x05, 0x73,
0x3f, 0x09, 0x82, 0x5c, 0xd1, 0x27, 0x2c, 0x53, 0xf4, 0x09, 0x9b, 0xa0, 0x5c, 0xb9, 0x12, 0xe5,
0xef, 0x0c, 0x58, 0xd8, 0x3b, 0xc3, 0x21, 0x7f, 0x45, 0x39, 0x19, 0x12, 0xcf, 0xe5, 0x84, 0x86,
0xe8, 0x21, 0x98, 0x34, 0xf0, 0x9d, 0x2b, 0xc3, 0xd4, 0xa2, 0x41, 0xea, 0xf5, 0x43, 0x30, 0x43,
0x7c, 0xee, 0x5c, 0x79, 0x5d, 0x2b, 0xc4, 0xe7, 0x4a, 0x7a, 0x03, 0xba, 0x3e, 0x0e, 0x30, 0xc7,
0x4e, 0x1e, 0x1d, 0x11, 0xba, 0x8e, 0x22, 0xee, 0xa8, 0x70, 0xdc, 0x87, 0x9b, 0xc2, 0x64, 0xe4,
0x32, 0x1c, 0x72, 0x27, 0x72, 0xf9, 0x89, 0x8c, 0x89, 0x69, 0x77, 0x43, 0x7c, 0xfe, 0x46, 0x52,
0xdf, 0xb8, 0xfc, 0xc4, 0xfa, 0xcd, 0x00, 0x33, 0x0f, 0x26, 0xba, 0x0d, 0x4d, 0x71, 0xad, 0x43,
0xfc, 0x14, 0x89, 0x86, 0x38, 0x1e, 0xf8, 0xa2, 0x2a, 0xe8, 0x70, 0x18, 0x63, 0x2e, 0xdd, 0xab,
0xda, 0xe9, 0x49, 0x64, 0x56, 0x4c, 0xbe, 0x52, 0x85, 0x50, 0xb3, 0xe5, 0xb7, 0x40, 0x7c, 0xcc,
0xc9, 0x18, 0xcb, 0x0b, 0xab, 0xb6, 0x3a, 0xa0, 0x1e, 0xd4, 0xb1, 0xc3, 0xdd, 0x91, 0xcc, 0x70,
0xd3, 0xae, 0xe1, 0xb7, 0xee, 0x08, 0xfd, 0x19, 0x6e, 0xc4, 0x34, 0x61, 0x1e, 0x76, 0xb2, 0x6b,
0x1b, 0x92, 0xdb, 0x51, 0xd4, 0x7d, 0x75, 0xb9, 0x05, 0xd5, 0x21, 0xf1, 0xfb, 0x4d, 0x09, 0xcc,
0x7c, 0x31, 0x09, 0x0f, 0x7c, 0x5b, 0x30, 0xd1, 0xdf, 0x00, 0x72, 0x4b, 0x7e, 0xbf, 0x75, 0x89,
0xa8, 0x99, 0xd9, 0xf5, 0xad, 0xff, 0x41, 0x23, 0x35, 0xbf, 0x0c, 0xe6, 0x19, 0x0d, 0x92, 0x71,
0xfe, 0xec, 0xae, 0xdd, 0x52, 0x84, 0x03, 0x1f, 0xdd, 0x01, 0xd9, 0xe7, 0x1c, 0x91, 0x55, 0x15,
0xf9, 0x48, 0x89, 0xd0, 0x7f, 0xb0, 0xec, 0x14, 0x1e, 0xa5, 0xa7, 0x44, 0xbd, 0xbe, 0x69, 0xa7,
0x27, 0xeb, 0xd7, 0x0a, 0xdc, 0x28, 0xa6, 0xbb, 0xb8, 0x42, 0x5a, 0x91, 0x58, 0x19, 0xd2, 0x8c,
0x34, 0x7b, 0x54, 0xc0, 0xab, 0xa2, 0xe3, 0x95, 0xa9, 0x8c, 0xa9, 0xaf, 0x2e, 0xe8, 0x2a, 0x95,
0x97, 0xd4, 0xc7, 0x22, 0x5b, 0x13, 0xe2, 0x4b, 0x80, 0xbb, 0xb6, 0xf8, 0x14, 0x94, 0x11, 0xf1,
0xd3, 0xf6, 0x21, 0x3e, 0xa5, 0x7b, 0x4c, 0xda, 0x6d, 0xa8, 0x90, 0xa9, 0x93, 0x08, 0xd9, 0x58,
0x50, 0x9b, 0x2a, 0x0e, 0xe2, 0x1b, 0xad, 0x43, 0x9b, 0xe1, 0x28, 0x48, 0xb3, 0x57, 0xc2, 0x67,
0xda, 0x3a, 0x09, 0xad, 0x01, 0x78, 0x34, 0x08, 0xb0, 0x27, 0x05, 0x4c, 0x29, 0xa0, 0x51, 0x44,
0xe6, 0x70, 0x1e, 0x38, 0x31, 0xf6, 0xfa, 0xb0, 0x6e, 0x6c, 0xd6, 0xed, 0x06, 0xe7, 0xc1, 0x11,
0xf6, 0xc4, 0x3b, 0x92, 0x18, 0x33, 0x47, 0x36, 0xa0, 0xb6, 0xd4, 0x6b, 0x09, 0x82, 0x6c, 0x93,
0xab, 0x00, 0x23, 0x46, 0x93, 0x48, 0x71, 0x3b, 0xeb, 0x55, 0xd1, 0x8b, 0x25, 0x45, 0xb2, 0xef,
0xc1, 0x8d, 0xf8, 0xfd, 0x38, 0x20, 0xe1, 0xa9, 0xc3, 0x5d, 0x36, 0xc2, 0xbc, 0xdf, 0x55, 0x39,
0x9c, 0x52, 0xdf, 0x4a, 0xa2, 0x15, 0x01, 0xda, 0x61, 0xd8, 0xe5, 0xf8, 0x23, 0xc6, 0xce, 0x87,
0x55, 0x37, 0x5a, 0x82, 0x06, 0x75, 0xf0, 0x85, 0x17, 0xa4, 0x45, 0x56, 0xa7, 0x7b, 0x17, 0x5e,
0x60, 0x3d, 0x80, 0x5e, 0xe1, 0xc6, 0xb4, 0x31, 0x2f, 0x42, 0x1d, 0x33, 0x46, 0xb3, 0x36, 0xa2,
0x0e, 0xd6, 0xff, 0x01, 0xbd, 0x8b, 0xfc, 0x4f, 0xe1, 0x9e, 0xb5, 0x04, 0xbd, 0x82, 0x69, 0xe5,
0x87, 0xf5, 0xa3, 0x01, 0x68, 0x57, 0x76, 0x83, 0x3f, 0x36, 0x88, 0x45, 0x7d, 0x8a, 0x21, 0xa1,
0xba, 0x8d, 0xef, 0x72, 0x37, 0x1d, 0x61, 0x1d, 0x12, 0x2b, 0xfb, 0xbb, 0x2e, 0x77, 0xd3, 0x51,
0xc2, 0xb0, 0x97, 0x30, 0x31, 0xd5, 0x64, 0x12, 0xca, 0x51, 0x62, 0x67, 0x24, 0xf4, 0x18, 0x6e,
0x91, 0x51, 0x48, 0x19, 0x9e, 0x88, 0x39, 0x0a, 0xaa, 0x86, 0x14, 0x5e, 0x54, 0xdc, 0x5c, 0x61,
0x4f, 0x22, 0xb7, 0x04, 0xbd, 0xc2, 0x33, 0xd2, 0xe7, 0x7d, 0x6d, 0x40, 0xff, 0x39, 0xa7, 0x63,
0xe2, 0xd9, 0x58, 0xb8, 0x59, 0x78, 0xe4, 0x06, 0x74, 0x45, 0xe7, 0x9d, 0x7e, 0x68, 0x87, 0x06,
0xfe, 0x64, 0xb2, 0xdd, 0x01, 0xd1, 0x7c, 0x1d, 0xed, 0xbd, 0x4d, 0x1a, 0xf8, 0x32, 0xe7, 0x36,
0x40, 0x74, 0x48, 0x4d, 0x5f, 0xcd, 0xf8, 0x4e, 0x88, 0xcf, 0x0b, 0xfa, 0x42, 0x48, 0xea, 0xab,
0xb6, 0xda, 0x0c, 0xf1, 0xb9, 0xd0, 0xb7, 0x96, 0xe1, 0xce, 0x0c, 0xdf, 0x52, 0xcf, 0x7f, 0x36,
0xa0, 0xf7, 0x3c, 0x8e, 0xc9, 0x28, 0xfc, 0xaf, 0x6c, 0x30, 0x99, 0xd3, 0x8b, 0x50, 0xf7, 0x68,
0x12, 0x72, 0xe9, 0x6c, 0xdd, 0x56, 0x87, 0xa9, 0x9a, 0xab, 0x94, 0x6a, 0x6e, 0xaa, 0x6a, 0xab,
0xe5, 0xaa, 0xd5, 0xaa, 0xb2, 0x56, 0xa8, 0xca, 0x3f, 0x41, 0x5b, 0x84, 0xd3, 0xf1, 0x70, 0xc8,
0x31, 0x4b, 0x7b, 0x32, 0x08, 0xd2, 0x8e, 0xa4, 0x08, 0x01, 0x7d, 0x76, 0xa8, 0xb6, 0x0c, 0xd1,
0x64, 0x70, 0xfc, 0x62, 0xc0, 0x62, 0xf1, 0x29, 0x69, 0x11, 0x5c, 0x3a, 0x43, 0x44, 0xd3, 0x62,
0x41, 0xfa, 0x0e, 0xf1, 0x29, 0xca, 0x3f, 0x4a, 0x8e, 0x03, 0xe2, 0x39, 0x82, 0xa1, 0xfc, 0x37,
0x15, 0xe5, 0x1d, 0x0b, 0x26, 0xa8, 0xd4, 0x74, 0x54, 0x10, 0xd4, 0xdc, 0x84, 0x9f, 0x64, 0x73,
0x44, 0x7c, 0x4f, 0x21, 0xd5, 0xb8, 0x0e, 0xa9, 0x66, 0x09, 0x29, 0xeb, 0x31, 0xf4, 0xd4, 0xca,
0x59, 0x0c, 0xcc, 0x2a, 0x40, 0x3e, 0x1b, 0xe2, 0xbe, 0xa1, 0x1a, 0x54, 0x36, 0x1c, 0x62, 0xeb,
0x5f, 0x60, 0x1e, 0x52, 0x65, 0x21, 0x46, 0x8f, 0xc0, 0x0c, 0xb2, 0x83, 0x14, 0x6d, 0x6f, 0xa3,
0x49, 0xdd, 0x66, 0x72, 0xf6, 0x44, 0xc8, 0x7a, 0x06, 0xad, 0x8c, 0x9c, 0xa1, 0x63, 0x5c, 0x86,
0x4e, 0x65, 0x0a, 0x1d, 0xeb, 0x07, 0x03, 0x16, 0x8b, 0x2e, 0xa7, 0x01, 0x78, 0x07, 0xdd, 0xfc,
0x0a, 0x67, 0xec, 0x46, 0xa9, 0x2f, 0x8f, 0x74, 0x5f, 0xca, 0x6a, 0xb9, 0x83, 0xf1, 0x4b, 0x37,
0x52, 0x59, 0xdb, 0x09, 0x34, 0xd2, 0xe0, 0x2d, 0x2c, 0x94, 0x44, 0x66, 0xec, 0x5b, 0x7f, 0xd5,
0xf7, 0xad, 0xc2, 0xce, 0x98, 0x6b, 0xeb, 0x4b, 0xd8, 0x53, 0xb8, 0xad, 0x4a, 0x7c, 0x27, 0x8f,
0x56, 0x86, 0x7d, 0x31, 0xa8, 0xc6, 0x74, 0x50, 0xad, 0x01, 0xf4, 0xcb, 0xaa, 0x69, 0xa1, 0x8d,
0x60, 0xe1, 0x88, 0xbb, 0x9c, 0xc4, 0x9c, 0x78, 0xf9, 0xe2, 0x3f, 0x95, 0x05, 0xc6, 0x75, 0x53,
0xae, 0x5c, 0x71, 0xf3, 0x50, 0xe5, 0x3c, 0xcb, 0x54, 0xf1, 0x29, 0xa2, 0x80, 0xf4, 0x9b, 0xd2,
0x18, 0x7c, 0x82, 0xab, 0x44, 0x3e, 0x70, 0xca, 0xdd, 0x40, 0x6d, 0x11, 0x35, 0xb9, 0x45, 0x98,
0x92, 0x22, 0xd7, 0x08, 0x35, 0x68, 0x7d, 0xc5, 0xad, 0xab, 0x1d, 0x43, 0x10, 0x24, 0x73, 0x15,
0x40, 0x16, 0xa5, 0xaa, 0xa7, 0x86, 0xd2, 0x15, 0x94, 0x1d, 0x41, 0xb0, 0xd6, 0x60, 0xe5, 0xdf,
0x98, 0x8b, 0x7d, 0x88, 0xed, 0xd0, 0x70, 0x48, 0x46, 0x09, 0x73, 0xb5, 0x50, 0x58, 0xdf, 0x1a,
0xb0, 0x7a, 0x89, 0x40, 0xfa, 0xe0, 0x3e, 0x34, 0xc7, 0x6e, 0xcc, 0x31, 0xcb, 0xaa, 0x24, 0x3b,
0x4e, 0x43, 0x51, 0xb9, 0x0e, 0x8a, 0x6a, 0x09, 0x8a, 0x25, 0x68, 0x8c, 0xdd, 0x0b, 0x67, 0x7c,
0x9c, 0x2e, 0x3c, 0xf5, 0xb1, 0x7b, 0xf1, 0xf2, 0x58, 0xf6, 0x30, 0xc2, 0x9c, 0xe3, 0xc4, 0x3b,
0xc5, 0x3c, 0xce, 0x7b, 0x18, 0x61, 0x2f, 0x14, 0x65, 0xfb, 0xa7, 0x26, 0x74, 0x8e, 0xb0, 0x7b,
0x8e, 0xb1, 0x2f, 0x3d, 0x47, 0xa3, 0xac, 0x62, 0x8a, 0xbf, 0x2b, 0xd1, 0xbd, 0xe9, 0xd2, 0x98,
0xf9, 0x43, 0x76, 0x70, 0xff, 0x3a, 0xb1, 0x34, 0xf9, 0xe6, 0xd0, 0x2b, 0x68, 0x6b, 0x3f, 0xdc,
0xd0, 0x8a, 0xa6, 0x58, 0xfa, 0x3d, 0x3a, 0x58, 0xbd, 0x84, 0x9b, 0x59, 0x7b, 0x64, 0xa0, 0x43,
0x68, 0x6b, 0xfb, 0x86, 0x6e, 0xaf, 0xbc, 0xf8, 0xe8, 0xf6, 0x66, 0x2c, 0x29, 0xd6, 0x9c, 0xb0,
0xa6, 0x6d, 0x0d, 0xba, 0xb5, 0xf2, 0x9e, 0xa2, 0x5b, 0x9b, 0xb5, 0x6a, 0x48, 0x6b, 0xda, 0x90,
0xd6, 0xad, 0x95, 0x57, 0x10, 0xdd, 0xda, 0xac, 0xc9, 0x3e, 0x87, 0x3e, 0x87, 0x85, 0xd2, 0xf8,
0x44, 0xd6, 0x44, 0xeb, 0xb2, 0xb9, 0x3f, 0xd8, 0xb8, 0x52, 0x26, 0xb7, 0xff, 0x1a, 0x3a, 0xfa,
0xd4, 0x42, 0x9a, 0x43, 0x33, 0x06, 0xf3, 0x60, 0xed, 0x32, 0xb6, 0x6e, 0x50, 0x6f, 0xa7, 0xba,
0xc1, 0x19, 0x03, 0x45, 0x37, 0x38, 0xab, 0x0b, 0x5b, 0x73, 0xe8, 0x33, 0x98, 0x9f, 0x6e, 0x6b,
0xe8, 0xee, 0x34, 0x6c, 0xa5, 0x6e, 0x39, 0xb0, 0xae, 0x12, 0xc9, 0x8d, 0x1f, 0x00, 0x4c, 0xba,
0x15, 0x5a, 0x9e, 0xe8, 0x94, 0xba, 0xe5, 0x60, 0x65, 0x36, 0x33, 0x37, 0xf5, 0x05, 0x2c, 0xcd,
0x6c, 0x09, 0x48, 0x2b, 0x93, 0xab, 0x9a, 0xca, 0xe0, 0x2f, 0xd7, 0xca, 0x65, 0x77, 0xbd, 0x58,
0x83, 0xf9, 0x58, 0x15, 0xf2, 0x30, 0xde, 0xf2, 0x02, 0x82, 0x43, 0xfe, 0x02, 0xa4, 0xc6, 0x1b,
0x46, 0x39, 0x3d, 0x6e, 0xc8, 0xbf, 0xa4, 0xfe, 0xfe, 0x7b, 0x00, 0x00, 0x00, 0xff, 0xff, 0xaa,
0x2b, 0xd6, 0xf6, 0xa1, 0x12, 0x00, 0x00,
}

View file

@ -3,10 +3,11 @@ package filersink
import (
"context"
"fmt"
"google.golang.org/grpc"
"strings"
"sync"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
@ -14,7 +15,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/util"
)
func (fs *FilerSink) replicateChunks(ctx context.Context, sourceChunks []*filer_pb.FileChunk) (replicatedChunks []*filer_pb.FileChunk, err error) {
func (fs *FilerSink) replicateChunks(ctx context.Context, sourceChunks []*filer_pb.FileChunk, dir string) (replicatedChunks []*filer_pb.FileChunk, err error) {
if len(sourceChunks) == 0 {
return
}
@ -23,7 +24,7 @@ func (fs *FilerSink) replicateChunks(ctx context.Context, sourceChunks []*filer_
wg.Add(1)
go func(chunk *filer_pb.FileChunk) {
defer wg.Done()
replicatedChunk, e := fs.replicateOneChunk(ctx, chunk)
replicatedChunk, e := fs.replicateOneChunk(ctx, chunk, dir)
if e != nil {
err = e
}
@ -35,9 +36,9 @@ func (fs *FilerSink) replicateChunks(ctx context.Context, sourceChunks []*filer_
return
}
func (fs *FilerSink) replicateOneChunk(ctx context.Context, sourceChunk *filer_pb.FileChunk) (*filer_pb.FileChunk, error) {
func (fs *FilerSink) replicateOneChunk(ctx context.Context, sourceChunk *filer_pb.FileChunk, dir string) (*filer_pb.FileChunk, error) {
fileId, err := fs.fetchAndWrite(ctx, sourceChunk)
fileId, err := fs.fetchAndWrite(ctx, sourceChunk, dir)
if err != nil {
return nil, fmt.Errorf("copy %s: %v", sourceChunk.GetFileIdString(), err)
}
@ -52,7 +53,7 @@ func (fs *FilerSink) replicateOneChunk(ctx context.Context, sourceChunk *filer_p
}, nil
}
func (fs *FilerSink) fetchAndWrite(ctx context.Context, sourceChunk *filer_pb.FileChunk) (fileId string, err error) {
func (fs *FilerSink) fetchAndWrite(ctx context.Context, sourceChunk *filer_pb.FileChunk, dir string) (fileId string, err error) {
filename, header, readCloser, err := fs.filerSource.ReadPart(ctx, sourceChunk.GetFileIdString())
if err != nil {
@ -71,6 +72,7 @@ func (fs *FilerSink) fetchAndWrite(ctx context.Context, sourceChunk *filer_pb.Fi
Collection: fs.collection,
TtlSec: fs.ttlSec,
DataCenter: fs.dataCenter,
ParentPath: dir,
}
resp, err := client.AssignVolume(ctx, request)

View file

@ -105,7 +105,7 @@ func (fs *FilerSink) CreateEntry(ctx context.Context, key string, entry *filer_p
}
}
replicatedChunks, err := fs.replicateChunks(ctx, entry.Chunks)
replicatedChunks, err := fs.replicateChunks(ctx, entry.Chunks, dir)
if err != nil {
glog.V(0).Infof("replicate entry chunks %s: %v", key, err)
@ -184,7 +184,7 @@ func (fs *FilerSink) UpdateEntry(ctx context.Context, key string, oldEntry *file
}
// replicate the chunks that are new in the source
replicatedChunks, err := fs.replicateChunks(ctx, newChunks)
replicatedChunks, err := fs.replicateChunks(ctx, newChunks, newParentPath)
if err != nil {
return true, fmt.Errorf("replicte %s chunks error: %v", key, err)
}

View file

@ -52,8 +52,7 @@ func (s3a *S3ApiServer) PutObjectHandler(w http.ResponseWriter, r *http.Request)
}
defer dataReader.Close()
uploadUrl := fmt.Sprintf("http://%s%s/%s%s?collection=%s",
s3a.option.Filer, s3a.option.BucketsPath, bucket, object, bucket)
uploadUrl := fmt.Sprintf("http://%s%s/%s%s", s3a.option.Filer, s3a.option.BucketsPath, bucket, object)
etag, errCode := s3a.putToFiler(r, uploadUrl, dataReader)
@ -167,7 +166,7 @@ func passThroughResponse(proxyResonse *http.Response, w http.ResponseWriter) {
func (s3a *S3ApiServer) putToFiler(r *http.Request, uploadUrl string, dataReader io.Reader) (etag string, code ErrorCode) {
hash := md5.New()
var body io.Reader = io.TeeReader(dataReader, hash)
var body = io.TeeReader(dataReader, hash)
proxyReq, err := http.NewRequest("PUT", uploadUrl, body)

View file

@ -54,6 +54,7 @@ func (fs *FilerServer) ListEntries(req *filer_pb.ListEntriesRequest, stream file
includeLastFile := req.InclusiveStartFrom
for limit > 0 {
entries, err := fs.filer.ListDirectoryEntries(stream.Context(), filer2.FullPath(req.Directory), lastFileName, includeLastFile, paginationLimit)
if err != nil {
return err
}
@ -84,6 +85,7 @@ func (fs *FilerServer) ListEntries(req *filer_pb.ListEntriesRequest, stream file
}); err != nil {
return err
}
limit--
if limit == 0 {
return nil
@ -226,6 +228,7 @@ func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVol
if req.TtlSec > 0 {
ttlStr = strconv.Itoa(int(req.TtlSec))
}
collection, replication := fs.detectCollection(req.ParentPath, req.Collection, req.Replication)
var altRequest *operation.VolumeAssignRequest
@ -236,16 +239,16 @@ func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVol
assignRequest := &operation.VolumeAssignRequest{
Count: uint64(req.Count),
Replication: req.Replication,
Collection: req.Collection,
Replication: replication,
Collection: collection,
Ttl: ttlStr,
DataCenter: dataCenter,
}
if dataCenter != "" {
altRequest = &operation.VolumeAssignRequest{
Count: uint64(req.Count),
Replication: req.Replication,
Collection: req.Collection,
Replication: replication,
Collection: collection,
Ttl: ttlStr,
DataCenter: "",
}
@ -261,11 +264,13 @@ func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVol
}
return &filer_pb.AssignVolumeResponse{
FileId: assignResult.Fid,
Count: int32(assignResult.Count),
Url: assignResult.Url,
PublicUrl: assignResult.PublicUrl,
Auth: string(assignResult.Auth),
FileId: assignResult.Fid,
Count: int32(assignResult.Count),
Url: assignResult.Url,
PublicUrl: assignResult.PublicUrl,
Auth: string(assignResult.Auth),
Collection: collection,
Replication: replication,
}, err
}

View file

@ -67,7 +67,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
glog.Fatal("master list is required!")
}
fs.filer = filer2.NewFiler(option.Masters, fs.grpcDialOption)
fs.filer = filer2.NewFiler(option.Masters, fs.grpcDialOption, option.DirBucketsPath)
go fs.filer.KeepConnectedToMaster()
@ -83,6 +83,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
util.LoadConfiguration("notification", false)
fs.option.recursiveDelete = v.GetBool("filer.options.recursive_delete")
v.Set("filer.option.buckets_folder", "/buckets")
fs.option.DirBucketsPath = v.GetString("filer.option.buckets_folder")
fs.filer.LoadConfiguration(v)
@ -96,6 +97,8 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
readonlyMux.HandleFunc("/", fs.readonlyFilerHandler)
}
fs.filer.LoadBuckets(fs.option.DirBucketsPath)
maybeStartMetrics(fs, option)
return fs, nil

View file

@ -80,14 +80,7 @@ func (fs *FilerServer) PostHandler(w http.ResponseWriter, r *http.Request) {
ctx := context.Background()
query := r.URL.Query()
replication := query.Get("replication")
if replication == "" {
replication = fs.option.DefaultReplication
}
collection := query.Get("collection")
if collection == "" {
collection = fs.option.Collection
}
collection, replication := fs.detectCollection(r.RequestURI, query.Get("collection"), query.Get("replication"))
dataCenter := query.Get("dataCenter")
if dataCenter == "" {
dataCenter = fs.option.DataCenter
@ -305,3 +298,32 @@ func (fs *FilerServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
w.WriteHeader(http.StatusNoContent)
}
func (fs *FilerServer) detectCollection(requestURI, qCollection, qReplication string) (collection, replication string) {
// default
collection = fs.option.Collection
replication = fs.option.DefaultReplication
// get default collection settings
if qCollection != "" {
collection = qCollection
}
if qReplication != "" {
replication = qReplication
}
// required by buckets folder
if strings.HasPrefix(requestURI, fs.filer.DirBucketsPath+"/") {
bucketAndObjectKey := requestURI[len(fs.filer.DirBucketsPath)+1:]
t := strings.Index(bucketAndObjectKey, "/")
if t < 0 {
collection = bucketAndObjectKey
}
if t > 0 {
collection = bucketAndObjectKey[:t]
}
replication = fs.filer.ReadBucketOption(collection)
}
return
}

View file

@ -367,6 +367,8 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
glog.V(2).Infof("WebDavFileSystem.Write %v", f.name)
dir, _ := filer2.FullPath(f.name).DirAndName()
var err error
ctx := context.Background()
if f.entry == nil {
@ -382,13 +384,15 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
var fileId, host string
var auth security.EncodedJwt
var collection, replication string
if err = f.fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
Replication: "000",
Replication: "",
Collection: f.fs.option.Collection,
ParentPath: dir,
}
resp, err := client.AssignVolume(ctx, request)
@ -398,6 +402,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
}
fileId, host, auth = resp.FileId, resp.Url, security.EncodedJwt(resp.Auth)
collection, replication = resp.Collection, resp.Replication
return nil
}); err != nil {
@ -425,10 +430,11 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
}
f.entry.Chunks = append(f.entry.Chunks, chunk)
dir, _ := filer2.FullPath(f.name).DirAndName()
err = f.fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
f.entry.Attributes.Mtime = time.Now().Unix()
f.entry.Attributes.Collection = collection
f.entry.Attributes.Replication = replication
request := &filer_pb.UpdateEntryRequest{
Directory: dir,