Browse Source

filer: dynamically create bucket under /buckets folder

pull/1213/head
Chris Lu 5 years ago
parent
commit
6ab7368ef2
  1. 3
      other/java/client/src/main/proto/filer.proto
  2. 75
      weed/command/filer_copy.go
  3. 2
      weed/command/mount_std.go
  4. 1
      weed/command/s3.go
  5. 18
      weed/filer2/filer.go
  6. 113
      weed/filer2/filer_buckets.go
  7. 32
      weed/filer2/filer_delete_entry.go
  8. 4
      weed/filer2/leveldb/leveldb_store_test.go
  9. 4
      weed/filer2/leveldb2/leveldb2_store_test.go
  10. 12
      weed/filesys/dirty_page.go
  11. 2
      weed/filesys/filehandle.go
  12. 6
      weed/operation/assign_file_id.go
  13. 3
      weed/pb/filer.proto
  14. 245
      weed/pb/filer_pb/filer.pb.go
  15. 14
      weed/replication/sink/filersink/fetch_write.go
  16. 4
      weed/replication/sink/filersink/filer_sink.go
  17. 5
      weed/s3api/s3api_object_handlers.go
  18. 23
      weed/server/filer_grpc_server.go
  19. 5
      weed/server/filer_server.go
  20. 38
      weed/server/filer_server_handlers_write.go
  21. 10
      weed/server/webdav_server.go

3
other/java/client/src/main/proto/filer.proto

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

75
weed/command/filer_copy.go

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

2
weed/command/mount_std.go

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

1
weed/command/s3.go

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

18
weed/filer2/filer.go

@ -30,9 +30,11 @@ type Filer struct {
MasterClient *wdclient.MasterClient MasterClient *wdclient.MasterClient
fileIdDeletionQueue *util.UnboundedQueue fileIdDeletionQueue *util.UnboundedQueue
GrpcDialOption grpc.DialOption 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{ f := &Filer{
directoryCache: ccache.New(ccache.Configure().MaxSize(1000).ItemsToPrune(100)), directoryCache: ccache.New(ccache.Configure().MaxSize(1000).ItemsToPrune(100)),
MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "filer", masters), 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{ dirEntry = &Entry{
FullPath: FullPath(dirPath), FullPath: FullPath(dirPath),
Attr: Attr{ 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) return fmt.Errorf("mkdir %s: %v", dirPath, mkdirErr)
} }
} else { } else {
f.maybeAddBucket(dirEntry)
f.NotifyUpdateEvent(nil, dirEntry, false) 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.NotifyUpdateEvent(oldEntry, entry, true)
f.deleteChunksIfNotNew(oldEntry, entry) f.deleteChunksIfNotNew(oldEntry, entry)

113
weed/filer2/filer_buckets.go

@ -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))
}

32
weed/filer2/filer_delete_entry.go

@ -6,6 +6,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "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) { 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 return findErr
} }
isCollection := f.isBucket(entry)
var chunks []*filer_pb.FileChunk var chunks []*filer_pb.FileChunk
chunks = append(chunks, entry.Chunks...) chunks = append(chunks, entry.Chunks...)
if entry.IsDirectory() { if entry.IsDirectory() {
// delete the folder children, not including the folder itself // delete the folder children, not including the folder itself
var dirChunks []*filer_pb.FileChunk 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 { if err != nil {
glog.V(0).Infof("delete directory %s: %v", p, err)
return fmt.Errorf("delete directory %s: %v", p, err) return fmt.Errorf("delete directory %s: %v", p, err)
} }
chunks = append(chunks, dirChunks...) chunks = append(chunks, dirChunks...)
f.cacheDelDirectory(string(p))
} }
// delete the file or folder // delete the file or folder
err = f.doDeleteEntryMetaAndData(ctx, entry, shouldDeleteChunks) err = f.doDeleteEntryMetaAndData(ctx, entry, shouldDeleteChunks)
if err != nil { if err != nil {
return fmt.Errorf("delete file %s: %v", p, err) return fmt.Errorf("delete file %s: %v", p, err)
} }
if shouldDeleteChunks {
if shouldDeleteChunks && !isCollection {
go f.DeleteChunks(chunks) go f.DeleteChunks(chunks)
} }
if isCollection {
collectionName := entry.Name()
f.doDeleteCollection(ctx, collectionName)
f.deleteBucket(collectionName)
}
return nil return nil
} }
@ -55,6 +64,9 @@ func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry
} }
if lastFileName == "" && !isRecursive && len(entries) > 0 { if lastFileName == "" && !isRecursive && len(entries) > 0 {
// only for first iteration in the loop // 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) 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 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
})
}

4
weed/filer2/leveldb/leveldb_store_test.go

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

4
weed/filer2/leveldb2/leveldb2_store_test.go

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

12
weed/filesys/dirty_page.go

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

2
weed/filesys/filehandle.go

@ -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.Mtime = time.Now().Unix()
fh.f.entry.Attributes.Crtime = 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.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{ request := &filer_pb.CreateEntryRequest{

6
weed/operation/assign_file_id.go

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

3
weed/pb/filer.proto

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

245
weed/pb/filer_pb/filer.pb.go

@ -685,6 +685,7 @@ type AssignVolumeRequest struct {
Replication string `protobuf:"bytes,3,opt,name=replication" json:"replication,omitempty"` 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"` 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"` 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{} } func (m *AssignVolumeRequest) Reset() { *m = AssignVolumeRequest{} }
@ -727,12 +728,21 @@ func (m *AssignVolumeRequest) GetDataCenter() string {
return "" return ""
} }
func (m *AssignVolumeRequest) GetParentPath() string {
if m != nil {
return m.ParentPath
}
return ""
}
type AssignVolumeResponse struct { 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{} } func (m *AssignVolumeResponse) Reset() { *m = AssignVolumeResponse{} }
@ -775,6 +785,20 @@ func (m *AssignVolumeResponse) GetAuth() string {
return "" 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 { type LookupVolumeRequest struct {
VolumeIds []string `protobuf:"bytes,1,rep,name=volume_ids,json=volumeIds" json:"volume_ids,omitempty"` 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) } func init() { proto.RegisterFile("filer.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{ 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,
} }

14
weed/replication/sink/filersink/fetch_write.go

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

4
weed/replication/sink/filersink/filer_sink.go

@ -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 { if err != nil {
glog.V(0).Infof("replicate entry chunks %s: %v", key, err) 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 // 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 { if err != nil {
return true, fmt.Errorf("replicte %s chunks error: %v", key, err) return true, fmt.Errorf("replicte %s chunks error: %v", key, err)
} }

5
weed/s3api/s3api_object_handlers.go

@ -52,8 +52,7 @@ func (s3a *S3ApiServer) PutObjectHandler(w http.ResponseWriter, r *http.Request)
} }
defer dataReader.Close() 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) 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) { func (s3a *S3ApiServer) putToFiler(r *http.Request, uploadUrl string, dataReader io.Reader) (etag string, code ErrorCode) {
hash := md5.New() hash := md5.New()
var body io.Reader = io.TeeReader(dataReader, hash)
var body = io.TeeReader(dataReader, hash)
proxyReq, err := http.NewRequest("PUT", uploadUrl, body) proxyReq, err := http.NewRequest("PUT", uploadUrl, body)

23
weed/server/filer_grpc_server.go

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

5
weed/server/filer_server.go

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

38
weed/server/filer_server_handlers_write.go

@ -80,14 +80,7 @@ func (fs *FilerServer) PostHandler(w http.ResponseWriter, r *http.Request) {
ctx := context.Background() ctx := context.Background()
query := r.URL.Query() 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") dataCenter := query.Get("dataCenter")
if dataCenter == "" { if dataCenter == "" {
dataCenter = fs.option.DataCenter dataCenter = fs.option.DataCenter
@ -305,3 +298,32 @@ func (fs *FilerServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
w.WriteHeader(http.StatusNoContent) 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
}

10
weed/server/webdav_server.go

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

Loading…
Cancel
Save