diff options
Diffstat (limited to 'weed')
97 files changed, 4830 insertions, 4141 deletions
diff --git a/weed/command/filer.go b/weed/command/filer.go index 633e02d79..876b1bbf0 100644 --- a/weed/command/filer.go +++ b/weed/command/filer.go @@ -84,6 +84,7 @@ func init() { filerS3Options.tlsPrivateKey = cmdFiler.Flag.String("s3.key.file", "", "path to the TLS private key file") filerS3Options.tlsCertificate = cmdFiler.Flag.String("s3.cert.file", "", "path to the TLS certificate file") filerS3Options.config = cmdFiler.Flag.String("s3.config", "", "path to the config file") + filerS3Options.auditLogConfig = cmdFiler.Flag.String("s3.auditLogConfig", "", "path to the audit log config file") filerS3Options.allowEmptyFolder = cmdFiler.Flag.Bool("s3.allowEmptyFolder", true, "allow empty folders") // start webdav on filer @@ -137,6 +138,7 @@ func runFiler(cmd *Command, args []string) bool { startDelay := time.Duration(2) if *filerStartS3 { filerS3Options.filer = &filerAddress + filerS3Options.bindIp = f.bindIp go func() { time.Sleep(startDelay * time.Second) filerS3Options.startS3Server() diff --git a/weed/command/filer_remote_gateway_buckets.go b/weed/command/filer_remote_gateway_buckets.go index d70e96904..fc11cdbc5 100644 --- a/weed/command/filer_remote_gateway_buckets.go +++ b/weed/command/filer_remote_gateway_buckets.go @@ -86,12 +86,26 @@ func (option *RemoteGatewayOptions) makeBucketedEventProcessor(filerSource *sour return nil } } - if *option.createBucketRandomSuffix { - // https://docs.aws.amazon.com/AmazonS3/latest/userguide/bucketnamingrules.html - if len(bucketName)+5 > 63 { - bucketName = bucketName[:58] + + bucketPath := util.FullPath(option.bucketsDir).Child(entry.Name) + remoteLocation, found := option.mappings.Mappings[string(bucketPath)] + if !found { + if *option.createBucketRandomSuffix { + // https://docs.aws.amazon.com/AmazonS3/latest/userguide/bucketnamingrules.html + if len(bucketName)+5 > 63 { + bucketName = bucketName[:58] + } + bucketName = fmt.Sprintf("%s-%04d", bucketName, rand.Uint32()%10000) + } + remoteLocation = &remote_pb.RemoteStorageLocation{ + Name: *option.createBucketAt, + Bucket: bucketName, + Path: "/", } - bucketName = fmt.Sprintf("%s-%4d", bucketName, rand.Uint32()%10000) + // need to add new mapping here before getting updates from metadata tailing + option.mappings.Mappings[string(bucketPath)] = remoteLocation + } else { + bucketName = remoteLocation.Bucket } glog.V(0).Infof("create bucket %s", bucketName) @@ -99,16 +113,6 @@ func (option *RemoteGatewayOptions) makeBucketedEventProcessor(filerSource *sour return fmt.Errorf("create bucket %s in %s: %v", bucketName, remoteConf.Name, err) } - bucketPath := util.FullPath(option.bucketsDir).Child(entry.Name) - remoteLocation := &remote_pb.RemoteStorageLocation{ - Name: *option.createBucketAt, - Bucket: bucketName, - Path: "/", - } - - // need to add new mapping here before getting upates from metadata tailing - option.mappings.Mappings[string(bucketPath)] = remoteLocation - return filer.InsertMountMapping(option, string(bucketPath), remoteLocation) } @@ -177,6 +181,9 @@ func (option *RemoteGatewayOptions) makeBucketedEventProcessor(filerSource *sour if message.NewParentPath == option.bucketsDir { return handleCreateBucket(message.NewEntry) } + if strings.HasPrefix(message.NewParentPath, option.bucketsDir) && strings.Contains(message.NewParentPath, "/.uploads/") { + return nil + } if !filer.HasData(message.NewEntry) { return nil } diff --git a/weed/command/filer_remote_sync.go b/weed/command/filer_remote_sync.go index c55544925..bceeb097e 100644 --- a/weed/command/filer_remote_sync.go +++ b/weed/command/filer_remote_sync.go @@ -40,7 +40,7 @@ func init() { remoteSyncOptions.filerAddress = cmdFilerRemoteSynchronize.Flag.String("filer", "localhost:8888", "filer of the SeaweedFS cluster") remoteSyncOptions.dir = cmdFilerRemoteSynchronize.Flag.String("dir", "", "a mounted directory on filer") remoteSyncOptions.readChunkFromFiler = cmdFilerRemoteSynchronize.Flag.Bool("filerProxy", false, "read file chunks from filer instead of volume servers") - remoteSyncOptions.timeAgo = cmdFilerRemoteSynchronize.Flag.Duration("timeAgo", 0, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"") + remoteSyncOptions.timeAgo = cmdFilerRemoteSynchronize.Flag.Duration("timeAgo", 0, "start time before now, skipping previous metadata changes. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"") } var cmdFilerRemoteSynchronize = &Command{ @@ -54,6 +54,11 @@ var cmdFilerRemoteSynchronize = &Command{ weed filer.remote.sync -dir=/mount/s3_on_cloud + The metadata sync starting time is determined with the following priority order: + 1. specified by timeAgo + 2. last sync timestamp for this directory + 3. directory creation time + `, } diff --git a/weed/command/s3.go b/weed/command/s3.go index e9f4ea885..d7cd7818d 100644 --- a/weed/command/s3.go +++ b/weed/command/s3.go @@ -3,6 +3,7 @@ package command import ( "context" "fmt" + "github.com/chrislusf/seaweedfs/weed/s3api/s3err" "net/http" "time" @@ -24,6 +25,7 @@ var ( type S3Options struct { filer *string + bindIp *string port *int config *string domainName *string @@ -31,14 +33,17 @@ type S3Options struct { tlsCertificate *string metricsHttpPort *int allowEmptyFolder *bool + auditLogConfig *string } func init() { cmdS3.Run = runS3 // break init cycle s3StandaloneOptions.filer = cmdS3.Flag.String("filer", "localhost:8888", "filer server address") + s3StandaloneOptions.bindIp = cmdS3.Flag.String("ip.bind", "", "ip address to bind to") s3StandaloneOptions.port = cmdS3.Flag.Int("port", 8333, "s3 server http listen port") s3StandaloneOptions.domainName = cmdS3.Flag.String("domainName", "", "suffix of the host name in comma separated list, {bucket}.{domainName}") s3StandaloneOptions.config = cmdS3.Flag.String("config", "", "path to the config file") + s3StandaloneOptions.auditLogConfig = cmdS3.Flag.String("auditLogConfig", "", "path to the audit log config file") s3StandaloneOptions.tlsPrivateKey = cmdS3.Flag.String("key.file", "", "path to the TLS private key file") s3StandaloneOptions.tlsCertificate = cmdS3.Flag.String("cert.file", "", "path to the TLS certificate file") s3StandaloneOptions.metricsHttpPort = cmdS3.Flag.Int("metricsPort", 0, "Prometheus metrics listen port") @@ -186,12 +191,19 @@ func (s3opt *S3Options) startS3Server() bool { httpS := &http.Server{Handler: router} - listenAddress := fmt.Sprintf(":%d", *s3opt.port) + listenAddress := fmt.Sprintf("%s:%d", *s3opt.bindIp, *s3opt.port) s3ApiListener, err := util.NewListener(listenAddress, time.Duration(10)*time.Second) if err != nil { glog.Fatalf("S3 API Server listener on %s error: %v", listenAddress, err) } + if len(*s3opt.auditLogConfig) > 0 { + s3err.InitAuditLog(*s3opt.auditLogConfig) + if s3err.Logger != nil { + defer s3err.Logger.Close() + } + } + if *s3opt.tlsPrivateKey != "" { glog.V(0).Infof("Start Seaweed S3 API Server %s at https port %d", util.Version(), *s3opt.port) if err = httpS.ServeTLS(s3ApiListener, *s3opt.tlsCertificate, *s3opt.tlsPrivateKey); err != nil { diff --git a/weed/command/scaffold/filer.toml b/weed/command/scaffold/filer.toml index bf895f484..a6a45923e 100644 --- a/weed/command/scaffold/filer.toml +++ b/weed/command/scaffold/filer.toml @@ -199,6 +199,14 @@ address = "localhost:6379" password = "" database = 0 +[redis3_sentinel] +enabled = false +addresses = ["172.22.12.7:26379","172.22.12.8:26379","172.22.12.9:26379"] +masterName = "master" +username = "" +password = "" +database = 0 + [redis_cluster3] # beta enabled = false addresses = [ diff --git a/weed/command/server.go b/weed/command/server.go index 5c6c4b1cf..0cb748381 100644 --- a/weed/command/server.go +++ b/weed/command/server.go @@ -131,6 +131,7 @@ func init() { s3Options.tlsPrivateKey = cmdServer.Flag.String("s3.key.file", "", "path to the TLS private key file") s3Options.tlsCertificate = cmdServer.Flag.String("s3.cert.file", "", "path to the TLS certificate file") s3Options.config = cmdServer.Flag.String("s3.config", "", "path to the config file") + s3Options.auditLogConfig = cmdServer.Flag.String("s3.auditLogConfig", "", "path to the audit log config file") s3Options.allowEmptyFolder = cmdServer.Flag.Bool("s3.allowEmptyFolder", true, "allow empty folders") webdavOptions.port = cmdServer.Flag.Int("webdav.port", 7333, "webdav server http listen port") @@ -179,6 +180,7 @@ func runServer(cmd *Command, args []string) bool { filerOptions.masters = pb.ServerAddresses(*masterOptions.peers).ToAddresses() filerOptions.ip = serverIp filerOptions.bindIp = serverBindIp + s3Options.bindIp = serverBindIp serverOptions.v.ip = serverIp serverOptions.v.bindIp = serverBindIp serverOptions.v.masters = pb.ServerAddresses(*masterOptions.peers).ToAddresses() diff --git a/weed/filer/filechunk_manifest.go b/weed/filer/filechunk_manifest.go index 32008271b..b6a64b30d 100644 --- a/weed/filer/filechunk_manifest.go +++ b/weed/filer/filechunk_manifest.go @@ -101,6 +101,15 @@ func fetchChunk(lookupFileIdFn wdclient.LookupFileIdFunctionType, fileId string, return retriedFetchChunkData(urlStrings, cipherKey, isGzipped, true, 0, 0) } +func fetchChunkRange(lookupFileIdFn wdclient.LookupFileIdFunctionType, fileId string, cipherKey []byte, isGzipped bool, offset int64, size int) ([]byte, error) { + urlStrings, err := lookupFileIdFn(fileId) + if err != nil { + glog.Errorf("operation LookupFileId %s failed, err: %v", fileId, err) + return nil, err + } + return retriedFetchChunkData(urlStrings, cipherKey, isGzipped, false, offset, size) +} + func retriedFetchChunkData(urlStrings []string, cipherKey []byte, isGzipped bool, isFullChunk bool, offset int64, size int) ([]byte, error) { var err error diff --git a/weed/filer/mongodb/mongodb_store.go b/weed/filer/mongodb/mongodb_store.go index 1ef5056f4..6935be1ab 100644 --- a/weed/filer/mongodb/mongodb_store.go +++ b/weed/filer/mongodb/mongodb_store.go @@ -193,6 +193,10 @@ func (store *MongodbStore) ListDirectoryEntries(ctx context.Context, dirPath uti optLimit := int64(limit) opts := &options.FindOptions{Limit: &optLimit, Sort: bson.M{"name": 1}} cur, err := store.connect.Database(store.database).Collection(store.collectionName).Find(ctx, where, opts) + if err != nil { + return lastFileName, fmt.Errorf("failed to list directory entries: find error: %w", err) + } + for cur.Next(ctx) { var data Model err := cur.Decode(&data) diff --git a/weed/filer/reader_at.go b/weed/filer/reader_at.go index 458cf88be..5f58b870c 100644 --- a/weed/filer/reader_at.go +++ b/weed/filer/reader_at.go @@ -26,6 +26,7 @@ type ChunkReadAt struct { chunkCache chunk_cache.ChunkCache lastChunkFileId string lastChunkData []byte + readerPattern *ReaderPattern } var _ = io.ReaderAt(&ChunkReadAt{}) @@ -88,10 +89,11 @@ func LookupFn(filerClient filer_pb.FilerClient) wdclient.LookupFileIdFunctionTyp func NewChunkReaderAtFromClient(lookupFn wdclient.LookupFileIdFunctionType, chunkViews []*ChunkView, chunkCache chunk_cache.ChunkCache, fileSize int64) *ChunkReadAt { return &ChunkReadAt{ - chunkViews: chunkViews, - lookupFileId: lookupFn, - chunkCache: chunkCache, - fileSize: fileSize, + chunkViews: chunkViews, + lookupFileId: lookupFn, + chunkCache: chunkCache, + fileSize: fileSize, + readerPattern: NewReaderPattern(), } } @@ -106,6 +108,8 @@ func (c *ChunkReadAt) ReadAt(p []byte, offset int64) (n int, err error) { c.readerLock.Lock() defer c.readerLock.Unlock() + c.readerPattern.MonitorReadAt(offset, len(p)) + // glog.V(4).Infof("ReadAt [%d,%d) of total file size %d bytes %d chunk views", offset, offset+int64(len(p)), c.fileSize, len(c.chunkViews)) return c.doReadAt(p, offset) } @@ -171,7 +175,14 @@ func (c *ChunkReadAt) doReadAt(p []byte, offset int64) (n int, err error) { func (c *ChunkReadAt) readChunkSlice(chunkView *ChunkView, nextChunkViews *ChunkView, offset, length uint64) ([]byte, error) { - chunkSlice := c.chunkCache.GetChunkSlice(chunkView.FileId, offset, length) + if c.readerPattern.IsRandomMode() { + return c.doFetchRangeChunkData(chunkView, offset, length) + } + + var chunkSlice []byte + if chunkView.LogicOffset == 0 { + chunkSlice = c.chunkCache.GetChunkSlice(chunkView.FileId, offset, length) + } if len(chunkSlice) > 0 { return chunkSlice, nil } @@ -217,7 +228,10 @@ func (c *ChunkReadAt) readOneWholeChunk(chunkView *ChunkView) (interface{}, erro glog.V(4).Infof("readFromWholeChunkData %s offset %d [%d,%d) size at least %d", chunkView.FileId, chunkView.Offset, chunkView.LogicOffset, chunkView.LogicOffset+int64(chunkView.Size), chunkView.ChunkSize) - data := c.chunkCache.GetChunk(chunkView.FileId, chunkView.ChunkSize) + var data []byte + if chunkView.LogicOffset == 0 { + data = c.chunkCache.GetChunk(chunkView.FileId, chunkView.ChunkSize) + } if data != nil { glog.V(4).Infof("cache hit %s [%d,%d)", chunkView.FileId, chunkView.LogicOffset-chunkView.Offset, chunkView.LogicOffset-chunkView.Offset+int64(len(data))) } else { @@ -226,7 +240,10 @@ func (c *ChunkReadAt) readOneWholeChunk(chunkView *ChunkView) (interface{}, erro if err != nil { return data, err } - c.chunkCache.SetChunk(chunkView.FileId, data) + if chunkView.LogicOffset == 0 { + // only cache the first chunk + c.chunkCache.SetChunk(chunkView.FileId, data) + } } return data, err }) @@ -243,3 +260,15 @@ func (c *ChunkReadAt) doFetchFullChunkData(chunkView *ChunkView) ([]byte, error) return data, err } + +func (c *ChunkReadAt) doFetchRangeChunkData(chunkView *ChunkView, offset, length uint64) ([]byte, error) { + + glog.V(4).Infof("+ doFetchFullChunkData %s", chunkView.FileId) + + data, err := fetchChunkRange(c.lookupFileId, chunkView.FileId, chunkView.CipherKey, chunkView.IsGzipped, int64(offset), int(length)) + + glog.V(4).Infof("- doFetchFullChunkData %s", chunkView.FileId) + + return data, err + +} diff --git a/weed/filer/reader_pattern.go b/weed/filer/reader_pattern.go new file mode 100644 index 000000000..2bf18d141 --- /dev/null +++ b/weed/filer/reader_pattern.go @@ -0,0 +1,31 @@ +package filer + +type ReaderPattern struct { + isStreaming bool + lastReadOffset int64 +} + +// For streaming read: only cache the first chunk +// For random read: only fetch the requested range, instead of the whole chunk + +func NewReaderPattern() *ReaderPattern { + return &ReaderPattern{ + isStreaming: true, + lastReadOffset: 0, + } +} + +func (rp *ReaderPattern) MonitorReadAt(offset int64, size int) { + if rp.lastReadOffset > offset { + rp.isStreaming = false + } + rp.lastReadOffset = offset +} + +func (rp *ReaderPattern) IsStreamingMode() bool { + return rp.isStreaming +} + +func (rp *ReaderPattern) IsRandomMode() bool { + return !rp.isStreaming +} diff --git a/weed/filer/redis3/redis_sentinel_store.go b/weed/filer/redis3/redis_sentinel_store.go new file mode 100644 index 000000000..a87302167 --- /dev/null +++ b/weed/filer/redis3/redis_sentinel_store.go @@ -0,0 +1,49 @@ +package redis3 + +import ( + "time" + + "github.com/chrislusf/seaweedfs/weed/filer" + "github.com/chrislusf/seaweedfs/weed/util" + "github.com/go-redis/redis/v8" + "github.com/go-redsync/redsync/v4" + "github.com/go-redsync/redsync/v4/redis/goredis/v8" +) + +func init() { + filer.Stores = append(filer.Stores, &Redis3SentinelStore{}) +} + +type Redis3SentinelStore struct { + UniversalRedis3Store +} + +func (store *Redis3SentinelStore) GetName() string { + return "redis3_sentinel" +} + +func (store *Redis3SentinelStore) Initialize(configuration util.Configuration, prefix string) (err error) { + return store.initialize( + configuration.GetStringSlice(prefix+"addresses"), + configuration.GetString(prefix+"masterName"), + configuration.GetString(prefix+"username"), + configuration.GetString(prefix+"password"), + configuration.GetInt(prefix+"database"), + ) +} + +func (store *Redis3SentinelStore) initialize(addresses []string, masterName string, username string, password string, database int) (err error) { + store.Client = redis.NewFailoverClient(&redis.FailoverOptions{ + MasterName: masterName, + SentinelAddrs: addresses, + Username: username, + Password: password, + DB: database, + MinRetryBackoff: time.Millisecond * 100, + MaxRetryBackoff: time.Minute * 1, + ReadTimeout: time.Second * 30, + WriteTimeout: time.Second * 5, + }) + store.redsync = redsync.New(goredis.NewPool(store.Client)) + return +} diff --git a/weed/filesys/dir.go b/weed/filesys/dir.go index 9a791e013..cedcf2d76 100644 --- a/weed/filesys/dir.go +++ b/weed/filesys/dir.go @@ -161,7 +161,7 @@ func (dir *Dir) Create(ctx context.Context, req *fuse.CreateRequest, }, } file.dirtyMetadata = true - fh := dir.wfs.AcquireHandle(file, req.Uid, req.Gid, req.Flags&fuse.OpenWriteOnly > 0) + fh := dir.wfs.AcquireHandle(file, req.Uid, req.Gid) return file, fh, nil } diff --git a/weed/filesys/dir_rename.go b/weed/filesys/dir_rename.go index 8a80559f6..1ee6922d8 100644 --- a/weed/filesys/dir_rename.go +++ b/weed/filesys/dir_rename.go @@ -111,7 +111,7 @@ func (dir *Dir) handleRenameResponse(ctx context.Context, resp *filer_pb.StreamR // change file handle inodeId := oldPath.AsInode() dir.wfs.handlesLock.Lock() - if existingHandle, found := dir.wfs.handles[inodeId]; found && existingHandle == nil { + if existingHandle, found := dir.wfs.handles[inodeId]; found && existingHandle != nil { glog.V(4).Infof("opened file handle %s => %s", oldPath, newPath) delete(dir.wfs.handles, inodeId) dir.wfs.handles[newPath.AsInode()] = existingHandle diff --git a/weed/filesys/dirty_pages_continuous.go b/weed/filesys/dirty_pages_continuous.go index b7514a2eb..88b50ce41 100644 --- a/weed/filesys/dirty_pages_continuous.go +++ b/weed/filesys/dirty_pages_continuous.go @@ -3,6 +3,7 @@ package filesys import ( "bytes" "fmt" + "github.com/chrislusf/seaweedfs/weed/filesys/page_writer" "io" "sync" "time" @@ -12,9 +13,8 @@ import ( ) type ContinuousDirtyPages struct { - intervals *ContinuousIntervals + intervals *page_writer.ContinuousIntervals f *File - writeOnly bool writeWaitGroup sync.WaitGroup chunkAddLock sync.Mutex lastErr error @@ -22,11 +22,10 @@ type ContinuousDirtyPages struct { replication string } -func newContinuousDirtyPages(file *File, writeOnly bool) *ContinuousDirtyPages { +func newContinuousDirtyPages(file *File) *ContinuousDirtyPages { dirtyPages := &ContinuousDirtyPages{ - intervals: &ContinuousIntervals{}, + intervals: &page_writer.ContinuousIntervals{}, f: file, - writeOnly: writeOnly, } return dirtyPages } @@ -107,7 +106,7 @@ func (pages *ContinuousDirtyPages) saveToStorage(reader io.Reader, offset int64, defer pages.writeWaitGroup.Done() reader = io.LimitReader(reader, size) - chunk, collection, replication, err := pages.f.wfs.saveDataAsChunk(pages.f.fullpath(), pages.writeOnly)(reader, pages.f.Name, offset) + chunk, collection, replication, err := pages.f.wfs.saveDataAsChunk(pages.f.fullpath())(reader, pages.f.Name, offset) if err != nil { glog.V(0).Infof("%s saveToStorage [%d,%d): %v", pages.f.fullpath(), offset, offset+size, err) pages.lastErr = err @@ -148,13 +147,3 @@ func (pages *ContinuousDirtyPages) ReadDirtyDataAt(data []byte, startOffset int6 func (pages *ContinuousDirtyPages) GetStorageOptions() (collection, replication string) { return pages.collection, pages.replication } - -func (pages *ContinuousDirtyPages) SetWriteOnly(writeOnly bool) { - if pages.writeOnly { - pages.writeOnly = writeOnly - } -} - -func (pages *ContinuousDirtyPages) GetWriteOnly() (writeOnly bool) { - return pages.writeOnly -} diff --git a/weed/filesys/dirty_pages_temp_file.go b/weed/filesys/dirty_pages_temp_file.go index 9fa7c0c8e..6a22889dc 100644 --- a/weed/filesys/dirty_pages_temp_file.go +++ b/weed/filesys/dirty_pages_temp_file.go @@ -2,6 +2,7 @@ package filesys import ( "fmt" + "github.com/chrislusf/seaweedfs/weed/filesys/page_writer" "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "io" @@ -13,8 +14,7 @@ import ( type TempFileDirtyPages struct { f *File tf *os.File - writtenIntervals *WrittenContinuousIntervals - writeOnly bool + writtenIntervals *page_writer.WrittenContinuousIntervals writeWaitGroup sync.WaitGroup pageAddLock sync.Mutex chunkAddLock sync.Mutex @@ -23,12 +23,11 @@ type TempFileDirtyPages struct { replication string } -func newTempFileDirtyPages(file *File, writeOnly bool) *TempFileDirtyPages { +func newTempFileDirtyPages(file *File) *TempFileDirtyPages { tempFile := &TempFileDirtyPages{ f: file, - writeOnly: writeOnly, - writtenIntervals: &WrittenContinuousIntervals{}, + writtenIntervals: &page_writer.WrittenContinuousIntervals{}, } return tempFile @@ -47,11 +46,11 @@ func (pages *TempFileDirtyPages) AddPage(offset int64, data []byte) { return } pages.tf = tf - pages.writtenIntervals.tempFile = tf - pages.writtenIntervals.lastOffset = 0 + pages.writtenIntervals.TempFile = tf + pages.writtenIntervals.LastOffset = 0 } - writtenOffset := pages.writtenIntervals.lastOffset + writtenOffset := pages.writtenIntervals.LastOffset dataSize := int64(len(data)) // glog.V(4).Infof("%s AddPage %v at %d [%d,%d)", pages.f.fullpath(), pages.tf.Name(), writtenOffset, offset, offset+dataSize) @@ -60,7 +59,7 @@ func (pages *TempFileDirtyPages) AddPage(offset int64, data []byte) { pages.lastErr = err } else { pages.writtenIntervals.AddInterval(writtenOffset, len(data), offset) - pages.writtenIntervals.lastOffset += dataSize + pages.writtenIntervals.LastOffset += dataSize } // pages.writtenIntervals.debug() @@ -79,8 +78,8 @@ func (pages *TempFileDirtyPages) FlushData() error { defer pages.pageAddLock.Unlock() if pages.tf != nil { - pages.writtenIntervals.tempFile = nil - pages.writtenIntervals.lists = nil + pages.writtenIntervals.TempFile = nil + pages.writtenIntervals.Lists = nil pages.tf.Close() os.Remove(pages.tf.Name()) @@ -95,7 +94,7 @@ func (pages *TempFileDirtyPages) saveExistingPagesToStorage() { // glog.V(4).Infof("%v saveExistingPagesToStorage %d lists", pages.f.Name, len(pages.writtenIntervals.lists)) - for _, list := range pages.writtenIntervals.lists { + for _, list := range pages.writtenIntervals.Lists { listStopOffset := list.Offset() + list.Size() for uploadedOffset := int64(0); uploadedOffset < listStopOffset; uploadedOffset += pageSize { start, stop := max(list.Offset(), uploadedOffset), min(listStopOffset, uploadedOffset+pageSize) @@ -117,7 +116,7 @@ func (pages *TempFileDirtyPages) saveToStorage(reader io.Reader, offset int64, s defer pages.writeWaitGroup.Done() reader = io.LimitReader(reader, size) - chunk, collection, replication, err := pages.f.wfs.saveDataAsChunk(pages.f.fullpath(), pages.writeOnly)(reader, pages.f.Name, offset) + chunk, collection, replication, err := pages.f.wfs.saveDataAsChunk(pages.f.fullpath())(reader, pages.f.Name, offset) if err != nil { glog.V(0).Infof("%s saveToStorage [%d,%d): %v", pages.f.fullpath(), offset, offset+size, err) pages.lastErr = err @@ -145,13 +144,3 @@ func (pages *TempFileDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64) func (pages *TempFileDirtyPages) GetStorageOptions() (collection, replication string) { return pages.collection, pages.replication } - -func (pages *TempFileDirtyPages) SetWriteOnly(writeOnly bool) { - if pages.writeOnly { - pages.writeOnly = writeOnly - } -} - -func (pages *TempFileDirtyPages) GetWriteOnly() (writeOnly bool) { - return pages.writeOnly -} diff --git a/weed/filesys/file.go b/weed/filesys/file.go index f8fd7ad99..767841f9d 100644 --- a/weed/filesys/file.go +++ b/weed/filesys/file.go @@ -97,7 +97,7 @@ func (file *File) Open(ctx context.Context, req *fuse.OpenRequest, resp *fuse.Op glog.V(4).Infof("file %v open %+v", file.fullpath(), req) - handle := file.wfs.AcquireHandle(file, req.Uid, req.Gid, req.Flags&fuse.OpenWriteOnly > 0) + handle := file.wfs.AcquireHandle(file, req.Uid, req.Gid) resp.Handle = fuse.HandleID(handle.handle) diff --git a/weed/filesys/filehandle.go b/weed/filesys/filehandle.go index 34affddb9..232d28667 100644 --- a/weed/filesys/filehandle.go +++ b/weed/filesys/filehandle.go @@ -3,6 +3,7 @@ package filesys import ( "context" "fmt" + "github.com/chrislusf/seaweedfs/weed/filesys/page_writer" "io" "math" "net/http" @@ -20,7 +21,7 @@ import ( type FileHandle struct { // cache file has been written to - dirtyPages DirtyPages + dirtyPages page_writer.DirtyPages entryViewCache []filer.VisibleInterval reader io.ReaderAt contentType string @@ -36,11 +37,11 @@ type FileHandle struct { isDeleted bool } -func newFileHandle(file *File, uid, gid uint32, writeOnly bool) *FileHandle { +func newFileHandle(file *File, uid, gid uint32) *FileHandle { fh := &FileHandle{ f: file, // dirtyPages: newContinuousDirtyPages(file, writeOnly), - dirtyPages: newTempFileDirtyPages(file, writeOnly), + dirtyPages: newTempFileDirtyPages(file), Uid: uid, Gid: gid, } @@ -62,10 +63,11 @@ var _ = fs.HandleReleaser(&FileHandle{}) func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fuse.ReadResponse) error { - glog.V(4).Infof("%s read fh %d: [%d,%d) size %d resp.Data cap=%d", fh.f.fullpath(), fh.handle, req.Offset, req.Offset+int64(req.Size), req.Size, cap(resp.Data)) fh.Lock() defer fh.Unlock() + glog.V(4).Infof("%s read fh %d: [%d,%d) size %d resp.Data cap=%d", fh.f.fullpath(), fh.handle, req.Offset, req.Offset+int64(req.Size), req.Size, cap(resp.Data)) + if req.Size <= 0 { return nil } @@ -173,7 +175,7 @@ func (fh *FileHandle) Write(ctx context.Context, req *fuse.WriteRequest, resp *f // write the request to volume servers data := req.Data - if len(data) <= 512 { + if len(data) <= 512 && req.Offset == 0 { // fuse message cacheable size data = make([]byte, len(req.Data)) copy(data, req.Data) @@ -303,7 +305,7 @@ func (fh *FileHandle) doFlush(ctx context.Context, header fuse.Header) error { manifestChunks, nonManifestChunks := filer.SeparateManifestChunks(entry.Chunks) chunks, _ := filer.CompactFileChunks(fh.f.wfs.LookupFn(), nonManifestChunks) - chunks, manifestErr := filer.MaybeManifestize(fh.f.wfs.saveDataAsChunk(fh.f.fullpath(), fh.dirtyPages.GetWriteOnly()), chunks) + chunks, manifestErr := filer.MaybeManifestize(fh.f.wfs.saveDataAsChunk(fh.f.fullpath()), chunks) if manifestErr != nil { // not good, but should be ok glog.V(0).Infof("MaybeManifestize: %v", manifestErr) diff --git a/weed/filesys/dirty_page_interval.go b/weed/filesys/page_writer/dirty_page_interval.go index 304793340..6d73b8cd7 100644 --- a/weed/filesys/dirty_page_interval.go +++ b/weed/filesys/page_writer/dirty_page_interval.go @@ -1,4 +1,4 @@ -package filesys +package page_writer import ( "io" diff --git a/weed/filesys/dirty_page_interval_test.go b/weed/filesys/page_writer/dirty_page_interval_test.go index d02ad27fd..2a2a1df4d 100644 --- a/weed/filesys/dirty_page_interval_test.go +++ b/weed/filesys/page_writer/dirty_page_interval_test.go @@ -1,4 +1,4 @@ -package filesys +package page_writer import ( "bytes" diff --git a/weed/filesys/dirty_pages.go b/weed/filesys/page_writer/dirty_pages.go index 8505323ef..c18f847b7 100644 --- a/weed/filesys/dirty_pages.go +++ b/weed/filesys/page_writer/dirty_pages.go @@ -1,10 +1,8 @@ -package filesys +package page_writer type DirtyPages interface { AddPage(offset int64, data []byte) FlushData() error ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) GetStorageOptions() (collection, replication string) - SetWriteOnly(writeOnly bool) - GetWriteOnly() (writeOnly bool) } diff --git a/weed/filesys/dirty_pages_temp_interval.go b/weed/filesys/page_writer/dirty_pages_temp_interval.go index 42c4b5a3b..aeaf0ec6f 100644 --- a/weed/filesys/dirty_pages_temp_interval.go +++ b/weed/filesys/page_writer/dirty_pages_temp_interval.go @@ -1,4 +1,4 @@ -package filesys +package page_writer import ( "io" @@ -20,9 +20,9 @@ type WrittenIntervalLinkedList struct { } type WrittenContinuousIntervals struct { - tempFile *os.File - lastOffset int64 - lists []*WrittenIntervalLinkedList + TempFile *os.File + LastOffset int64 + Lists []*WrittenIntervalLinkedList } func (list *WrittenIntervalLinkedList) Offset() int64 { @@ -65,7 +65,7 @@ func (list *WrittenIntervalLinkedList) ReadData(buf []byte, start, stop int64) { } func (c *WrittenContinuousIntervals) TotalSize() (total int64) { - for _, list := range c.lists { + for _, list := range c.Lists { total += list.Size() } return @@ -98,7 +98,7 @@ func (list *WrittenIntervalLinkedList) subList(start, stop int64) *WrittenInterv func (c *WrittenContinuousIntervals) debug() { log.Printf("++") - for _, l := range c.lists { + for _, l := range c.Lists { log.Printf("++++") for t := l.Head; ; t = t.Next { log.Printf("[%d,%d) => [%d,%d) %d", t.DataOffset, t.DataOffset+t.Size, t.TempOffset, t.TempOffset+t.Size, t.Size) @@ -116,8 +116,8 @@ func (c *WrittenContinuousIntervals) AddInterval(tempOffset int64, dataSize int, interval := &WrittenIntervalNode{DataOffset: dataOffset, TempOffset: tempOffset, Size: int64(dataSize)} // append to the tail and return - if len(c.lists) == 1 { - lastSpan := c.lists[0] + if len(c.Lists) == 1 { + lastSpan := c.Lists[0] if lastSpan.Tail.DataOffset+lastSpan.Tail.Size == dataOffset { lastSpan.addNodeToTail(interval) return @@ -125,7 +125,7 @@ func (c *WrittenContinuousIntervals) AddInterval(tempOffset int64, dataSize int, } var newLists []*WrittenIntervalLinkedList - for _, list := range c.lists { + for _, list := range c.Lists { // if list is to the left of new interval, add to the new list if list.Tail.DataOffset+list.Tail.Size <= interval.DataOffset { newLists = append(newLists, list) @@ -147,18 +147,18 @@ func (c *WrittenContinuousIntervals) AddInterval(tempOffset int64, dataSize int, // skip anything that is fully overwritten by the new interval } - c.lists = newLists + c.Lists = newLists // add the new interval to the lists, connecting neighbor lists var prevList, nextList *WrittenIntervalLinkedList - for _, list := range c.lists { + for _, list := range c.Lists { if list.Head.DataOffset == interval.DataOffset+interval.Size { nextList = list break } } - for _, list := range c.lists { + for _, list := range c.Lists { if list.Head.DataOffset+list.Size() == dataOffset { list.addNodeToTail(interval) prevList = list @@ -176,8 +176,8 @@ func (c *WrittenContinuousIntervals) AddInterval(tempOffset int64, dataSize int, nextList.addNodeToHead(interval) } if prevList == nil && nextList == nil { - c.lists = append(c.lists, &WrittenIntervalLinkedList{ - tempFile: c.tempFile, + c.Lists = append(c.Lists, &WrittenIntervalLinkedList{ + tempFile: c.TempFile, Head: interval, Tail: interval, }) @@ -189,7 +189,7 @@ func (c *WrittenContinuousIntervals) AddInterval(tempOffset int64, dataSize int, func (c *WrittenContinuousIntervals) RemoveLargestIntervalLinkedList() *WrittenIntervalLinkedList { var maxSize int64 maxIndex := -1 - for k, list := range c.lists { + for k, list := range c.Lists { if maxSize <= list.Size() { maxSize = list.Size() maxIndex = k @@ -199,16 +199,16 @@ func (c *WrittenContinuousIntervals) RemoveLargestIntervalLinkedList() *WrittenI return nil } - t := c.lists[maxIndex] - t.tempFile = c.tempFile - c.lists = append(c.lists[0:maxIndex], c.lists[maxIndex+1:]...) + t := c.Lists[maxIndex] + t.tempFile = c.TempFile + c.Lists = append(c.Lists[0:maxIndex], c.Lists[maxIndex+1:]...) return t } func (c *WrittenContinuousIntervals) removeList(target *WrittenIntervalLinkedList) { index := -1 - for k, list := range c.lists { + for k, list := range c.Lists { if list.Offset() == target.Offset() { index = k } @@ -217,12 +217,12 @@ func (c *WrittenContinuousIntervals) removeList(target *WrittenIntervalLinkedLis return } - c.lists = append(c.lists[0:index], c.lists[index+1:]...) + c.Lists = append(c.Lists[0:index], c.Lists[index+1:]...) } func (c *WrittenContinuousIntervals) ReadDataAt(data []byte, startOffset int64) (maxStop int64) { - for _, list := range c.lists { + for _, list := range c.Lists { start := max(startOffset, list.Offset()) stop := min(startOffset+int64(len(data)), list.Offset()+list.Size()) if start < stop { @@ -287,3 +287,16 @@ func (f *FileSectionReader) Read(p []byte) (n int, err error) { } return } + +func max(x, y int64) int64 { + if x > y { + return x + } + return y +} +func min(x, y int64) int64 { + if x < y { + return x + } + return y +} diff --git a/weed/filesys/page_writer/writer_pattern.go b/weed/filesys/page_writer/writer_pattern.go new file mode 100644 index 000000000..c7641c37f --- /dev/null +++ b/weed/filesys/page_writer/writer_pattern.go @@ -0,0 +1,31 @@ +package page_writer + +type WriterPattern struct { + isStreaming bool + lastWriteOffset int64 +} + +// For streaming write: only cache the first chunk +// For random write: fall back to temp file approach + +func NewWriterPattern() *WriterPattern { + return &WriterPattern{ + isStreaming: true, + lastWriteOffset: 0, + } +} + +func (rp *WriterPattern) MonitorWriteAt(offset int64, size int) { + if rp.lastWriteOffset > offset { + rp.isStreaming = false + } + rp.lastWriteOffset = offset +} + +func (rp *WriterPattern) IsStreamingMode() bool { + return rp.isStreaming +} + +func (rp *WriterPattern) IsRandomMode() bool { + return !rp.isStreaming +} diff --git a/weed/filesys/wfs.go b/weed/filesys/wfs.go index 92f6bae38..aa4f9dacd 100644 --- a/weed/filesys/wfs.go +++ b/weed/filesys/wfs.go @@ -148,7 +148,7 @@ func (wfs *WFS) Root() (fs.Node, error) { return wfs.root, nil } -func (wfs *WFS) AcquireHandle(file *File, uid, gid uint32, writeOnly bool) (fileHandle *FileHandle) { +func (wfs *WFS) AcquireHandle(file *File, uid, gid uint32) (fileHandle *FileHandle) { fullpath := file.fullpath() glog.V(4).Infof("AcquireHandle %s uid=%d gid=%d", fullpath, uid, gid) @@ -160,7 +160,6 @@ func (wfs *WFS) AcquireHandle(file *File, uid, gid uint32, writeOnly bool) (file if found && existingHandle != nil && existingHandle.f.isOpen > 0 { existingHandle.f.isOpen++ wfs.handlesLock.Unlock() - existingHandle.dirtyPages.SetWriteOnly(writeOnly) glog.V(4).Infof("Reuse AcquiredHandle %s open %d", fullpath, existingHandle.f.isOpen) return existingHandle } @@ -168,7 +167,7 @@ func (wfs *WFS) AcquireHandle(file *File, uid, gid uint32, writeOnly bool) (file entry, _ := file.maybeLoadEntry(context.Background()) file.entry = entry - fileHandle = newFileHandle(file, uid, gid, writeOnly) + fileHandle = newFileHandle(file, uid, gid) wfs.handlesLock.Lock() file.isOpen++ diff --git a/weed/filesys/wfs_write.go b/weed/filesys/wfs_write.go index 3d08cb5e2..61a463e88 100644 --- a/weed/filesys/wfs_write.go +++ b/weed/filesys/wfs_write.go @@ -13,7 +13,7 @@ import ( "github.com/chrislusf/seaweedfs/weed/util" ) -func (wfs *WFS) saveDataAsChunk(fullPath util.FullPath, writeOnly bool) filer.SaveDataAsChunkFunctionType { +func (wfs *WFS) saveDataAsChunk(fullPath util.FullPath) filer.SaveDataAsChunkFunctionType { return func(reader io.Reader, filename string, offset int64) (chunk *filer_pb.FileChunk, collection, replication string, err error) { var fileId, host string @@ -74,7 +74,7 @@ func (wfs *WFS) saveDataAsChunk(fullPath util.FullPath, writeOnly bool) filer.Sa return nil, "", "", fmt.Errorf("upload result: %v", uploadResult.Error) } - if !writeOnly { + if offset == 0 { wfs.chunkCache.SetChunk(fileId, data) } diff --git a/weed/messaging/broker/broker_server.go b/weed/messaging/broker/broker_server.go index fd41dd441..193c1c689 100644 --- a/weed/messaging/broker/broker_server.go +++ b/weed/messaging/broker/broker_server.go @@ -2,6 +2,7 @@ package broker import ( "context" + "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb" "time" "google.golang.org/grpc" @@ -22,6 +23,7 @@ type MessageBrokerOption struct { } type MessageBroker struct { + messaging_pb.UnimplementedSeaweedMessagingServer option *MessageBrokerOption grpcDialOption grpc.DialOption topicManager *TopicManager diff --git a/weed/pb/Makefile b/weed/pb/Makefile index 72c738135..dc27172b6 100644 --- a/weed/pb/Makefile +++ b/weed/pb/Makefile @@ -3,11 +3,11 @@ all: gen .PHONY : gen gen: - protoc master.proto --go_out=plugins=grpc:./master_pb --go_opt=paths=source_relative - protoc volume_server.proto --go_out=plugins=grpc:./volume_server_pb --go_opt=paths=source_relative - protoc filer.proto --go_out=plugins=grpc:./filer_pb --go_opt=paths=source_relative - protoc remote.proto --go_out=plugins=grpc:./remote_pb --go_opt=paths=source_relative - protoc iam.proto --go_out=plugins=grpc:./iam_pb --go_opt=paths=source_relative - protoc messaging.proto --go_out=plugins=grpc:./messaging_pb --go_opt=paths=source_relative + protoc master.proto --go_out=./master_pb --go-grpc_out=./master_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc volume_server.proto --go_out=./volume_server_pb --go-grpc_out=./volume_server_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc filer.proto --go_out=./filer_pb --go-grpc_out=./filer_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc remote.proto --go_out=./remote_pb --go-grpc_out=./remote_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc iam.proto --go_out=./iam_pb --go-grpc_out=./iam_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc messaging.proto --go_out=./messaging_pb --go-grpc_out=./messaging_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative # protoc filer.proto --java_out=../../other/java/client/src/main/java cp filer.proto ../../other/java/client/src/main/proto diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go index ba8a36b8b..393a4d037 100644 --- a/weed/pb/filer_pb/filer.pb.go +++ b/weed/pb/filer_pb/filer.pb.go @@ -1,17 +1,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.25.0 -// protoc v3.12.3 +// protoc-gen-go v1.26.0 +// protoc v3.17.3 // source: filer.proto package filer_pb import ( - context "context" - proto "github.com/golang/protobuf/proto" - grpc "google.golang.org/grpc" - codes "google.golang.org/grpc/codes" - status "google.golang.org/grpc/status" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -25,10 +20,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// This is a compile-time assertion that a sufficiently up-to-date version -// of the legacy proto package is being used. -const _ = proto.ProtoPackageIsVersion4 - type LookupDirectoryEntryRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5011,944 +5002,3 @@ func file_filer_proto_init() { file_filer_proto_goTypes = nil file_filer_proto_depIdxs = nil } - -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConnInterface - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion6 - -// SeaweedFilerClient is the client API for SeaweedFiler service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type SeaweedFilerClient interface { - LookupDirectoryEntry(ctx context.Context, in *LookupDirectoryEntryRequest, opts ...grpc.CallOption) (*LookupDirectoryEntryResponse, error) - ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (SeaweedFiler_ListEntriesClient, error) - CreateEntry(ctx context.Context, in *CreateEntryRequest, opts ...grpc.CallOption) (*CreateEntryResponse, error) - UpdateEntry(ctx context.Context, in *UpdateEntryRequest, opts ...grpc.CallOption) (*UpdateEntryResponse, error) - AppendToEntry(ctx context.Context, in *AppendToEntryRequest, opts ...grpc.CallOption) (*AppendToEntryResponse, error) - DeleteEntry(ctx context.Context, in *DeleteEntryRequest, opts ...grpc.CallOption) (*DeleteEntryResponse, error) - AtomicRenameEntry(ctx context.Context, in *AtomicRenameEntryRequest, opts ...grpc.CallOption) (*AtomicRenameEntryResponse, error) - StreamRenameEntry(ctx context.Context, in *StreamRenameEntryRequest, opts ...grpc.CallOption) (SeaweedFiler_StreamRenameEntryClient, error) - AssignVolume(ctx context.Context, in *AssignVolumeRequest, opts ...grpc.CallOption) (*AssignVolumeResponse, error) - LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) - CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) - DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) - Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) - GetFilerConfiguration(ctx context.Context, in *GetFilerConfigurationRequest, opts ...grpc.CallOption) (*GetFilerConfigurationResponse, error) - SubscribeMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeMetadataClient, error) - SubscribeLocalMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeLocalMetadataClient, error) - KeepConnected(ctx context.Context, opts ...grpc.CallOption) (SeaweedFiler_KeepConnectedClient, error) - LocateBroker(ctx context.Context, in *LocateBrokerRequest, opts ...grpc.CallOption) (*LocateBrokerResponse, error) - KvGet(ctx context.Context, in *KvGetRequest, opts ...grpc.CallOption) (*KvGetResponse, error) - KvPut(ctx context.Context, in *KvPutRequest, opts ...grpc.CallOption) (*KvPutResponse, error) - CacheRemoteObjectToLocalCluster(ctx context.Context, in *CacheRemoteObjectToLocalClusterRequest, opts ...grpc.CallOption) (*CacheRemoteObjectToLocalClusterResponse, error) -} - -type seaweedFilerClient struct { - cc grpc.ClientConnInterface -} - -func NewSeaweedFilerClient(cc grpc.ClientConnInterface) SeaweedFilerClient { - return &seaweedFilerClient{cc} -} - -func (c *seaweedFilerClient) LookupDirectoryEntry(ctx context.Context, in *LookupDirectoryEntryRequest, opts ...grpc.CallOption) (*LookupDirectoryEntryResponse, error) { - out := new(LookupDirectoryEntryResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/LookupDirectoryEntry", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (SeaweedFiler_ListEntriesClient, error) { - stream, err := c.cc.NewStream(ctx, &_SeaweedFiler_serviceDesc.Streams[0], "/filer_pb.SeaweedFiler/ListEntries", opts...) - if err != nil { - return nil, err - } - x := &seaweedFilerListEntriesClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type SeaweedFiler_ListEntriesClient interface { - Recv() (*ListEntriesResponse, error) - grpc.ClientStream -} - -type seaweedFilerListEntriesClient struct { - grpc.ClientStream -} - -func (x *seaweedFilerListEntriesClient) Recv() (*ListEntriesResponse, error) { - m := new(ListEntriesResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedFilerClient) CreateEntry(ctx context.Context, in *CreateEntryRequest, opts ...grpc.CallOption) (*CreateEntryResponse, error) { - out := new(CreateEntryResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/CreateEntry", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) UpdateEntry(ctx context.Context, in *UpdateEntryRequest, opts ...grpc.CallOption) (*UpdateEntryResponse, error) { - out := new(UpdateEntryResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/UpdateEntry", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) AppendToEntry(ctx context.Context, in *AppendToEntryRequest, opts ...grpc.CallOption) (*AppendToEntryResponse, error) { - out := new(AppendToEntryResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/AppendToEntry", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) DeleteEntry(ctx context.Context, in *DeleteEntryRequest, opts ...grpc.CallOption) (*DeleteEntryResponse, error) { - out := new(DeleteEntryResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/DeleteEntry", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) AtomicRenameEntry(ctx context.Context, in *AtomicRenameEntryRequest, opts ...grpc.CallOption) (*AtomicRenameEntryResponse, error) { - out := new(AtomicRenameEntryResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/AtomicRenameEntry", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) StreamRenameEntry(ctx context.Context, in *StreamRenameEntryRequest, opts ...grpc.CallOption) (SeaweedFiler_StreamRenameEntryClient, error) { - stream, err := c.cc.NewStream(ctx, &_SeaweedFiler_serviceDesc.Streams[1], "/filer_pb.SeaweedFiler/StreamRenameEntry", opts...) - if err != nil { - return nil, err - } - x := &seaweedFilerStreamRenameEntryClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type SeaweedFiler_StreamRenameEntryClient interface { - Recv() (*StreamRenameEntryResponse, error) - grpc.ClientStream -} - -type seaweedFilerStreamRenameEntryClient struct { - grpc.ClientStream -} - -func (x *seaweedFilerStreamRenameEntryClient) Recv() (*StreamRenameEntryResponse, error) { - m := new(StreamRenameEntryResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedFilerClient) AssignVolume(ctx context.Context, in *AssignVolumeRequest, opts ...grpc.CallOption) (*AssignVolumeResponse, error) { - out := new(AssignVolumeResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/AssignVolume", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) { - out := new(LookupVolumeResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/LookupVolume", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) { - out := new(CollectionListResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/CollectionList", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) { - out := new(DeleteCollectionResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/DeleteCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) { - out := new(StatisticsResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/Statistics", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) GetFilerConfiguration(ctx context.Context, in *GetFilerConfigurationRequest, opts ...grpc.CallOption) (*GetFilerConfigurationResponse, error) { - out := new(GetFilerConfigurationResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/GetFilerConfiguration", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) SubscribeMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeMetadataClient, error) { - stream, err := c.cc.NewStream(ctx, &_SeaweedFiler_serviceDesc.Streams[2], "/filer_pb.SeaweedFiler/SubscribeMetadata", opts...) - if err != nil { - return nil, err - } - x := &seaweedFilerSubscribeMetadataClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type SeaweedFiler_SubscribeMetadataClient interface { - Recv() (*SubscribeMetadataResponse, error) - grpc.ClientStream -} - -type seaweedFilerSubscribeMetadataClient struct { - grpc.ClientStream -} - -func (x *seaweedFilerSubscribeMetadataClient) Recv() (*SubscribeMetadataResponse, error) { - m := new(SubscribeMetadataResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedFilerClient) SubscribeLocalMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeLocalMetadataClient, error) { - stream, err := c.cc.NewStream(ctx, &_SeaweedFiler_serviceDesc.Streams[3], "/filer_pb.SeaweedFiler/SubscribeLocalMetadata", opts...) - if err != nil { - return nil, err - } - x := &seaweedFilerSubscribeLocalMetadataClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type SeaweedFiler_SubscribeLocalMetadataClient interface { - Recv() (*SubscribeMetadataResponse, error) - grpc.ClientStream -} - -type seaweedFilerSubscribeLocalMetadataClient struct { - grpc.ClientStream -} - -func (x *seaweedFilerSubscribeLocalMetadataClient) Recv() (*SubscribeMetadataResponse, error) { - m := new(SubscribeMetadataResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedFilerClient) KeepConnected(ctx context.Context, opts ...grpc.CallOption) (SeaweedFiler_KeepConnectedClient, error) { - stream, err := c.cc.NewStream(ctx, &_SeaweedFiler_serviceDesc.Streams[4], "/filer_pb.SeaweedFiler/KeepConnected", opts...) - if err != nil { - return nil, err - } - x := &seaweedFilerKeepConnectedClient{stream} - return x, nil -} - -type SeaweedFiler_KeepConnectedClient interface { - Send(*KeepConnectedRequest) error - Recv() (*KeepConnectedResponse, error) - grpc.ClientStream -} - -type seaweedFilerKeepConnectedClient struct { - grpc.ClientStream -} - -func (x *seaweedFilerKeepConnectedClient) Send(m *KeepConnectedRequest) error { - return x.ClientStream.SendMsg(m) -} - -func (x *seaweedFilerKeepConnectedClient) Recv() (*KeepConnectedResponse, error) { - m := new(KeepConnectedResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedFilerClient) LocateBroker(ctx context.Context, in *LocateBrokerRequest, opts ...grpc.CallOption) (*LocateBrokerResponse, error) { - out := new(LocateBrokerResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/LocateBroker", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) KvGet(ctx context.Context, in *KvGetRequest, opts ...grpc.CallOption) (*KvGetResponse, error) { - out := new(KvGetResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/KvGet", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) KvPut(ctx context.Context, in *KvPutRequest, opts ...grpc.CallOption) (*KvPutResponse, error) { - out := new(KvPutResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/KvPut", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedFilerClient) CacheRemoteObjectToLocalCluster(ctx context.Context, in *CacheRemoteObjectToLocalClusterRequest, opts ...grpc.CallOption) (*CacheRemoteObjectToLocalClusterResponse, error) { - out := new(CacheRemoteObjectToLocalClusterResponse) - err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/CacheRemoteObjectToLocalCluster", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -// SeaweedFilerServer is the server API for SeaweedFiler service. -type SeaweedFilerServer interface { - LookupDirectoryEntry(context.Context, *LookupDirectoryEntryRequest) (*LookupDirectoryEntryResponse, error) - ListEntries(*ListEntriesRequest, SeaweedFiler_ListEntriesServer) error - CreateEntry(context.Context, *CreateEntryRequest) (*CreateEntryResponse, error) - UpdateEntry(context.Context, *UpdateEntryRequest) (*UpdateEntryResponse, error) - AppendToEntry(context.Context, *AppendToEntryRequest) (*AppendToEntryResponse, error) - DeleteEntry(context.Context, *DeleteEntryRequest) (*DeleteEntryResponse, error) - AtomicRenameEntry(context.Context, *AtomicRenameEntryRequest) (*AtomicRenameEntryResponse, error) - StreamRenameEntry(*StreamRenameEntryRequest, SeaweedFiler_StreamRenameEntryServer) error - AssignVolume(context.Context, *AssignVolumeRequest) (*AssignVolumeResponse, error) - LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) - CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) - DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) - Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) - GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error) - SubscribeMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeMetadataServer) error - SubscribeLocalMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeLocalMetadataServer) error - KeepConnected(SeaweedFiler_KeepConnectedServer) error - LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error) - KvGet(context.Context, *KvGetRequest) (*KvGetResponse, error) - KvPut(context.Context, *KvPutRequest) (*KvPutResponse, error) - CacheRemoteObjectToLocalCluster(context.Context, *CacheRemoteObjectToLocalClusterRequest) (*CacheRemoteObjectToLocalClusterResponse, error) -} - -// UnimplementedSeaweedFilerServer can be embedded to have forward compatible implementations. -type UnimplementedSeaweedFilerServer struct { -} - -func (*UnimplementedSeaweedFilerServer) LookupDirectoryEntry(context.Context, *LookupDirectoryEntryRequest) (*LookupDirectoryEntryResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method LookupDirectoryEntry not implemented") -} -func (*UnimplementedSeaweedFilerServer) ListEntries(*ListEntriesRequest, SeaweedFiler_ListEntriesServer) error { - return status.Errorf(codes.Unimplemented, "method ListEntries not implemented") -} -func (*UnimplementedSeaweedFilerServer) CreateEntry(context.Context, *CreateEntryRequest) (*CreateEntryResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CreateEntry not implemented") -} -func (*UnimplementedSeaweedFilerServer) UpdateEntry(context.Context, *UpdateEntryRequest) (*UpdateEntryResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method UpdateEntry not implemented") -} -func (*UnimplementedSeaweedFilerServer) AppendToEntry(context.Context, *AppendToEntryRequest) (*AppendToEntryResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method AppendToEntry not implemented") -} -func (*UnimplementedSeaweedFilerServer) DeleteEntry(context.Context, *DeleteEntryRequest) (*DeleteEntryResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DeleteEntry not implemented") -} -func (*UnimplementedSeaweedFilerServer) AtomicRenameEntry(context.Context, *AtomicRenameEntryRequest) (*AtomicRenameEntryResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method AtomicRenameEntry not implemented") -} -func (*UnimplementedSeaweedFilerServer) StreamRenameEntry(*StreamRenameEntryRequest, SeaweedFiler_StreamRenameEntryServer) error { - return status.Errorf(codes.Unimplemented, "method StreamRenameEntry not implemented") -} -func (*UnimplementedSeaweedFilerServer) AssignVolume(context.Context, *AssignVolumeRequest) (*AssignVolumeResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method AssignVolume not implemented") -} -func (*UnimplementedSeaweedFilerServer) LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method LookupVolume not implemented") -} -func (*UnimplementedSeaweedFilerServer) CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CollectionList not implemented") -} -func (*UnimplementedSeaweedFilerServer) DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DeleteCollection not implemented") -} -func (*UnimplementedSeaweedFilerServer) Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method Statistics not implemented") -} -func (*UnimplementedSeaweedFilerServer) GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method GetFilerConfiguration not implemented") -} -func (*UnimplementedSeaweedFilerServer) SubscribeMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeMetadataServer) error { - return status.Errorf(codes.Unimplemented, "method SubscribeMetadata not implemented") -} -func (*UnimplementedSeaweedFilerServer) SubscribeLocalMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeLocalMetadataServer) error { - return status.Errorf(codes.Unimplemented, "method SubscribeLocalMetadata not implemented") -} -func (*UnimplementedSeaweedFilerServer) KeepConnected(SeaweedFiler_KeepConnectedServer) error { - return status.Errorf(codes.Unimplemented, "method KeepConnected not implemented") -} -func (*UnimplementedSeaweedFilerServer) LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method LocateBroker not implemented") -} -func (*UnimplementedSeaweedFilerServer) KvGet(context.Context, *KvGetRequest) (*KvGetResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method KvGet not implemented") -} -func (*UnimplementedSeaweedFilerServer) KvPut(context.Context, *KvPutRequest) (*KvPutResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method KvPut not implemented") -} -func (*UnimplementedSeaweedFilerServer) CacheRemoteObjectToLocalCluster(context.Context, *CacheRemoteObjectToLocalClusterRequest) (*CacheRemoteObjectToLocalClusterResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CacheRemoteObjectToLocalCluster not implemented") -} - -func RegisterSeaweedFilerServer(s *grpc.Server, srv SeaweedFilerServer) { - s.RegisterService(&_SeaweedFiler_serviceDesc, srv) -} - -func _SeaweedFiler_LookupDirectoryEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LookupDirectoryEntryRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).LookupDirectoryEntry(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/LookupDirectoryEntry", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).LookupDirectoryEntry(ctx, req.(*LookupDirectoryEntryRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_ListEntries_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(ListEntriesRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(SeaweedFilerServer).ListEntries(m, &seaweedFilerListEntriesServer{stream}) -} - -type SeaweedFiler_ListEntriesServer interface { - Send(*ListEntriesResponse) error - grpc.ServerStream -} - -type seaweedFilerListEntriesServer struct { - grpc.ServerStream -} - -func (x *seaweedFilerListEntriesServer) Send(m *ListEntriesResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _SeaweedFiler_CreateEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CreateEntryRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).CreateEntry(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/CreateEntry", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).CreateEntry(ctx, req.(*CreateEntryRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_UpdateEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(UpdateEntryRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).UpdateEntry(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/UpdateEntry", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).UpdateEntry(ctx, req.(*UpdateEntryRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_AppendToEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AppendToEntryRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).AppendToEntry(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/AppendToEntry", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).AppendToEntry(ctx, req.(*AppendToEntryRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_DeleteEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DeleteEntryRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).DeleteEntry(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/DeleteEntry", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).DeleteEntry(ctx, req.(*DeleteEntryRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_AtomicRenameEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AtomicRenameEntryRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).AtomicRenameEntry(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/AtomicRenameEntry", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).AtomicRenameEntry(ctx, req.(*AtomicRenameEntryRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_StreamRenameEntry_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(StreamRenameEntryRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(SeaweedFilerServer).StreamRenameEntry(m, &seaweedFilerStreamRenameEntryServer{stream}) -} - -type SeaweedFiler_StreamRenameEntryServer interface { - Send(*StreamRenameEntryResponse) error - grpc.ServerStream -} - -type seaweedFilerStreamRenameEntryServer struct { - grpc.ServerStream -} - -func (x *seaweedFilerStreamRenameEntryServer) Send(m *StreamRenameEntryResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _SeaweedFiler_AssignVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AssignVolumeRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).AssignVolume(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/AssignVolume", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).AssignVolume(ctx, req.(*AssignVolumeRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_LookupVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LookupVolumeRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).LookupVolume(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/LookupVolume", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).LookupVolume(ctx, req.(*LookupVolumeRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_CollectionList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CollectionListRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).CollectionList(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/CollectionList", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).CollectionList(ctx, req.(*CollectionListRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_DeleteCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DeleteCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).DeleteCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/DeleteCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).DeleteCollection(ctx, req.(*DeleteCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_Statistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(StatisticsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).Statistics(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/Statistics", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).Statistics(ctx, req.(*StatisticsRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_GetFilerConfiguration_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(GetFilerConfigurationRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).GetFilerConfiguration(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/GetFilerConfiguration", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).GetFilerConfiguration(ctx, req.(*GetFilerConfigurationRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_SubscribeMetadata_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(SubscribeMetadataRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(SeaweedFilerServer).SubscribeMetadata(m, &seaweedFilerSubscribeMetadataServer{stream}) -} - -type SeaweedFiler_SubscribeMetadataServer interface { - Send(*SubscribeMetadataResponse) error - grpc.ServerStream -} - -type seaweedFilerSubscribeMetadataServer struct { - grpc.ServerStream -} - -func (x *seaweedFilerSubscribeMetadataServer) Send(m *SubscribeMetadataResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _SeaweedFiler_SubscribeLocalMetadata_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(SubscribeMetadataRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(SeaweedFilerServer).SubscribeLocalMetadata(m, &seaweedFilerSubscribeLocalMetadataServer{stream}) -} - -type SeaweedFiler_SubscribeLocalMetadataServer interface { - Send(*SubscribeMetadataResponse) error - grpc.ServerStream -} - -type seaweedFilerSubscribeLocalMetadataServer struct { - grpc.ServerStream -} - -func (x *seaweedFilerSubscribeLocalMetadataServer) Send(m *SubscribeMetadataResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _SeaweedFiler_KeepConnected_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(SeaweedFilerServer).KeepConnected(&seaweedFilerKeepConnectedServer{stream}) -} - -type SeaweedFiler_KeepConnectedServer interface { - Send(*KeepConnectedResponse) error - Recv() (*KeepConnectedRequest, error) - grpc.ServerStream -} - -type seaweedFilerKeepConnectedServer struct { - grpc.ServerStream -} - -func (x *seaweedFilerKeepConnectedServer) Send(m *KeepConnectedResponse) error { - return x.ServerStream.SendMsg(m) -} - -func (x *seaweedFilerKeepConnectedServer) Recv() (*KeepConnectedRequest, error) { - m := new(KeepConnectedRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func _SeaweedFiler_LocateBroker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LocateBrokerRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).LocateBroker(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/LocateBroker", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).LocateBroker(ctx, req.(*LocateBrokerRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_KvGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(KvGetRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).KvGet(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/KvGet", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).KvGet(ctx, req.(*KvGetRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_KvPut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(KvPutRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).KvPut(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/KvPut", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).KvPut(ctx, req.(*KvPutRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedFiler_CacheRemoteObjectToLocalCluster_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CacheRemoteObjectToLocalClusterRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedFilerServer).CacheRemoteObjectToLocalCluster(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/filer_pb.SeaweedFiler/CacheRemoteObjectToLocalCluster", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedFilerServer).CacheRemoteObjectToLocalCluster(ctx, req.(*CacheRemoteObjectToLocalClusterRequest)) - } - return interceptor(ctx, in, info, handler) -} - -var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{ - ServiceName: "filer_pb.SeaweedFiler", - HandlerType: (*SeaweedFilerServer)(nil), - Methods: []grpc.MethodDesc{ - { - MethodName: "LookupDirectoryEntry", - Handler: _SeaweedFiler_LookupDirectoryEntry_Handler, - }, - { - MethodName: "CreateEntry", - Handler: _SeaweedFiler_CreateEntry_Handler, - }, - { - MethodName: "UpdateEntry", - Handler: _SeaweedFiler_UpdateEntry_Handler, - }, - { - MethodName: "AppendToEntry", - Handler: _SeaweedFiler_AppendToEntry_Handler, - }, - { - MethodName: "DeleteEntry", - Handler: _SeaweedFiler_DeleteEntry_Handler, - }, - { - MethodName: "AtomicRenameEntry", - Handler: _SeaweedFiler_AtomicRenameEntry_Handler, - }, - { - MethodName: "AssignVolume", - Handler: _SeaweedFiler_AssignVolume_Handler, - }, - { - MethodName: "LookupVolume", - Handler: _SeaweedFiler_LookupVolume_Handler, - }, - { - MethodName: "CollectionList", - Handler: _SeaweedFiler_CollectionList_Handler, - }, - { - MethodName: "DeleteCollection", - Handler: _SeaweedFiler_DeleteCollection_Handler, - }, - { - MethodName: "Statistics", - Handler: _SeaweedFiler_Statistics_Handler, - }, - { - MethodName: "GetFilerConfiguration", - Handler: _SeaweedFiler_GetFilerConfiguration_Handler, - }, - { - MethodName: "LocateBroker", - Handler: _SeaweedFiler_LocateBroker_Handler, - }, - { - MethodName: "KvGet", - Handler: _SeaweedFiler_KvGet_Handler, - }, - { - MethodName: "KvPut", - Handler: _SeaweedFiler_KvPut_Handler, - }, - { - MethodName: "CacheRemoteObjectToLocalCluster", - Handler: _SeaweedFiler_CacheRemoteObjectToLocalCluster_Handler, - }, - }, - Streams: []grpc.StreamDesc{ - { - StreamName: "ListEntries", - Handler: _SeaweedFiler_ListEntries_Handler, - ServerStreams: true, - }, - { - StreamName: "StreamRenameEntry", - Handler: _SeaweedFiler_StreamRenameEntry_Handler, - ServerStreams: true, - }, - { - StreamName: "SubscribeMetadata", - Handler: _SeaweedFiler_SubscribeMetadata_Handler, - ServerStreams: true, - }, - { - StreamName: "SubscribeLocalMetadata", - Handler: _SeaweedFiler_SubscribeLocalMetadata_Handler, - ServerStreams: true, - }, - { - StreamName: "KeepConnected", - Handler: _SeaweedFiler_KeepConnected_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, - Metadata: "filer.proto", -} diff --git a/weed/pb/filer_pb/filer_grpc.pb.go b/weed/pb/filer_pb/filer_grpc.pb.go new file mode 100644 index 000000000..02302d317 --- /dev/null +++ b/weed/pb/filer_pb/filer_grpc.pb.go @@ -0,0 +1,962 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. + +package filer_pb + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// SeaweedFilerClient is the client API for SeaweedFiler service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type SeaweedFilerClient interface { + LookupDirectoryEntry(ctx context.Context, in *LookupDirectoryEntryRequest, opts ...grpc.CallOption) (*LookupDirectoryEntryResponse, error) + ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (SeaweedFiler_ListEntriesClient, error) + CreateEntry(ctx context.Context, in *CreateEntryRequest, opts ...grpc.CallOption) (*CreateEntryResponse, error) + UpdateEntry(ctx context.Context, in *UpdateEntryRequest, opts ...grpc.CallOption) (*UpdateEntryResponse, error) + AppendToEntry(ctx context.Context, in *AppendToEntryRequest, opts ...grpc.CallOption) (*AppendToEntryResponse, error) + DeleteEntry(ctx context.Context, in *DeleteEntryRequest, opts ...grpc.CallOption) (*DeleteEntryResponse, error) + AtomicRenameEntry(ctx context.Context, in *AtomicRenameEntryRequest, opts ...grpc.CallOption) (*AtomicRenameEntryResponse, error) + StreamRenameEntry(ctx context.Context, in *StreamRenameEntryRequest, opts ...grpc.CallOption) (SeaweedFiler_StreamRenameEntryClient, error) + AssignVolume(ctx context.Context, in *AssignVolumeRequest, opts ...grpc.CallOption) (*AssignVolumeResponse, error) + LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) + CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) + DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) + Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) + GetFilerConfiguration(ctx context.Context, in *GetFilerConfigurationRequest, opts ...grpc.CallOption) (*GetFilerConfigurationResponse, error) + SubscribeMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeMetadataClient, error) + SubscribeLocalMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeLocalMetadataClient, error) + KeepConnected(ctx context.Context, opts ...grpc.CallOption) (SeaweedFiler_KeepConnectedClient, error) + LocateBroker(ctx context.Context, in *LocateBrokerRequest, opts ...grpc.CallOption) (*LocateBrokerResponse, error) + KvGet(ctx context.Context, in *KvGetRequest, opts ...grpc.CallOption) (*KvGetResponse, error) + KvPut(ctx context.Context, in *KvPutRequest, opts ...grpc.CallOption) (*KvPutResponse, error) + CacheRemoteObjectToLocalCluster(ctx context.Context, in *CacheRemoteObjectToLocalClusterRequest, opts ...grpc.CallOption) (*CacheRemoteObjectToLocalClusterResponse, error) +} + +type seaweedFilerClient struct { + cc grpc.ClientConnInterface +} + +func NewSeaweedFilerClient(cc grpc.ClientConnInterface) SeaweedFilerClient { + return &seaweedFilerClient{cc} +} + +func (c *seaweedFilerClient) LookupDirectoryEntry(ctx context.Context, in *LookupDirectoryEntryRequest, opts ...grpc.CallOption) (*LookupDirectoryEntryResponse, error) { + out := new(LookupDirectoryEntryResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/LookupDirectoryEntry", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (SeaweedFiler_ListEntriesClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedFiler_ServiceDesc.Streams[0], "/filer_pb.SeaweedFiler/ListEntries", opts...) + if err != nil { + return nil, err + } + x := &seaweedFilerListEntriesClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type SeaweedFiler_ListEntriesClient interface { + Recv() (*ListEntriesResponse, error) + grpc.ClientStream +} + +type seaweedFilerListEntriesClient struct { + grpc.ClientStream +} + +func (x *seaweedFilerListEntriesClient) Recv() (*ListEntriesResponse, error) { + m := new(ListEntriesResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedFilerClient) CreateEntry(ctx context.Context, in *CreateEntryRequest, opts ...grpc.CallOption) (*CreateEntryResponse, error) { + out := new(CreateEntryResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/CreateEntry", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) UpdateEntry(ctx context.Context, in *UpdateEntryRequest, opts ...grpc.CallOption) (*UpdateEntryResponse, error) { + out := new(UpdateEntryResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/UpdateEntry", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) AppendToEntry(ctx context.Context, in *AppendToEntryRequest, opts ...grpc.CallOption) (*AppendToEntryResponse, error) { + out := new(AppendToEntryResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/AppendToEntry", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) DeleteEntry(ctx context.Context, in *DeleteEntryRequest, opts ...grpc.CallOption) (*DeleteEntryResponse, error) { + out := new(DeleteEntryResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/DeleteEntry", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) AtomicRenameEntry(ctx context.Context, in *AtomicRenameEntryRequest, opts ...grpc.CallOption) (*AtomicRenameEntryResponse, error) { + out := new(AtomicRenameEntryResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/AtomicRenameEntry", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) StreamRenameEntry(ctx context.Context, in *StreamRenameEntryRequest, opts ...grpc.CallOption) (SeaweedFiler_StreamRenameEntryClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedFiler_ServiceDesc.Streams[1], "/filer_pb.SeaweedFiler/StreamRenameEntry", opts...) + if err != nil { + return nil, err + } + x := &seaweedFilerStreamRenameEntryClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type SeaweedFiler_StreamRenameEntryClient interface { + Recv() (*StreamRenameEntryResponse, error) + grpc.ClientStream +} + +type seaweedFilerStreamRenameEntryClient struct { + grpc.ClientStream +} + +func (x *seaweedFilerStreamRenameEntryClient) Recv() (*StreamRenameEntryResponse, error) { + m := new(StreamRenameEntryResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedFilerClient) AssignVolume(ctx context.Context, in *AssignVolumeRequest, opts ...grpc.CallOption) (*AssignVolumeResponse, error) { + out := new(AssignVolumeResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/AssignVolume", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) { + out := new(LookupVolumeResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/LookupVolume", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) { + out := new(CollectionListResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/CollectionList", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) { + out := new(DeleteCollectionResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/DeleteCollection", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) { + out := new(StatisticsResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/Statistics", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) GetFilerConfiguration(ctx context.Context, in *GetFilerConfigurationRequest, opts ...grpc.CallOption) (*GetFilerConfigurationResponse, error) { + out := new(GetFilerConfigurationResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/GetFilerConfiguration", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) SubscribeMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeMetadataClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedFiler_ServiceDesc.Streams[2], "/filer_pb.SeaweedFiler/SubscribeMetadata", opts...) + if err != nil { + return nil, err + } + x := &seaweedFilerSubscribeMetadataClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type SeaweedFiler_SubscribeMetadataClient interface { + Recv() (*SubscribeMetadataResponse, error) + grpc.ClientStream +} + +type seaweedFilerSubscribeMetadataClient struct { + grpc.ClientStream +} + +func (x *seaweedFilerSubscribeMetadataClient) Recv() (*SubscribeMetadataResponse, error) { + m := new(SubscribeMetadataResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedFilerClient) SubscribeLocalMetadata(ctx context.Context, in *SubscribeMetadataRequest, opts ...grpc.CallOption) (SeaweedFiler_SubscribeLocalMetadataClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedFiler_ServiceDesc.Streams[3], "/filer_pb.SeaweedFiler/SubscribeLocalMetadata", opts...) + if err != nil { + return nil, err + } + x := &seaweedFilerSubscribeLocalMetadataClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type SeaweedFiler_SubscribeLocalMetadataClient interface { + Recv() (*SubscribeMetadataResponse, error) + grpc.ClientStream +} + +type seaweedFilerSubscribeLocalMetadataClient struct { + grpc.ClientStream +} + +func (x *seaweedFilerSubscribeLocalMetadataClient) Recv() (*SubscribeMetadataResponse, error) { + m := new(SubscribeMetadataResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedFilerClient) KeepConnected(ctx context.Context, opts ...grpc.CallOption) (SeaweedFiler_KeepConnectedClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedFiler_ServiceDesc.Streams[4], "/filer_pb.SeaweedFiler/KeepConnected", opts...) + if err != nil { + return nil, err + } + x := &seaweedFilerKeepConnectedClient{stream} + return x, nil +} + +type SeaweedFiler_KeepConnectedClient interface { + Send(*KeepConnectedRequest) error + Recv() (*KeepConnectedResponse, error) + grpc.ClientStream +} + +type seaweedFilerKeepConnectedClient struct { + grpc.ClientStream +} + +func (x *seaweedFilerKeepConnectedClient) Send(m *KeepConnectedRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *seaweedFilerKeepConnectedClient) Recv() (*KeepConnectedResponse, error) { + m := new(KeepConnectedResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedFilerClient) LocateBroker(ctx context.Context, in *LocateBrokerRequest, opts ...grpc.CallOption) (*LocateBrokerResponse, error) { + out := new(LocateBrokerResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/LocateBroker", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) KvGet(ctx context.Context, in *KvGetRequest, opts ...grpc.CallOption) (*KvGetResponse, error) { + out := new(KvGetResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/KvGet", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) KvPut(ctx context.Context, in *KvPutRequest, opts ...grpc.CallOption) (*KvPutResponse, error) { + out := new(KvPutResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/KvPut", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedFilerClient) CacheRemoteObjectToLocalCluster(ctx context.Context, in *CacheRemoteObjectToLocalClusterRequest, opts ...grpc.CallOption) (*CacheRemoteObjectToLocalClusterResponse, error) { + out := new(CacheRemoteObjectToLocalClusterResponse) + err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/CacheRemoteObjectToLocalCluster", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// SeaweedFilerServer is the server API for SeaweedFiler service. +// All implementations must embed UnimplementedSeaweedFilerServer +// for forward compatibility +type SeaweedFilerServer interface { + LookupDirectoryEntry(context.Context, *LookupDirectoryEntryRequest) (*LookupDirectoryEntryResponse, error) + ListEntries(*ListEntriesRequest, SeaweedFiler_ListEntriesServer) error + CreateEntry(context.Context, *CreateEntryRequest) (*CreateEntryResponse, error) + UpdateEntry(context.Context, *UpdateEntryRequest) (*UpdateEntryResponse, error) + AppendToEntry(context.Context, *AppendToEntryRequest) (*AppendToEntryResponse, error) + DeleteEntry(context.Context, *DeleteEntryRequest) (*DeleteEntryResponse, error) + AtomicRenameEntry(context.Context, *AtomicRenameEntryRequest) (*AtomicRenameEntryResponse, error) + StreamRenameEntry(*StreamRenameEntryRequest, SeaweedFiler_StreamRenameEntryServer) error + AssignVolume(context.Context, *AssignVolumeRequest) (*AssignVolumeResponse, error) + LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) + CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) + DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) + Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) + GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error) + SubscribeMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeMetadataServer) error + SubscribeLocalMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeLocalMetadataServer) error + KeepConnected(SeaweedFiler_KeepConnectedServer) error + LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error) + KvGet(context.Context, *KvGetRequest) (*KvGetResponse, error) + KvPut(context.Context, *KvPutRequest) (*KvPutResponse, error) + CacheRemoteObjectToLocalCluster(context.Context, *CacheRemoteObjectToLocalClusterRequest) (*CacheRemoteObjectToLocalClusterResponse, error) + mustEmbedUnimplementedSeaweedFilerServer() +} + +// UnimplementedSeaweedFilerServer must be embedded to have forward compatible implementations. +type UnimplementedSeaweedFilerServer struct { +} + +func (UnimplementedSeaweedFilerServer) LookupDirectoryEntry(context.Context, *LookupDirectoryEntryRequest) (*LookupDirectoryEntryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupDirectoryEntry not implemented") +} +func (UnimplementedSeaweedFilerServer) ListEntries(*ListEntriesRequest, SeaweedFiler_ListEntriesServer) error { + return status.Errorf(codes.Unimplemented, "method ListEntries not implemented") +} +func (UnimplementedSeaweedFilerServer) CreateEntry(context.Context, *CreateEntryRequest) (*CreateEntryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateEntry not implemented") +} +func (UnimplementedSeaweedFilerServer) UpdateEntry(context.Context, *UpdateEntryRequest) (*UpdateEntryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateEntry not implemented") +} +func (UnimplementedSeaweedFilerServer) AppendToEntry(context.Context, *AppendToEntryRequest) (*AppendToEntryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AppendToEntry not implemented") +} +func (UnimplementedSeaweedFilerServer) DeleteEntry(context.Context, *DeleteEntryRequest) (*DeleteEntryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteEntry not implemented") +} +func (UnimplementedSeaweedFilerServer) AtomicRenameEntry(context.Context, *AtomicRenameEntryRequest) (*AtomicRenameEntryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AtomicRenameEntry not implemented") +} +func (UnimplementedSeaweedFilerServer) StreamRenameEntry(*StreamRenameEntryRequest, SeaweedFiler_StreamRenameEntryServer) error { + return status.Errorf(codes.Unimplemented, "method StreamRenameEntry not implemented") +} +func (UnimplementedSeaweedFilerServer) AssignVolume(context.Context, *AssignVolumeRequest) (*AssignVolumeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AssignVolume not implemented") +} +func (UnimplementedSeaweedFilerServer) LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupVolume not implemented") +} +func (UnimplementedSeaweedFilerServer) CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CollectionList not implemented") +} +func (UnimplementedSeaweedFilerServer) DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteCollection not implemented") +} +func (UnimplementedSeaweedFilerServer) Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Statistics not implemented") +} +func (UnimplementedSeaweedFilerServer) GetFilerConfiguration(context.Context, *GetFilerConfigurationRequest) (*GetFilerConfigurationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetFilerConfiguration not implemented") +} +func (UnimplementedSeaweedFilerServer) SubscribeMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeMetadataServer) error { + return status.Errorf(codes.Unimplemented, "method SubscribeMetadata not implemented") +} +func (UnimplementedSeaweedFilerServer) SubscribeLocalMetadata(*SubscribeMetadataRequest, SeaweedFiler_SubscribeLocalMetadataServer) error { + return status.Errorf(codes.Unimplemented, "method SubscribeLocalMetadata not implemented") +} +func (UnimplementedSeaweedFilerServer) KeepConnected(SeaweedFiler_KeepConnectedServer) error { + return status.Errorf(codes.Unimplemented, "method KeepConnected not implemented") +} +func (UnimplementedSeaweedFilerServer) LocateBroker(context.Context, *LocateBrokerRequest) (*LocateBrokerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LocateBroker not implemented") +} +func (UnimplementedSeaweedFilerServer) KvGet(context.Context, *KvGetRequest) (*KvGetResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method KvGet not implemented") +} +func (UnimplementedSeaweedFilerServer) KvPut(context.Context, *KvPutRequest) (*KvPutResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method KvPut not implemented") +} +func (UnimplementedSeaweedFilerServer) CacheRemoteObjectToLocalCluster(context.Context, *CacheRemoteObjectToLocalClusterRequest) (*CacheRemoteObjectToLocalClusterResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CacheRemoteObjectToLocalCluster not implemented") +} +func (UnimplementedSeaweedFilerServer) mustEmbedUnimplementedSeaweedFilerServer() {} + +// UnsafeSeaweedFilerServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to SeaweedFilerServer will +// result in compilation errors. +type UnsafeSeaweedFilerServer interface { + mustEmbedUnimplementedSeaweedFilerServer() +} + +func RegisterSeaweedFilerServer(s grpc.ServiceRegistrar, srv SeaweedFilerServer) { + s.RegisterService(&SeaweedFiler_ServiceDesc, srv) +} + +func _SeaweedFiler_LookupDirectoryEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupDirectoryEntryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).LookupDirectoryEntry(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/LookupDirectoryEntry", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).LookupDirectoryEntry(ctx, req.(*LookupDirectoryEntryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_ListEntries_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(ListEntriesRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(SeaweedFilerServer).ListEntries(m, &seaweedFilerListEntriesServer{stream}) +} + +type SeaweedFiler_ListEntriesServer interface { + Send(*ListEntriesResponse) error + grpc.ServerStream +} + +type seaweedFilerListEntriesServer struct { + grpc.ServerStream +} + +func (x *seaweedFilerListEntriesServer) Send(m *ListEntriesResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _SeaweedFiler_CreateEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateEntryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).CreateEntry(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/CreateEntry", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).CreateEntry(ctx, req.(*CreateEntryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_UpdateEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateEntryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).UpdateEntry(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/UpdateEntry", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).UpdateEntry(ctx, req.(*UpdateEntryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_AppendToEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AppendToEntryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).AppendToEntry(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/AppendToEntry", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).AppendToEntry(ctx, req.(*AppendToEntryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_DeleteEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteEntryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).DeleteEntry(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/DeleteEntry", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).DeleteEntry(ctx, req.(*DeleteEntryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_AtomicRenameEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AtomicRenameEntryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).AtomicRenameEntry(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/AtomicRenameEntry", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).AtomicRenameEntry(ctx, req.(*AtomicRenameEntryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_StreamRenameEntry_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(StreamRenameEntryRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(SeaweedFilerServer).StreamRenameEntry(m, &seaweedFilerStreamRenameEntryServer{stream}) +} + +type SeaweedFiler_StreamRenameEntryServer interface { + Send(*StreamRenameEntryResponse) error + grpc.ServerStream +} + +type seaweedFilerStreamRenameEntryServer struct { + grpc.ServerStream +} + +func (x *seaweedFilerStreamRenameEntryServer) Send(m *StreamRenameEntryResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _SeaweedFiler_AssignVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AssignVolumeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).AssignVolume(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/AssignVolume", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).AssignVolume(ctx, req.(*AssignVolumeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_LookupVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupVolumeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).LookupVolume(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/LookupVolume", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).LookupVolume(ctx, req.(*LookupVolumeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_CollectionList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CollectionListRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).CollectionList(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/CollectionList", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).CollectionList(ctx, req.(*CollectionListRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_DeleteCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).DeleteCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/DeleteCollection", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).DeleteCollection(ctx, req.(*DeleteCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_Statistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).Statistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/Statistics", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).Statistics(ctx, req.(*StatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_GetFilerConfiguration_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetFilerConfigurationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).GetFilerConfiguration(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/GetFilerConfiguration", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).GetFilerConfiguration(ctx, req.(*GetFilerConfigurationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_SubscribeMetadata_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(SubscribeMetadataRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(SeaweedFilerServer).SubscribeMetadata(m, &seaweedFilerSubscribeMetadataServer{stream}) +} + +type SeaweedFiler_SubscribeMetadataServer interface { + Send(*SubscribeMetadataResponse) error + grpc.ServerStream +} + +type seaweedFilerSubscribeMetadataServer struct { + grpc.ServerStream +} + +func (x *seaweedFilerSubscribeMetadataServer) Send(m *SubscribeMetadataResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _SeaweedFiler_SubscribeLocalMetadata_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(SubscribeMetadataRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(SeaweedFilerServer).SubscribeLocalMetadata(m, &seaweedFilerSubscribeLocalMetadataServer{stream}) +} + +type SeaweedFiler_SubscribeLocalMetadataServer interface { + Send(*SubscribeMetadataResponse) error + grpc.ServerStream +} + +type seaweedFilerSubscribeLocalMetadataServer struct { + grpc.ServerStream +} + +func (x *seaweedFilerSubscribeLocalMetadataServer) Send(m *SubscribeMetadataResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _SeaweedFiler_KeepConnected_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedFilerServer).KeepConnected(&seaweedFilerKeepConnectedServer{stream}) +} + +type SeaweedFiler_KeepConnectedServer interface { + Send(*KeepConnectedResponse) error + Recv() (*KeepConnectedRequest, error) + grpc.ServerStream +} + +type seaweedFilerKeepConnectedServer struct { + grpc.ServerStream +} + +func (x *seaweedFilerKeepConnectedServer) Send(m *KeepConnectedResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *seaweedFilerKeepConnectedServer) Recv() (*KeepConnectedRequest, error) { + m := new(KeepConnectedRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _SeaweedFiler_LocateBroker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LocateBrokerRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).LocateBroker(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/LocateBroker", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).LocateBroker(ctx, req.(*LocateBrokerRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_KvGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(KvGetRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).KvGet(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/KvGet", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).KvGet(ctx, req.(*KvGetRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_KvPut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(KvPutRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).KvPut(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/KvPut", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).KvPut(ctx, req.(*KvPutRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedFiler_CacheRemoteObjectToLocalCluster_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CacheRemoteObjectToLocalClusterRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedFilerServer).CacheRemoteObjectToLocalCluster(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/filer_pb.SeaweedFiler/CacheRemoteObjectToLocalCluster", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedFilerServer).CacheRemoteObjectToLocalCluster(ctx, req.(*CacheRemoteObjectToLocalClusterRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// SeaweedFiler_ServiceDesc is the grpc.ServiceDesc for SeaweedFiler service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var SeaweedFiler_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "filer_pb.SeaweedFiler", + HandlerType: (*SeaweedFilerServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "LookupDirectoryEntry", + Handler: _SeaweedFiler_LookupDirectoryEntry_Handler, + }, + { + MethodName: "CreateEntry", + Handler: _SeaweedFiler_CreateEntry_Handler, + }, + { + MethodName: "UpdateEntry", + Handler: _SeaweedFiler_UpdateEntry_Handler, + }, + { + MethodName: "AppendToEntry", + Handler: _SeaweedFiler_AppendToEntry_Handler, + }, + { + MethodName: "DeleteEntry", + Handler: _SeaweedFiler_DeleteEntry_Handler, + }, + { + MethodName: "AtomicRenameEntry", + Handler: _SeaweedFiler_AtomicRenameEntry_Handler, + }, + { + MethodName: "AssignVolume", + Handler: _SeaweedFiler_AssignVolume_Handler, + }, + { + MethodName: "LookupVolume", + Handler: _SeaweedFiler_LookupVolume_Handler, + }, + { + MethodName: "CollectionList", + Handler: _SeaweedFiler_CollectionList_Handler, + }, + { + MethodName: "DeleteCollection", + Handler: _SeaweedFiler_DeleteCollection_Handler, + }, + { + MethodName: "Statistics", + Handler: _SeaweedFiler_Statistics_Handler, + }, + { + MethodName: "GetFilerConfiguration", + Handler: _SeaweedFiler_GetFilerConfiguration_Handler, + }, + { + MethodName: "LocateBroker", + Handler: _SeaweedFiler_LocateBroker_Handler, + }, + { + MethodName: "KvGet", + Handler: _SeaweedFiler_KvGet_Handler, + }, + { + MethodName: "KvPut", + Handler: _SeaweedFiler_KvPut_Handler, + }, + { + MethodName: "CacheRemoteObjectToLocalCluster", + Handler: _SeaweedFiler_CacheRemoteObjectToLocalCluster_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "ListEntries", + Handler: _SeaweedFiler_ListEntries_Handler, + ServerStreams: true, + }, + { + StreamName: "StreamRenameEntry", + Handler: _SeaweedFiler_StreamRenameEntry_Handler, + ServerStreams: true, + }, + { + StreamName: "SubscribeMetadata", + Handler: _SeaweedFiler_SubscribeMetadata_Handler, + ServerStreams: true, + }, + { + StreamName: "SubscribeLocalMetadata", + Handler: _SeaweedFiler_SubscribeLocalMetadata_Handler, + ServerStreams: true, + }, + { + StreamName: "KeepConnected", + Handler: _SeaweedFiler_KeepConnected_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "filer.proto", +} diff --git a/weed/pb/iam_pb/iam.pb.go b/weed/pb/iam_pb/iam.pb.go index 7d0b6281b..89a4f1584 100644 --- a/weed/pb/iam_pb/iam.pb.go +++ b/weed/pb/iam_pb/iam.pb.go @@ -1,15 +1,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.25.0 -// protoc v3.12.3 +// protoc-gen-go v1.26.0 +// protoc v3.17.3 // source: iam.proto package iam_pb import ( - context "context" - proto "github.com/golang/protobuf/proto" - grpc "google.golang.org/grpc" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -23,10 +20,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// This is a compile-time assertion that a sufficiently up-to-date version -// of the legacy proto package is being used. -const _ = proto.ProtoPackageIsVersion4 - type S3ApiConfiguration struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -312,45 +305,3 @@ func file_iam_proto_init() { file_iam_proto_goTypes = nil file_iam_proto_depIdxs = nil } - -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConnInterface - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion6 - -// SeaweedIdentityAccessManagementClient is the client API for SeaweedIdentityAccessManagement service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type SeaweedIdentityAccessManagementClient interface { -} - -type seaweedIdentityAccessManagementClient struct { - cc grpc.ClientConnInterface -} - -func NewSeaweedIdentityAccessManagementClient(cc grpc.ClientConnInterface) SeaweedIdentityAccessManagementClient { - return &seaweedIdentityAccessManagementClient{cc} -} - -// SeaweedIdentityAccessManagementServer is the server API for SeaweedIdentityAccessManagement service. -type SeaweedIdentityAccessManagementServer interface { -} - -// UnimplementedSeaweedIdentityAccessManagementServer can be embedded to have forward compatible implementations. -type UnimplementedSeaweedIdentityAccessManagementServer struct { -} - -func RegisterSeaweedIdentityAccessManagementServer(s *grpc.Server, srv SeaweedIdentityAccessManagementServer) { - s.RegisterService(&_SeaweedIdentityAccessManagement_serviceDesc, srv) -} - -var _SeaweedIdentityAccessManagement_serviceDesc = grpc.ServiceDesc{ - ServiceName: "iam_pb.SeaweedIdentityAccessManagement", - HandlerType: (*SeaweedIdentityAccessManagementServer)(nil), - Methods: []grpc.MethodDesc{}, - Streams: []grpc.StreamDesc{}, - Metadata: "iam.proto", -} diff --git a/weed/pb/iam_pb/iam_grpc.pb.go b/weed/pb/iam_pb/iam_grpc.pb.go new file mode 100644 index 000000000..b9438a295 --- /dev/null +++ b/weed/pb/iam_pb/iam_grpc.pb.go @@ -0,0 +1,62 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. + +package iam_pb + +import ( + grpc "google.golang.org/grpc" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// SeaweedIdentityAccessManagementClient is the client API for SeaweedIdentityAccessManagement service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type SeaweedIdentityAccessManagementClient interface { +} + +type seaweedIdentityAccessManagementClient struct { + cc grpc.ClientConnInterface +} + +func NewSeaweedIdentityAccessManagementClient(cc grpc.ClientConnInterface) SeaweedIdentityAccessManagementClient { + return &seaweedIdentityAccessManagementClient{cc} +} + +// SeaweedIdentityAccessManagementServer is the server API for SeaweedIdentityAccessManagement service. +// All implementations must embed UnimplementedSeaweedIdentityAccessManagementServer +// for forward compatibility +type SeaweedIdentityAccessManagementServer interface { + mustEmbedUnimplementedSeaweedIdentityAccessManagementServer() +} + +// UnimplementedSeaweedIdentityAccessManagementServer must be embedded to have forward compatible implementations. +type UnimplementedSeaweedIdentityAccessManagementServer struct { +} + +func (UnimplementedSeaweedIdentityAccessManagementServer) mustEmbedUnimplementedSeaweedIdentityAccessManagementServer() { +} + +// UnsafeSeaweedIdentityAccessManagementServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to SeaweedIdentityAccessManagementServer will +// result in compilation errors. +type UnsafeSeaweedIdentityAccessManagementServer interface { + mustEmbedUnimplementedSeaweedIdentityAccessManagementServer() +} + +func RegisterSeaweedIdentityAccessManagementServer(s grpc.ServiceRegistrar, srv SeaweedIdentityAccessManagementServer) { + s.RegisterService(&SeaweedIdentityAccessManagement_ServiceDesc, srv) +} + +// SeaweedIdentityAccessManagement_ServiceDesc is the grpc.ServiceDesc for SeaweedIdentityAccessManagement service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var SeaweedIdentityAccessManagement_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "iam_pb.SeaweedIdentityAccessManagement", + HandlerType: (*SeaweedIdentityAccessManagementServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{}, + Metadata: "iam.proto", +} diff --git a/weed/pb/master.proto b/weed/pb/master.proto index 3383d1dd5..7a257bb28 100644 --- a/weed/pb/master.proto +++ b/weed/pb/master.proto @@ -314,6 +314,7 @@ message LeaseAdminTokenRequest { int64 previous_lock_time = 2; string lock_name = 3; string client_name = 4; + string message = 5; } message LeaseAdminTokenResponse { int64 token = 1; diff --git a/weed/pb/master_pb/master.pb.go b/weed/pb/master_pb/master.pb.go index ca8a50931..681534fc3 100644 --- a/weed/pb/master_pb/master.pb.go +++ b/weed/pb/master_pb/master.pb.go @@ -1,17 +1,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.25.0 -// protoc v3.12.3 +// protoc-gen-go v1.26.0 +// protoc v3.17.3 // source: master.proto package master_pb import ( - context "context" - proto "github.com/golang/protobuf/proto" - grpc "google.golang.org/grpc" - codes "google.golang.org/grpc/codes" - status "google.golang.org/grpc/status" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -25,10 +20,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// This is a compile-time assertion that a sufficiently up-to-date version -// of the legacy proto package is being used. -const _ = proto.ProtoPackageIsVersion4 - type Heartbeat struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2651,6 +2642,7 @@ type LeaseAdminTokenRequest struct { PreviousLockTime int64 `protobuf:"varint,2,opt,name=previous_lock_time,json=previousLockTime,proto3" json:"previous_lock_time,omitempty"` LockName string `protobuf:"bytes,3,opt,name=lock_name,json=lockName,proto3" json:"lock_name,omitempty"` ClientName string `protobuf:"bytes,4,opt,name=client_name,json=clientName,proto3" json:"client_name,omitempty"` + Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` } func (x *LeaseAdminTokenRequest) Reset() { @@ -2713,6 +2705,13 @@ func (x *LeaseAdminTokenRequest) GetClientName() string { return "" } +func (x *LeaseAdminTokenRequest) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + type LeaseAdminTokenResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3580,7 +3579,7 @@ var file_master_proto_rawDesc = []byte{ 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x22, 0xab, 0x01, 0x0a, 0x16, 0x4c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, + 0x22, 0xc5, 0x01, 0x0a, 0x16, 0x4c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x54, 0x6f, 0x6b, @@ -3590,104 +3589,105 @@ var file_master_proto_rawDesc = []byte{ 0x12, 0x1b, 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x4d, - 0x0a, 0x17, 0x4c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, - 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x12, - 0x1c, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 0x8c, 0x01, - 0x0a, 0x18, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, - 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, - 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0d, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x54, 0x6f, 0x6b, 0x65, - 0x6e, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x6c, 0x6f, - 0x63, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x70, - 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x4c, 0x6f, 0x63, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x12, - 0x1b, 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x1b, 0x0a, 0x19, - 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xce, 0x09, 0x0a, 0x07, 0x53, 0x65, - 0x61, 0x77, 0x65, 0x65, 0x64, 0x12, 0x49, 0x0a, 0x0d, 0x53, 0x65, 0x6e, 0x64, 0x48, 0x65, 0x61, - 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x14, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, - 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x1a, 0x1c, 0x2e, 0x6d, - 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, - 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, - 0x12, 0x58, 0x0a, 0x0d, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, - 0x64, 0x12, 0x1f, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x65, - 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, - 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x51, 0x0a, 0x0c, 0x4c, 0x6f, - 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1e, 0x2e, 0x6d, 0x61, 0x73, - 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, - 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x6d, 0x61, 0x73, - 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, - 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3f, 0x0a, - 0x06, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x12, 0x18, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, - 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x19, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, - 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, - 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x1c, 0x2e, 0x6d, - 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, - 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x6d, 0x61, 0x73, - 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x20, 0x2e, - 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x21, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x22, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, - 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, - 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, - 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4c, 0x69, 0x73, - 0x74, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, - 0x6c, 0x75, 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1d, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, - 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x57, 0x0a, 0x0e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x45, 0x63, 0x56, 0x6f, 0x6c, 0x75, - 0x6d, 0x65, 0x12, 0x20, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, - 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x45, 0x63, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, - 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x45, 0x63, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x56, 0x61, 0x63, - 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1e, 0x2e, 0x6d, 0x61, 0x73, 0x74, - 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, - 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x6d, 0x61, 0x73, 0x74, - 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, - 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, - 0x47, 0x65, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, - 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x29, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, + 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, + 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x4d, 0x0a, 0x17, 0x4c, 0x65, 0x61, 0x73, + 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1c, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, + 0x6b, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x6c, + 0x6f, 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 0x8c, 0x01, 0x0a, 0x18, 0x52, 0x65, 0x6c, 0x65, + 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, + 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x70, 0x72, + 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x2c, 0x0a, 0x12, 0x70, + 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, + 0x73, 0x4c, 0x6f, 0x63, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x6c, 0x6f, 0x63, + 0x6b, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x6f, + 0x63, 0x6b, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x1b, 0x0a, 0x19, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, + 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x32, 0xce, 0x09, 0x0a, 0x07, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x12, + 0x49, 0x0a, 0x0d, 0x53, 0x65, 0x6e, 0x64, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, + 0x12, 0x14, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61, + 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, + 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x58, 0x0a, 0x0d, 0x4b, 0x65, + 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x12, 0x1f, 0x2e, 0x6d, 0x61, + 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, + 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x28, 0x01, 0x30, 0x01, 0x12, 0x51, 0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1e, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3f, 0x0a, 0x06, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x12, 0x18, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x6d, 0x61, + 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, + 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, + 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x20, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, + 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, + 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x6d, 0x61, 0x73, 0x74, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, + 0x0a, 0x10, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x12, 0x22, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, + 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, + 0x0a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1c, 0x2e, 0x6d, 0x61, + 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4c, 0x69, + 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x6d, 0x61, 0x73, 0x74, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x4c, 0x6f, + 0x6f, 0x6b, 0x75, 0x70, 0x45, 0x63, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x20, 0x2e, 0x6d, + 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x45, + 0x63, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, + 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, + 0x70, 0x45, 0x63, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x12, 0x1e, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, + 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, + 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x73, + 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x28, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, - 0x10, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, - 0x73, 0x12, 0x22, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, - 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, - 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, - 0x4c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, - 0x21, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x65, 0x61, 0x73, - 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, - 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, - 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x23, 0x2e, - 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, - 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, - 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x69, 0x73, 0x6c, 0x75, - 0x73, 0x66, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, - 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x61, 0x73, + 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x43, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x22, 0x2e, 0x6d, 0x61, + 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x23, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x4c, 0x65, 0x61, 0x73, 0x65, 0x41, + 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x21, 0x2e, 0x6d, 0x61, 0x73, 0x74, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x6d, + 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, + 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, + 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x23, 0x2e, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, + 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x6e, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, + 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, + 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x69, 0x73, 0x6c, 0x75, 0x73, 0x66, 0x2f, 0x73, 0x65, 0x61, + 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x6d, + 0x61, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -4419,616 +4419,3 @@ func file_master_proto_init() { file_master_proto_goTypes = nil file_master_proto_depIdxs = nil } - -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConnInterface - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion6 - -// SeaweedClient is the client API for Seaweed service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type SeaweedClient interface { - SendHeartbeat(ctx context.Context, opts ...grpc.CallOption) (Seaweed_SendHeartbeatClient, error) - KeepConnected(ctx context.Context, opts ...grpc.CallOption) (Seaweed_KeepConnectedClient, error) - LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) - Assign(ctx context.Context, in *AssignRequest, opts ...grpc.CallOption) (*AssignResponse, error) - Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) - CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) - CollectionDelete(ctx context.Context, in *CollectionDeleteRequest, opts ...grpc.CallOption) (*CollectionDeleteResponse, error) - VolumeList(ctx context.Context, in *VolumeListRequest, opts ...grpc.CallOption) (*VolumeListResponse, error) - LookupEcVolume(ctx context.Context, in *LookupEcVolumeRequest, opts ...grpc.CallOption) (*LookupEcVolumeResponse, error) - VacuumVolume(ctx context.Context, in *VacuumVolumeRequest, opts ...grpc.CallOption) (*VacuumVolumeResponse, error) - GetMasterConfiguration(ctx context.Context, in *GetMasterConfigurationRequest, opts ...grpc.CallOption) (*GetMasterConfigurationResponse, error) - ListClusterNodes(ctx context.Context, in *ListClusterNodesRequest, opts ...grpc.CallOption) (*ListClusterNodesResponse, error) - LeaseAdminToken(ctx context.Context, in *LeaseAdminTokenRequest, opts ...grpc.CallOption) (*LeaseAdminTokenResponse, error) - ReleaseAdminToken(ctx context.Context, in *ReleaseAdminTokenRequest, opts ...grpc.CallOption) (*ReleaseAdminTokenResponse, error) -} - -type seaweedClient struct { - cc grpc.ClientConnInterface -} - -func NewSeaweedClient(cc grpc.ClientConnInterface) SeaweedClient { - return &seaweedClient{cc} -} - -func (c *seaweedClient) SendHeartbeat(ctx context.Context, opts ...grpc.CallOption) (Seaweed_SendHeartbeatClient, error) { - stream, err := c.cc.NewStream(ctx, &_Seaweed_serviceDesc.Streams[0], "/master_pb.Seaweed/SendHeartbeat", opts...) - if err != nil { - return nil, err - } - x := &seaweedSendHeartbeatClient{stream} - return x, nil -} - -type Seaweed_SendHeartbeatClient interface { - Send(*Heartbeat) error - Recv() (*HeartbeatResponse, error) - grpc.ClientStream -} - -type seaweedSendHeartbeatClient struct { - grpc.ClientStream -} - -func (x *seaweedSendHeartbeatClient) Send(m *Heartbeat) error { - return x.ClientStream.SendMsg(m) -} - -func (x *seaweedSendHeartbeatClient) Recv() (*HeartbeatResponse, error) { - m := new(HeartbeatResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedClient) KeepConnected(ctx context.Context, opts ...grpc.CallOption) (Seaweed_KeepConnectedClient, error) { - stream, err := c.cc.NewStream(ctx, &_Seaweed_serviceDesc.Streams[1], "/master_pb.Seaweed/KeepConnected", opts...) - if err != nil { - return nil, err - } - x := &seaweedKeepConnectedClient{stream} - return x, nil -} - -type Seaweed_KeepConnectedClient interface { - Send(*KeepConnectedRequest) error - Recv() (*KeepConnectedResponse, error) - grpc.ClientStream -} - -type seaweedKeepConnectedClient struct { - grpc.ClientStream -} - -func (x *seaweedKeepConnectedClient) Send(m *KeepConnectedRequest) error { - return x.ClientStream.SendMsg(m) -} - -func (x *seaweedKeepConnectedClient) Recv() (*KeepConnectedResponse, error) { - m := new(KeepConnectedResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedClient) LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) { - out := new(LookupVolumeResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/LookupVolume", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) Assign(ctx context.Context, in *AssignRequest, opts ...grpc.CallOption) (*AssignResponse, error) { - out := new(AssignResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/Assign", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) { - out := new(StatisticsResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/Statistics", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) { - out := new(CollectionListResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/CollectionList", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) CollectionDelete(ctx context.Context, in *CollectionDeleteRequest, opts ...grpc.CallOption) (*CollectionDeleteResponse, error) { - out := new(CollectionDeleteResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/CollectionDelete", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) VolumeList(ctx context.Context, in *VolumeListRequest, opts ...grpc.CallOption) (*VolumeListResponse, error) { - out := new(VolumeListResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/VolumeList", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) LookupEcVolume(ctx context.Context, in *LookupEcVolumeRequest, opts ...grpc.CallOption) (*LookupEcVolumeResponse, error) { - out := new(LookupEcVolumeResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/LookupEcVolume", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) VacuumVolume(ctx context.Context, in *VacuumVolumeRequest, opts ...grpc.CallOption) (*VacuumVolumeResponse, error) { - out := new(VacuumVolumeResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/VacuumVolume", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) GetMasterConfiguration(ctx context.Context, in *GetMasterConfigurationRequest, opts ...grpc.CallOption) (*GetMasterConfigurationResponse, error) { - out := new(GetMasterConfigurationResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/GetMasterConfiguration", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) ListClusterNodes(ctx context.Context, in *ListClusterNodesRequest, opts ...grpc.CallOption) (*ListClusterNodesResponse, error) { - out := new(ListClusterNodesResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/ListClusterNodes", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) LeaseAdminToken(ctx context.Context, in *LeaseAdminTokenRequest, opts ...grpc.CallOption) (*LeaseAdminTokenResponse, error) { - out := new(LeaseAdminTokenResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/LeaseAdminToken", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedClient) ReleaseAdminToken(ctx context.Context, in *ReleaseAdminTokenRequest, opts ...grpc.CallOption) (*ReleaseAdminTokenResponse, error) { - out := new(ReleaseAdminTokenResponse) - err := c.cc.Invoke(ctx, "/master_pb.Seaweed/ReleaseAdminToken", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -// SeaweedServer is the server API for Seaweed service. -type SeaweedServer interface { - SendHeartbeat(Seaweed_SendHeartbeatServer) error - KeepConnected(Seaweed_KeepConnectedServer) error - LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) - Assign(context.Context, *AssignRequest) (*AssignResponse, error) - Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) - CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) - CollectionDelete(context.Context, *CollectionDeleteRequest) (*CollectionDeleteResponse, error) - VolumeList(context.Context, *VolumeListRequest) (*VolumeListResponse, error) - LookupEcVolume(context.Context, *LookupEcVolumeRequest) (*LookupEcVolumeResponse, error) - VacuumVolume(context.Context, *VacuumVolumeRequest) (*VacuumVolumeResponse, error) - GetMasterConfiguration(context.Context, *GetMasterConfigurationRequest) (*GetMasterConfigurationResponse, error) - ListClusterNodes(context.Context, *ListClusterNodesRequest) (*ListClusterNodesResponse, error) - LeaseAdminToken(context.Context, *LeaseAdminTokenRequest) (*LeaseAdminTokenResponse, error) - ReleaseAdminToken(context.Context, *ReleaseAdminTokenRequest) (*ReleaseAdminTokenResponse, error) -} - -// UnimplementedSeaweedServer can be embedded to have forward compatible implementations. -type UnimplementedSeaweedServer struct { -} - -func (*UnimplementedSeaweedServer) SendHeartbeat(Seaweed_SendHeartbeatServer) error { - return status.Errorf(codes.Unimplemented, "method SendHeartbeat not implemented") -} -func (*UnimplementedSeaweedServer) KeepConnected(Seaweed_KeepConnectedServer) error { - return status.Errorf(codes.Unimplemented, "method KeepConnected not implemented") -} -func (*UnimplementedSeaweedServer) LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method LookupVolume not implemented") -} -func (*UnimplementedSeaweedServer) Assign(context.Context, *AssignRequest) (*AssignResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method Assign not implemented") -} -func (*UnimplementedSeaweedServer) Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method Statistics not implemented") -} -func (*UnimplementedSeaweedServer) CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CollectionList not implemented") -} -func (*UnimplementedSeaweedServer) CollectionDelete(context.Context, *CollectionDeleteRequest) (*CollectionDeleteResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CollectionDelete not implemented") -} -func (*UnimplementedSeaweedServer) VolumeList(context.Context, *VolumeListRequest) (*VolumeListResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeList not implemented") -} -func (*UnimplementedSeaweedServer) LookupEcVolume(context.Context, *LookupEcVolumeRequest) (*LookupEcVolumeResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method LookupEcVolume not implemented") -} -func (*UnimplementedSeaweedServer) VacuumVolume(context.Context, *VacuumVolumeRequest) (*VacuumVolumeResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VacuumVolume not implemented") -} -func (*UnimplementedSeaweedServer) GetMasterConfiguration(context.Context, *GetMasterConfigurationRequest) (*GetMasterConfigurationResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method GetMasterConfiguration not implemented") -} -func (*UnimplementedSeaweedServer) ListClusterNodes(context.Context, *ListClusterNodesRequest) (*ListClusterNodesResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ListClusterNodes not implemented") -} -func (*UnimplementedSeaweedServer) LeaseAdminToken(context.Context, *LeaseAdminTokenRequest) (*LeaseAdminTokenResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method LeaseAdminToken not implemented") -} -func (*UnimplementedSeaweedServer) ReleaseAdminToken(context.Context, *ReleaseAdminTokenRequest) (*ReleaseAdminTokenResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ReleaseAdminToken not implemented") -} - -func RegisterSeaweedServer(s *grpc.Server, srv SeaweedServer) { - s.RegisterService(&_Seaweed_serviceDesc, srv) -} - -func _Seaweed_SendHeartbeat_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(SeaweedServer).SendHeartbeat(&seaweedSendHeartbeatServer{stream}) -} - -type Seaweed_SendHeartbeatServer interface { - Send(*HeartbeatResponse) error - Recv() (*Heartbeat, error) - grpc.ServerStream -} - -type seaweedSendHeartbeatServer struct { - grpc.ServerStream -} - -func (x *seaweedSendHeartbeatServer) Send(m *HeartbeatResponse) error { - return x.ServerStream.SendMsg(m) -} - -func (x *seaweedSendHeartbeatServer) Recv() (*Heartbeat, error) { - m := new(Heartbeat) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func _Seaweed_KeepConnected_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(SeaweedServer).KeepConnected(&seaweedKeepConnectedServer{stream}) -} - -type Seaweed_KeepConnectedServer interface { - Send(*KeepConnectedResponse) error - Recv() (*KeepConnectedRequest, error) - grpc.ServerStream -} - -type seaweedKeepConnectedServer struct { - grpc.ServerStream -} - -func (x *seaweedKeepConnectedServer) Send(m *KeepConnectedResponse) error { - return x.ServerStream.SendMsg(m) -} - -func (x *seaweedKeepConnectedServer) Recv() (*KeepConnectedRequest, error) { - m := new(KeepConnectedRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func _Seaweed_LookupVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LookupVolumeRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).LookupVolume(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/LookupVolume", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).LookupVolume(ctx, req.(*LookupVolumeRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_Assign_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AssignRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).Assign(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/Assign", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).Assign(ctx, req.(*AssignRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_Statistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(StatisticsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).Statistics(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/Statistics", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).Statistics(ctx, req.(*StatisticsRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_CollectionList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CollectionListRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).CollectionList(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/CollectionList", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).CollectionList(ctx, req.(*CollectionListRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_CollectionDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CollectionDeleteRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).CollectionDelete(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/CollectionDelete", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).CollectionDelete(ctx, req.(*CollectionDeleteRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_VolumeList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeListRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).VolumeList(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/VolumeList", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).VolumeList(ctx, req.(*VolumeListRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_LookupEcVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LookupEcVolumeRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).LookupEcVolume(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/LookupEcVolume", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).LookupEcVolume(ctx, req.(*LookupEcVolumeRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_VacuumVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VacuumVolumeRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).VacuumVolume(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/VacuumVolume", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).VacuumVolume(ctx, req.(*VacuumVolumeRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_GetMasterConfiguration_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(GetMasterConfigurationRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).GetMasterConfiguration(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/GetMasterConfiguration", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).GetMasterConfiguration(ctx, req.(*GetMasterConfigurationRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_ListClusterNodes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ListClusterNodesRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).ListClusterNodes(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/ListClusterNodes", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).ListClusterNodes(ctx, req.(*ListClusterNodesRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_LeaseAdminToken_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LeaseAdminTokenRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).LeaseAdminToken(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/LeaseAdminToken", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).LeaseAdminToken(ctx, req.(*LeaseAdminTokenRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Seaweed_ReleaseAdminToken_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ReleaseAdminTokenRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedServer).ReleaseAdminToken(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/master_pb.Seaweed/ReleaseAdminToken", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedServer).ReleaseAdminToken(ctx, req.(*ReleaseAdminTokenRequest)) - } - return interceptor(ctx, in, info, handler) -} - -var _Seaweed_serviceDesc = grpc.ServiceDesc{ - ServiceName: "master_pb.Seaweed", - HandlerType: (*SeaweedServer)(nil), - Methods: []grpc.MethodDesc{ - { - MethodName: "LookupVolume", - Handler: _Seaweed_LookupVolume_Handler, - }, - { - MethodName: "Assign", - Handler: _Seaweed_Assign_Handler, - }, - { - MethodName: "Statistics", - Handler: _Seaweed_Statistics_Handler, - }, - { - MethodName: "CollectionList", - Handler: _Seaweed_CollectionList_Handler, - }, - { - MethodName: "CollectionDelete", - Handler: _Seaweed_CollectionDelete_Handler, - }, - { - MethodName: "VolumeList", - Handler: _Seaweed_VolumeList_Handler, - }, - { - MethodName: "LookupEcVolume", - Handler: _Seaweed_LookupEcVolume_Handler, - }, - { - MethodName: "VacuumVolume", - Handler: _Seaweed_VacuumVolume_Handler, - }, - { - MethodName: "GetMasterConfiguration", - Handler: _Seaweed_GetMasterConfiguration_Handler, - }, - { - MethodName: "ListClusterNodes", - Handler: _Seaweed_ListClusterNodes_Handler, - }, - { - MethodName: "LeaseAdminToken", - Handler: _Seaweed_LeaseAdminToken_Handler, - }, - { - MethodName: "ReleaseAdminToken", - Handler: _Seaweed_ReleaseAdminToken_Handler, - }, - }, - Streams: []grpc.StreamDesc{ - { - StreamName: "SendHeartbeat", - Handler: _Seaweed_SendHeartbeat_Handler, - ServerStreams: true, - ClientStreams: true, - }, - { - StreamName: "KeepConnected", - Handler: _Seaweed_KeepConnected_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, - Metadata: "master.proto", -} diff --git a/weed/pb/master_pb/master_grpc.pb.go b/weed/pb/master_pb/master_grpc.pb.go new file mode 100644 index 000000000..7046afec6 --- /dev/null +++ b/weed/pb/master_pb/master_grpc.pb.go @@ -0,0 +1,634 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. + +package master_pb + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// SeaweedClient is the client API for Seaweed service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type SeaweedClient interface { + SendHeartbeat(ctx context.Context, opts ...grpc.CallOption) (Seaweed_SendHeartbeatClient, error) + KeepConnected(ctx context.Context, opts ...grpc.CallOption) (Seaweed_KeepConnectedClient, error) + LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) + Assign(ctx context.Context, in *AssignRequest, opts ...grpc.CallOption) (*AssignResponse, error) + Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) + CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) + CollectionDelete(ctx context.Context, in *CollectionDeleteRequest, opts ...grpc.CallOption) (*CollectionDeleteResponse, error) + VolumeList(ctx context.Context, in *VolumeListRequest, opts ...grpc.CallOption) (*VolumeListResponse, error) + LookupEcVolume(ctx context.Context, in *LookupEcVolumeRequest, opts ...grpc.CallOption) (*LookupEcVolumeResponse, error) + VacuumVolume(ctx context.Context, in *VacuumVolumeRequest, opts ...grpc.CallOption) (*VacuumVolumeResponse, error) + GetMasterConfiguration(ctx context.Context, in *GetMasterConfigurationRequest, opts ...grpc.CallOption) (*GetMasterConfigurationResponse, error) + ListClusterNodes(ctx context.Context, in *ListClusterNodesRequest, opts ...grpc.CallOption) (*ListClusterNodesResponse, error) + LeaseAdminToken(ctx context.Context, in *LeaseAdminTokenRequest, opts ...grpc.CallOption) (*LeaseAdminTokenResponse, error) + ReleaseAdminToken(ctx context.Context, in *ReleaseAdminTokenRequest, opts ...grpc.CallOption) (*ReleaseAdminTokenResponse, error) +} + +type seaweedClient struct { + cc grpc.ClientConnInterface +} + +func NewSeaweedClient(cc grpc.ClientConnInterface) SeaweedClient { + return &seaweedClient{cc} +} + +func (c *seaweedClient) SendHeartbeat(ctx context.Context, opts ...grpc.CallOption) (Seaweed_SendHeartbeatClient, error) { + stream, err := c.cc.NewStream(ctx, &Seaweed_ServiceDesc.Streams[0], "/master_pb.Seaweed/SendHeartbeat", opts...) + if err != nil { + return nil, err + } + x := &seaweedSendHeartbeatClient{stream} + return x, nil +} + +type Seaweed_SendHeartbeatClient interface { + Send(*Heartbeat) error + Recv() (*HeartbeatResponse, error) + grpc.ClientStream +} + +type seaweedSendHeartbeatClient struct { + grpc.ClientStream +} + +func (x *seaweedSendHeartbeatClient) Send(m *Heartbeat) error { + return x.ClientStream.SendMsg(m) +} + +func (x *seaweedSendHeartbeatClient) Recv() (*HeartbeatResponse, error) { + m := new(HeartbeatResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedClient) KeepConnected(ctx context.Context, opts ...grpc.CallOption) (Seaweed_KeepConnectedClient, error) { + stream, err := c.cc.NewStream(ctx, &Seaweed_ServiceDesc.Streams[1], "/master_pb.Seaweed/KeepConnected", opts...) + if err != nil { + return nil, err + } + x := &seaweedKeepConnectedClient{stream} + return x, nil +} + +type Seaweed_KeepConnectedClient interface { + Send(*KeepConnectedRequest) error + Recv() (*KeepConnectedResponse, error) + grpc.ClientStream +} + +type seaweedKeepConnectedClient struct { + grpc.ClientStream +} + +func (x *seaweedKeepConnectedClient) Send(m *KeepConnectedRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *seaweedKeepConnectedClient) Recv() (*KeepConnectedResponse, error) { + m := new(KeepConnectedResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedClient) LookupVolume(ctx context.Context, in *LookupVolumeRequest, opts ...grpc.CallOption) (*LookupVolumeResponse, error) { + out := new(LookupVolumeResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/LookupVolume", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) Assign(ctx context.Context, in *AssignRequest, opts ...grpc.CallOption) (*AssignResponse, error) { + out := new(AssignResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/Assign", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) Statistics(ctx context.Context, in *StatisticsRequest, opts ...grpc.CallOption) (*StatisticsResponse, error) { + out := new(StatisticsResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/Statistics", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) CollectionList(ctx context.Context, in *CollectionListRequest, opts ...grpc.CallOption) (*CollectionListResponse, error) { + out := new(CollectionListResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/CollectionList", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) CollectionDelete(ctx context.Context, in *CollectionDeleteRequest, opts ...grpc.CallOption) (*CollectionDeleteResponse, error) { + out := new(CollectionDeleteResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/CollectionDelete", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) VolumeList(ctx context.Context, in *VolumeListRequest, opts ...grpc.CallOption) (*VolumeListResponse, error) { + out := new(VolumeListResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/VolumeList", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) LookupEcVolume(ctx context.Context, in *LookupEcVolumeRequest, opts ...grpc.CallOption) (*LookupEcVolumeResponse, error) { + out := new(LookupEcVolumeResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/LookupEcVolume", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) VacuumVolume(ctx context.Context, in *VacuumVolumeRequest, opts ...grpc.CallOption) (*VacuumVolumeResponse, error) { + out := new(VacuumVolumeResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/VacuumVolume", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) GetMasterConfiguration(ctx context.Context, in *GetMasterConfigurationRequest, opts ...grpc.CallOption) (*GetMasterConfigurationResponse, error) { + out := new(GetMasterConfigurationResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/GetMasterConfiguration", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) ListClusterNodes(ctx context.Context, in *ListClusterNodesRequest, opts ...grpc.CallOption) (*ListClusterNodesResponse, error) { + out := new(ListClusterNodesResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/ListClusterNodes", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) LeaseAdminToken(ctx context.Context, in *LeaseAdminTokenRequest, opts ...grpc.CallOption) (*LeaseAdminTokenResponse, error) { + out := new(LeaseAdminTokenResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/LeaseAdminToken", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedClient) ReleaseAdminToken(ctx context.Context, in *ReleaseAdminTokenRequest, opts ...grpc.CallOption) (*ReleaseAdminTokenResponse, error) { + out := new(ReleaseAdminTokenResponse) + err := c.cc.Invoke(ctx, "/master_pb.Seaweed/ReleaseAdminToken", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// SeaweedServer is the server API for Seaweed service. +// All implementations must embed UnimplementedSeaweedServer +// for forward compatibility +type SeaweedServer interface { + SendHeartbeat(Seaweed_SendHeartbeatServer) error + KeepConnected(Seaweed_KeepConnectedServer) error + LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) + Assign(context.Context, *AssignRequest) (*AssignResponse, error) + Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) + CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) + CollectionDelete(context.Context, *CollectionDeleteRequest) (*CollectionDeleteResponse, error) + VolumeList(context.Context, *VolumeListRequest) (*VolumeListResponse, error) + LookupEcVolume(context.Context, *LookupEcVolumeRequest) (*LookupEcVolumeResponse, error) + VacuumVolume(context.Context, *VacuumVolumeRequest) (*VacuumVolumeResponse, error) + GetMasterConfiguration(context.Context, *GetMasterConfigurationRequest) (*GetMasterConfigurationResponse, error) + ListClusterNodes(context.Context, *ListClusterNodesRequest) (*ListClusterNodesResponse, error) + LeaseAdminToken(context.Context, *LeaseAdminTokenRequest) (*LeaseAdminTokenResponse, error) + ReleaseAdminToken(context.Context, *ReleaseAdminTokenRequest) (*ReleaseAdminTokenResponse, error) + mustEmbedUnimplementedSeaweedServer() +} + +// UnimplementedSeaweedServer must be embedded to have forward compatible implementations. +type UnimplementedSeaweedServer struct { +} + +func (UnimplementedSeaweedServer) SendHeartbeat(Seaweed_SendHeartbeatServer) error { + return status.Errorf(codes.Unimplemented, "method SendHeartbeat not implemented") +} +func (UnimplementedSeaweedServer) KeepConnected(Seaweed_KeepConnectedServer) error { + return status.Errorf(codes.Unimplemented, "method KeepConnected not implemented") +} +func (UnimplementedSeaweedServer) LookupVolume(context.Context, *LookupVolumeRequest) (*LookupVolumeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupVolume not implemented") +} +func (UnimplementedSeaweedServer) Assign(context.Context, *AssignRequest) (*AssignResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Assign not implemented") +} +func (UnimplementedSeaweedServer) Statistics(context.Context, *StatisticsRequest) (*StatisticsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Statistics not implemented") +} +func (UnimplementedSeaweedServer) CollectionList(context.Context, *CollectionListRequest) (*CollectionListResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CollectionList not implemented") +} +func (UnimplementedSeaweedServer) CollectionDelete(context.Context, *CollectionDeleteRequest) (*CollectionDeleteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CollectionDelete not implemented") +} +func (UnimplementedSeaweedServer) VolumeList(context.Context, *VolumeListRequest) (*VolumeListResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeList not implemented") +} +func (UnimplementedSeaweedServer) LookupEcVolume(context.Context, *LookupEcVolumeRequest) (*LookupEcVolumeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupEcVolume not implemented") +} +func (UnimplementedSeaweedServer) VacuumVolume(context.Context, *VacuumVolumeRequest) (*VacuumVolumeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VacuumVolume not implemented") +} +func (UnimplementedSeaweedServer) GetMasterConfiguration(context.Context, *GetMasterConfigurationRequest) (*GetMasterConfigurationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetMasterConfiguration not implemented") +} +func (UnimplementedSeaweedServer) ListClusterNodes(context.Context, *ListClusterNodesRequest) (*ListClusterNodesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListClusterNodes not implemented") +} +func (UnimplementedSeaweedServer) LeaseAdminToken(context.Context, *LeaseAdminTokenRequest) (*LeaseAdminTokenResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LeaseAdminToken not implemented") +} +func (UnimplementedSeaweedServer) ReleaseAdminToken(context.Context, *ReleaseAdminTokenRequest) (*ReleaseAdminTokenResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReleaseAdminToken not implemented") +} +func (UnimplementedSeaweedServer) mustEmbedUnimplementedSeaweedServer() {} + +// UnsafeSeaweedServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to SeaweedServer will +// result in compilation errors. +type UnsafeSeaweedServer interface { + mustEmbedUnimplementedSeaweedServer() +} + +func RegisterSeaweedServer(s grpc.ServiceRegistrar, srv SeaweedServer) { + s.RegisterService(&Seaweed_ServiceDesc, srv) +} + +func _Seaweed_SendHeartbeat_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedServer).SendHeartbeat(&seaweedSendHeartbeatServer{stream}) +} + +type Seaweed_SendHeartbeatServer interface { + Send(*HeartbeatResponse) error + Recv() (*Heartbeat, error) + grpc.ServerStream +} + +type seaweedSendHeartbeatServer struct { + grpc.ServerStream +} + +func (x *seaweedSendHeartbeatServer) Send(m *HeartbeatResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *seaweedSendHeartbeatServer) Recv() (*Heartbeat, error) { + m := new(Heartbeat) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _Seaweed_KeepConnected_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedServer).KeepConnected(&seaweedKeepConnectedServer{stream}) +} + +type Seaweed_KeepConnectedServer interface { + Send(*KeepConnectedResponse) error + Recv() (*KeepConnectedRequest, error) + grpc.ServerStream +} + +type seaweedKeepConnectedServer struct { + grpc.ServerStream +} + +func (x *seaweedKeepConnectedServer) Send(m *KeepConnectedResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *seaweedKeepConnectedServer) Recv() (*KeepConnectedRequest, error) { + m := new(KeepConnectedRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _Seaweed_LookupVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupVolumeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).LookupVolume(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/LookupVolume", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).LookupVolume(ctx, req.(*LookupVolumeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_Assign_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AssignRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).Assign(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/Assign", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).Assign(ctx, req.(*AssignRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_Statistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StatisticsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).Statistics(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/Statistics", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).Statistics(ctx, req.(*StatisticsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_CollectionList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CollectionListRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).CollectionList(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/CollectionList", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).CollectionList(ctx, req.(*CollectionListRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_CollectionDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CollectionDeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).CollectionDelete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/CollectionDelete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).CollectionDelete(ctx, req.(*CollectionDeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_VolumeList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeListRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).VolumeList(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/VolumeList", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).VolumeList(ctx, req.(*VolumeListRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_LookupEcVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupEcVolumeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).LookupEcVolume(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/LookupEcVolume", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).LookupEcVolume(ctx, req.(*LookupEcVolumeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_VacuumVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VacuumVolumeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).VacuumVolume(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/VacuumVolume", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).VacuumVolume(ctx, req.(*VacuumVolumeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_GetMasterConfiguration_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetMasterConfigurationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).GetMasterConfiguration(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/GetMasterConfiguration", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).GetMasterConfiguration(ctx, req.(*GetMasterConfigurationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_ListClusterNodes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListClusterNodesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).ListClusterNodes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/ListClusterNodes", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).ListClusterNodes(ctx, req.(*ListClusterNodesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_LeaseAdminToken_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LeaseAdminTokenRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).LeaseAdminToken(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/LeaseAdminToken", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).LeaseAdminToken(ctx, req.(*LeaseAdminTokenRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Seaweed_ReleaseAdminToken_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReleaseAdminTokenRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedServer).ReleaseAdminToken(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/master_pb.Seaweed/ReleaseAdminToken", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedServer).ReleaseAdminToken(ctx, req.(*ReleaseAdminTokenRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// Seaweed_ServiceDesc is the grpc.ServiceDesc for Seaweed service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var Seaweed_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "master_pb.Seaweed", + HandlerType: (*SeaweedServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "LookupVolume", + Handler: _Seaweed_LookupVolume_Handler, + }, + { + MethodName: "Assign", + Handler: _Seaweed_Assign_Handler, + }, + { + MethodName: "Statistics", + Handler: _Seaweed_Statistics_Handler, + }, + { + MethodName: "CollectionList", + Handler: _Seaweed_CollectionList_Handler, + }, + { + MethodName: "CollectionDelete", + Handler: _Seaweed_CollectionDelete_Handler, + }, + { + MethodName: "VolumeList", + Handler: _Seaweed_VolumeList_Handler, + }, + { + MethodName: "LookupEcVolume", + Handler: _Seaweed_LookupEcVolume_Handler, + }, + { + MethodName: "VacuumVolume", + Handler: _Seaweed_VacuumVolume_Handler, + }, + { + MethodName: "GetMasterConfiguration", + Handler: _Seaweed_GetMasterConfiguration_Handler, + }, + { + MethodName: "ListClusterNodes", + Handler: _Seaweed_ListClusterNodes_Handler, + }, + { + MethodName: "LeaseAdminToken", + Handler: _Seaweed_LeaseAdminToken_Handler, + }, + { + MethodName: "ReleaseAdminToken", + Handler: _Seaweed_ReleaseAdminToken_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "SendHeartbeat", + Handler: _Seaweed_SendHeartbeat_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "KeepConnected", + Handler: _Seaweed_KeepConnected_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "master.proto", +} diff --git a/weed/pb/messaging_pb/messaging.pb.go b/weed/pb/messaging_pb/messaging.pb.go index 591406347..5b9ca1ee3 100644 --- a/weed/pb/messaging_pb/messaging.pb.go +++ b/weed/pb/messaging_pb/messaging.pb.go @@ -1,17 +1,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.25.0 -// protoc v3.12.3 +// protoc-gen-go v1.26.0 +// protoc v3.17.3 // source: messaging.proto package messaging_pb import ( - context "context" - proto "github.com/golang/protobuf/proto" - grpc "google.golang.org/grpc" - codes "google.golang.org/grpc/codes" - status "google.golang.org/grpc/status" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -25,10 +20,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// This is a compile-time assertion that a sufficiently up-to-date version -// of the legacy proto package is being used. -const _ = proto.ProtoPackageIsVersion4 - type SubscriberMessage_InitMessage_StartPosition int32 const ( @@ -1726,328 +1717,3 @@ func file_messaging_proto_init() { file_messaging_proto_goTypes = nil file_messaging_proto_depIdxs = nil } - -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConnInterface - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion6 - -// SeaweedMessagingClient is the client API for SeaweedMessaging service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type SeaweedMessagingClient interface { - Subscribe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) - Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) - DeleteTopic(ctx context.Context, in *DeleteTopicRequest, opts ...grpc.CallOption) (*DeleteTopicResponse, error) - ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) - GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error) - FindBroker(ctx context.Context, in *FindBrokerRequest, opts ...grpc.CallOption) (*FindBrokerResponse, error) -} - -type seaweedMessagingClient struct { - cc grpc.ClientConnInterface -} - -func NewSeaweedMessagingClient(cc grpc.ClientConnInterface) SeaweedMessagingClient { - return &seaweedMessagingClient{cc} -} - -func (c *seaweedMessagingClient) Subscribe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) { - stream, err := c.cc.NewStream(ctx, &_SeaweedMessaging_serviceDesc.Streams[0], "/messaging_pb.SeaweedMessaging/Subscribe", opts...) - if err != nil { - return nil, err - } - x := &seaweedMessagingSubscribeClient{stream} - return x, nil -} - -type SeaweedMessaging_SubscribeClient interface { - Send(*SubscriberMessage) error - Recv() (*BrokerMessage, error) - grpc.ClientStream -} - -type seaweedMessagingSubscribeClient struct { - grpc.ClientStream -} - -func (x *seaweedMessagingSubscribeClient) Send(m *SubscriberMessage) error { - return x.ClientStream.SendMsg(m) -} - -func (x *seaweedMessagingSubscribeClient) Recv() (*BrokerMessage, error) { - m := new(BrokerMessage) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedMessagingClient) Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) { - stream, err := c.cc.NewStream(ctx, &_SeaweedMessaging_serviceDesc.Streams[1], "/messaging_pb.SeaweedMessaging/Publish", opts...) - if err != nil { - return nil, err - } - x := &seaweedMessagingPublishClient{stream} - return x, nil -} - -type SeaweedMessaging_PublishClient interface { - Send(*PublishRequest) error - Recv() (*PublishResponse, error) - grpc.ClientStream -} - -type seaweedMessagingPublishClient struct { - grpc.ClientStream -} - -func (x *seaweedMessagingPublishClient) Send(m *PublishRequest) error { - return x.ClientStream.SendMsg(m) -} - -func (x *seaweedMessagingPublishClient) Recv() (*PublishResponse, error) { - m := new(PublishResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *seaweedMessagingClient) DeleteTopic(ctx context.Context, in *DeleteTopicRequest, opts ...grpc.CallOption) (*DeleteTopicResponse, error) { - out := new(DeleteTopicResponse) - err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/DeleteTopic", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedMessagingClient) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) { - out := new(ConfigureTopicResponse) - err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/ConfigureTopic", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedMessagingClient) GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error) { - out := new(GetTopicConfigurationResponse) - err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/GetTopicConfiguration", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedMessagingClient) FindBroker(ctx context.Context, in *FindBrokerRequest, opts ...grpc.CallOption) (*FindBrokerResponse, error) { - out := new(FindBrokerResponse) - err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/FindBroker", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -// SeaweedMessagingServer is the server API for SeaweedMessaging service. -type SeaweedMessagingServer interface { - Subscribe(SeaweedMessaging_SubscribeServer) error - Publish(SeaweedMessaging_PublishServer) error - DeleteTopic(context.Context, *DeleteTopicRequest) (*DeleteTopicResponse, error) - ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) - GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error) - FindBroker(context.Context, *FindBrokerRequest) (*FindBrokerResponse, error) -} - -// UnimplementedSeaweedMessagingServer can be embedded to have forward compatible implementations. -type UnimplementedSeaweedMessagingServer struct { -} - -func (*UnimplementedSeaweedMessagingServer) Subscribe(SeaweedMessaging_SubscribeServer) error { - return status.Errorf(codes.Unimplemented, "method Subscribe not implemented") -} -func (*UnimplementedSeaweedMessagingServer) Publish(SeaweedMessaging_PublishServer) error { - return status.Errorf(codes.Unimplemented, "method Publish not implemented") -} -func (*UnimplementedSeaweedMessagingServer) DeleteTopic(context.Context, *DeleteTopicRequest) (*DeleteTopicResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DeleteTopic not implemented") -} -func (*UnimplementedSeaweedMessagingServer) ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ConfigureTopic not implemented") -} -func (*UnimplementedSeaweedMessagingServer) GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method GetTopicConfiguration not implemented") -} -func (*UnimplementedSeaweedMessagingServer) FindBroker(context.Context, *FindBrokerRequest) (*FindBrokerResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method FindBroker not implemented") -} - -func RegisterSeaweedMessagingServer(s *grpc.Server, srv SeaweedMessagingServer) { - s.RegisterService(&_SeaweedMessaging_serviceDesc, srv) -} - -func _SeaweedMessaging_Subscribe_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(SeaweedMessagingServer).Subscribe(&seaweedMessagingSubscribeServer{stream}) -} - -type SeaweedMessaging_SubscribeServer interface { - Send(*BrokerMessage) error - Recv() (*SubscriberMessage, error) - grpc.ServerStream -} - -type seaweedMessagingSubscribeServer struct { - grpc.ServerStream -} - -func (x *seaweedMessagingSubscribeServer) Send(m *BrokerMessage) error { - return x.ServerStream.SendMsg(m) -} - -func (x *seaweedMessagingSubscribeServer) Recv() (*SubscriberMessage, error) { - m := new(SubscriberMessage) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func _SeaweedMessaging_Publish_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(SeaweedMessagingServer).Publish(&seaweedMessagingPublishServer{stream}) -} - -type SeaweedMessaging_PublishServer interface { - Send(*PublishResponse) error - Recv() (*PublishRequest, error) - grpc.ServerStream -} - -type seaweedMessagingPublishServer struct { - grpc.ServerStream -} - -func (x *seaweedMessagingPublishServer) Send(m *PublishResponse) error { - return x.ServerStream.SendMsg(m) -} - -func (x *seaweedMessagingPublishServer) Recv() (*PublishRequest, error) { - m := new(PublishRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func _SeaweedMessaging_DeleteTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DeleteTopicRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).DeleteTopic(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/messaging_pb.SeaweedMessaging/DeleteTopic", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).DeleteTopic(ctx, req.(*DeleteTopicRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedMessaging_ConfigureTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ConfigureTopicRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/messaging_pb.SeaweedMessaging/ConfigureTopic", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, req.(*ConfigureTopicRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedMessaging_GetTopicConfiguration_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(GetTopicConfigurationRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).GetTopicConfiguration(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/messaging_pb.SeaweedMessaging/GetTopicConfiguration", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).GetTopicConfiguration(ctx, req.(*GetTopicConfigurationRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedMessaging_FindBroker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(FindBrokerRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).FindBroker(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/messaging_pb.SeaweedMessaging/FindBroker", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).FindBroker(ctx, req.(*FindBrokerRequest)) - } - return interceptor(ctx, in, info, handler) -} - -var _SeaweedMessaging_serviceDesc = grpc.ServiceDesc{ - ServiceName: "messaging_pb.SeaweedMessaging", - HandlerType: (*SeaweedMessagingServer)(nil), - Methods: []grpc.MethodDesc{ - { - MethodName: "DeleteTopic", - Handler: _SeaweedMessaging_DeleteTopic_Handler, - }, - { - MethodName: "ConfigureTopic", - Handler: _SeaweedMessaging_ConfigureTopic_Handler, - }, - { - MethodName: "GetTopicConfiguration", - Handler: _SeaweedMessaging_GetTopicConfiguration_Handler, - }, - { - MethodName: "FindBroker", - Handler: _SeaweedMessaging_FindBroker_Handler, - }, - }, - Streams: []grpc.StreamDesc{ - { - StreamName: "Subscribe", - Handler: _SeaweedMessaging_Subscribe_Handler, - ServerStreams: true, - ClientStreams: true, - }, - { - StreamName: "Publish", - Handler: _SeaweedMessaging_Publish_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, - Metadata: "messaging.proto", -} diff --git a/weed/pb/messaging_pb/messaging_grpc.pb.go b/weed/pb/messaging_pb/messaging_grpc.pb.go new file mode 100644 index 000000000..234cffa95 --- /dev/null +++ b/weed/pb/messaging_pb/messaging_grpc.pb.go @@ -0,0 +1,346 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. + +package messaging_pb + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// SeaweedMessagingClient is the client API for SeaweedMessaging service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type SeaweedMessagingClient interface { + Subscribe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) + Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) + DeleteTopic(ctx context.Context, in *DeleteTopicRequest, opts ...grpc.CallOption) (*DeleteTopicResponse, error) + ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) + GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error) + FindBroker(ctx context.Context, in *FindBrokerRequest, opts ...grpc.CallOption) (*FindBrokerResponse, error) +} + +type seaweedMessagingClient struct { + cc grpc.ClientConnInterface +} + +func NewSeaweedMessagingClient(cc grpc.ClientConnInterface) SeaweedMessagingClient { + return &seaweedMessagingClient{cc} +} + +func (c *seaweedMessagingClient) Subscribe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[0], "/messaging_pb.SeaweedMessaging/Subscribe", opts...) + if err != nil { + return nil, err + } + x := &seaweedMessagingSubscribeClient{stream} + return x, nil +} + +type SeaweedMessaging_SubscribeClient interface { + Send(*SubscriberMessage) error + Recv() (*BrokerMessage, error) + grpc.ClientStream +} + +type seaweedMessagingSubscribeClient struct { + grpc.ClientStream +} + +func (x *seaweedMessagingSubscribeClient) Send(m *SubscriberMessage) error { + return x.ClientStream.SendMsg(m) +} + +func (x *seaweedMessagingSubscribeClient) Recv() (*BrokerMessage, error) { + m := new(BrokerMessage) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedMessagingClient) Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[1], "/messaging_pb.SeaweedMessaging/Publish", opts...) + if err != nil { + return nil, err + } + x := &seaweedMessagingPublishClient{stream} + return x, nil +} + +type SeaweedMessaging_PublishClient interface { + Send(*PublishRequest) error + Recv() (*PublishResponse, error) + grpc.ClientStream +} + +type seaweedMessagingPublishClient struct { + grpc.ClientStream +} + +func (x *seaweedMessagingPublishClient) Send(m *PublishRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *seaweedMessagingPublishClient) Recv() (*PublishResponse, error) { + m := new(PublishResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *seaweedMessagingClient) DeleteTopic(ctx context.Context, in *DeleteTopicRequest, opts ...grpc.CallOption) (*DeleteTopicResponse, error) { + out := new(DeleteTopicResponse) + err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/DeleteTopic", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingClient) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) { + out := new(ConfigureTopicResponse) + err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/ConfigureTopic", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingClient) GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error) { + out := new(GetTopicConfigurationResponse) + err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/GetTopicConfiguration", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingClient) FindBroker(ctx context.Context, in *FindBrokerRequest, opts ...grpc.CallOption) (*FindBrokerResponse, error) { + out := new(FindBrokerResponse) + err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/FindBroker", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// SeaweedMessagingServer is the server API for SeaweedMessaging service. +// All implementations must embed UnimplementedSeaweedMessagingServer +// for forward compatibility +type SeaweedMessagingServer interface { + Subscribe(SeaweedMessaging_SubscribeServer) error + Publish(SeaweedMessaging_PublishServer) error + DeleteTopic(context.Context, *DeleteTopicRequest) (*DeleteTopicResponse, error) + ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) + GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error) + FindBroker(context.Context, *FindBrokerRequest) (*FindBrokerResponse, error) + mustEmbedUnimplementedSeaweedMessagingServer() +} + +// UnimplementedSeaweedMessagingServer must be embedded to have forward compatible implementations. +type UnimplementedSeaweedMessagingServer struct { +} + +func (UnimplementedSeaweedMessagingServer) Subscribe(SeaweedMessaging_SubscribeServer) error { + return status.Errorf(codes.Unimplemented, "method Subscribe not implemented") +} +func (UnimplementedSeaweedMessagingServer) Publish(SeaweedMessaging_PublishServer) error { + return status.Errorf(codes.Unimplemented, "method Publish not implemented") +} +func (UnimplementedSeaweedMessagingServer) DeleteTopic(context.Context, *DeleteTopicRequest) (*DeleteTopicResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteTopic not implemented") +} +func (UnimplementedSeaweedMessagingServer) ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ConfigureTopic not implemented") +} +func (UnimplementedSeaweedMessagingServer) GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetTopicConfiguration not implemented") +} +func (UnimplementedSeaweedMessagingServer) FindBroker(context.Context, *FindBrokerRequest) (*FindBrokerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method FindBroker not implemented") +} +func (UnimplementedSeaweedMessagingServer) mustEmbedUnimplementedSeaweedMessagingServer() {} + +// UnsafeSeaweedMessagingServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to SeaweedMessagingServer will +// result in compilation errors. +type UnsafeSeaweedMessagingServer interface { + mustEmbedUnimplementedSeaweedMessagingServer() +} + +func RegisterSeaweedMessagingServer(s grpc.ServiceRegistrar, srv SeaweedMessagingServer) { + s.RegisterService(&SeaweedMessaging_ServiceDesc, srv) +} + +func _SeaweedMessaging_Subscribe_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedMessagingServer).Subscribe(&seaweedMessagingSubscribeServer{stream}) +} + +type SeaweedMessaging_SubscribeServer interface { + Send(*BrokerMessage) error + Recv() (*SubscriberMessage, error) + grpc.ServerStream +} + +type seaweedMessagingSubscribeServer struct { + grpc.ServerStream +} + +func (x *seaweedMessagingSubscribeServer) Send(m *BrokerMessage) error { + return x.ServerStream.SendMsg(m) +} + +func (x *seaweedMessagingSubscribeServer) Recv() (*SubscriberMessage, error) { + m := new(SubscriberMessage) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _SeaweedMessaging_Publish_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedMessagingServer).Publish(&seaweedMessagingPublishServer{stream}) +} + +type SeaweedMessaging_PublishServer interface { + Send(*PublishResponse) error + Recv() (*PublishRequest, error) + grpc.ServerStream +} + +type seaweedMessagingPublishServer struct { + grpc.ServerStream +} + +func (x *seaweedMessagingPublishServer) Send(m *PublishResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *seaweedMessagingPublishServer) Recv() (*PublishRequest, error) { + m := new(PublishRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _SeaweedMessaging_DeleteTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteTopicRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).DeleteTopic(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/messaging_pb.SeaweedMessaging/DeleteTopic", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).DeleteTopic(ctx, req.(*DeleteTopicRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessaging_ConfigureTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ConfigureTopicRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/messaging_pb.SeaweedMessaging/ConfigureTopic", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, req.(*ConfigureTopicRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessaging_GetTopicConfiguration_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetTopicConfigurationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).GetTopicConfiguration(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/messaging_pb.SeaweedMessaging/GetTopicConfiguration", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).GetTopicConfiguration(ctx, req.(*GetTopicConfigurationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessaging_FindBroker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FindBrokerRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).FindBroker(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/messaging_pb.SeaweedMessaging/FindBroker", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).FindBroker(ctx, req.(*FindBrokerRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// SeaweedMessaging_ServiceDesc is the grpc.ServiceDesc for SeaweedMessaging service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "messaging_pb.SeaweedMessaging", + HandlerType: (*SeaweedMessagingServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "DeleteTopic", + Handler: _SeaweedMessaging_DeleteTopic_Handler, + }, + { + MethodName: "ConfigureTopic", + Handler: _SeaweedMessaging_ConfigureTopic_Handler, + }, + { + MethodName: "GetTopicConfiguration", + Handler: _SeaweedMessaging_GetTopicConfiguration_Handler, + }, + { + MethodName: "FindBroker", + Handler: _SeaweedMessaging_FindBroker_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "Subscribe", + Handler: _SeaweedMessaging_Subscribe_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "Publish", + Handler: _SeaweedMessaging_Publish_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "messaging.proto", +} diff --git a/weed/pb/remote_pb/remote.pb.go b/weed/pb/remote_pb/remote.pb.go index fef9556fe..8f1bd9b5f 100644 --- a/weed/pb/remote_pb/remote.pb.go +++ b/weed/pb/remote_pb/remote.pb.go @@ -1,13 +1,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.25.0 -// protoc v3.12.3 +// protoc-gen-go v1.26.0 +// protoc v3.17.3 // source: remote.proto package remote_pb import ( - proto "github.com/golang/protobuf/proto" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -21,10 +20,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// This is a compile-time assertion that a sufficiently up-to-date version -// of the legacy proto package is being used. -const _ = proto.ProtoPackageIsVersion4 - ///////////////////////// // Remote Storage related ///////////////////////// diff --git a/weed/pb/volume_server_pb/volume_server.pb.go b/weed/pb/volume_server_pb/volume_server.pb.go index f73596c46..5ff6aa459 100644 --- a/weed/pb/volume_server_pb/volume_server.pb.go +++ b/weed/pb/volume_server_pb/volume_server.pb.go @@ -1,18 +1,13 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.25.0 -// protoc v3.12.3 +// protoc-gen-go v1.26.0 +// protoc v3.17.3 // source: volume_server.proto package volume_server_pb import ( - context "context" remote_pb "github.com/chrislusf/seaweedfs/weed/pb/remote_pb" - proto "github.com/golang/protobuf/proto" - grpc "google.golang.org/grpc" - codes "google.golang.org/grpc/codes" - status "google.golang.org/grpc/status" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -26,10 +21,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// This is a compile-time assertion that a sufficiently up-to-date version -// of the legacy proto package is being used. -const _ = proto.ProtoPackageIsVersion4 - type BatchDeleteRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -7742,1770 +7733,3 @@ func file_volume_server_proto_init() { file_volume_server_proto_goTypes = nil file_volume_server_proto_depIdxs = nil } - -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConnInterface - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion6 - -// VolumeServerClient is the client API for VolumeServer service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type VolumeServerClient interface { - //Experts only: takes multiple fid parameters. This function does not propagate deletes to replicas. - BatchDelete(ctx context.Context, in *BatchDeleteRequest, opts ...grpc.CallOption) (*BatchDeleteResponse, error) - VacuumVolumeCheck(ctx context.Context, in *VacuumVolumeCheckRequest, opts ...grpc.CallOption) (*VacuumVolumeCheckResponse, error) - VacuumVolumeCompact(ctx context.Context, in *VacuumVolumeCompactRequest, opts ...grpc.CallOption) (VolumeServer_VacuumVolumeCompactClient, error) - VacuumVolumeCommit(ctx context.Context, in *VacuumVolumeCommitRequest, opts ...grpc.CallOption) (*VacuumVolumeCommitResponse, error) - VacuumVolumeCleanup(ctx context.Context, in *VacuumVolumeCleanupRequest, opts ...grpc.CallOption) (*VacuumVolumeCleanupResponse, error) - DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) - AllocateVolume(ctx context.Context, in *AllocateVolumeRequest, opts ...grpc.CallOption) (*AllocateVolumeResponse, error) - VolumeSyncStatus(ctx context.Context, in *VolumeSyncStatusRequest, opts ...grpc.CallOption) (*VolumeSyncStatusResponse, error) - VolumeIncrementalCopy(ctx context.Context, in *VolumeIncrementalCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeIncrementalCopyClient, error) - VolumeMount(ctx context.Context, in *VolumeMountRequest, opts ...grpc.CallOption) (*VolumeMountResponse, error) - VolumeUnmount(ctx context.Context, in *VolumeUnmountRequest, opts ...grpc.CallOption) (*VolumeUnmountResponse, error) - VolumeDelete(ctx context.Context, in *VolumeDeleteRequest, opts ...grpc.CallOption) (*VolumeDeleteResponse, error) - VolumeMarkReadonly(ctx context.Context, in *VolumeMarkReadonlyRequest, opts ...grpc.CallOption) (*VolumeMarkReadonlyResponse, error) - VolumeMarkWritable(ctx context.Context, in *VolumeMarkWritableRequest, opts ...grpc.CallOption) (*VolumeMarkWritableResponse, error) - VolumeConfigure(ctx context.Context, in *VolumeConfigureRequest, opts ...grpc.CallOption) (*VolumeConfigureResponse, error) - VolumeStatus(ctx context.Context, in *VolumeStatusRequest, opts ...grpc.CallOption) (*VolumeStatusResponse, error) - // copy the .idx .dat files, and mount this volume - VolumeCopy(ctx context.Context, in *VolumeCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeCopyClient, error) - ReadVolumeFileStatus(ctx context.Context, in *ReadVolumeFileStatusRequest, opts ...grpc.CallOption) (*ReadVolumeFileStatusResponse, error) - CopyFile(ctx context.Context, in *CopyFileRequest, opts ...grpc.CallOption) (VolumeServer_CopyFileClient, error) - ReadNeedleBlob(ctx context.Context, in *ReadNeedleBlobRequest, opts ...grpc.CallOption) (*ReadNeedleBlobResponse, error) - WriteNeedleBlob(ctx context.Context, in *WriteNeedleBlobRequest, opts ...grpc.CallOption) (*WriteNeedleBlobResponse, error) - ReadAllNeedles(ctx context.Context, in *ReadAllNeedlesRequest, opts ...grpc.CallOption) (VolumeServer_ReadAllNeedlesClient, error) - VolumeTailSender(ctx context.Context, in *VolumeTailSenderRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTailSenderClient, error) - VolumeTailReceiver(ctx context.Context, in *VolumeTailReceiverRequest, opts ...grpc.CallOption) (*VolumeTailReceiverResponse, error) - // erasure coding - VolumeEcShardsGenerate(ctx context.Context, in *VolumeEcShardsGenerateRequest, opts ...grpc.CallOption) (*VolumeEcShardsGenerateResponse, error) - VolumeEcShardsRebuild(ctx context.Context, in *VolumeEcShardsRebuildRequest, opts ...grpc.CallOption) (*VolumeEcShardsRebuildResponse, error) - VolumeEcShardsCopy(ctx context.Context, in *VolumeEcShardsCopyRequest, opts ...grpc.CallOption) (*VolumeEcShardsCopyResponse, error) - VolumeEcShardsDelete(ctx context.Context, in *VolumeEcShardsDeleteRequest, opts ...grpc.CallOption) (*VolumeEcShardsDeleteResponse, error) - VolumeEcShardsMount(ctx context.Context, in *VolumeEcShardsMountRequest, opts ...grpc.CallOption) (*VolumeEcShardsMountResponse, error) - VolumeEcShardsUnmount(ctx context.Context, in *VolumeEcShardsUnmountRequest, opts ...grpc.CallOption) (*VolumeEcShardsUnmountResponse, error) - VolumeEcShardRead(ctx context.Context, in *VolumeEcShardReadRequest, opts ...grpc.CallOption) (VolumeServer_VolumeEcShardReadClient, error) - VolumeEcBlobDelete(ctx context.Context, in *VolumeEcBlobDeleteRequest, opts ...grpc.CallOption) (*VolumeEcBlobDeleteResponse, error) - VolumeEcShardsToVolume(ctx context.Context, in *VolumeEcShardsToVolumeRequest, opts ...grpc.CallOption) (*VolumeEcShardsToVolumeResponse, error) - // tiered storage - VolumeTierMoveDatToRemote(ctx context.Context, in *VolumeTierMoveDatToRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatToRemoteClient, error) - VolumeTierMoveDatFromRemote(ctx context.Context, in *VolumeTierMoveDatFromRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatFromRemoteClient, error) - VolumeServerStatus(ctx context.Context, in *VolumeServerStatusRequest, opts ...grpc.CallOption) (*VolumeServerStatusResponse, error) - VolumeServerLeave(ctx context.Context, in *VolumeServerLeaveRequest, opts ...grpc.CallOption) (*VolumeServerLeaveResponse, error) - // remote storage - FetchAndWriteNeedle(ctx context.Context, in *FetchAndWriteNeedleRequest, opts ...grpc.CallOption) (*FetchAndWriteNeedleResponse, error) - // <experimental> query - Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error) - VolumeNeedleStatus(ctx context.Context, in *VolumeNeedleStatusRequest, opts ...grpc.CallOption) (*VolumeNeedleStatusResponse, error) -} - -type volumeServerClient struct { - cc grpc.ClientConnInterface -} - -func NewVolumeServerClient(cc grpc.ClientConnInterface) VolumeServerClient { - return &volumeServerClient{cc} -} - -func (c *volumeServerClient) BatchDelete(ctx context.Context, in *BatchDeleteRequest, opts ...grpc.CallOption) (*BatchDeleteResponse, error) { - out := new(BatchDeleteResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/BatchDelete", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VacuumVolumeCheck(ctx context.Context, in *VacuumVolumeCheckRequest, opts ...grpc.CallOption) (*VacuumVolumeCheckResponse, error) { - out := new(VacuumVolumeCheckResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VacuumVolumeCheck", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VacuumVolumeCompact(ctx context.Context, in *VacuumVolumeCompactRequest, opts ...grpc.CallOption) (VolumeServer_VacuumVolumeCompactClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[0], "/volume_server_pb.VolumeServer/VacuumVolumeCompact", opts...) - if err != nil { - return nil, err - } - x := &volumeServerVacuumVolumeCompactClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_VacuumVolumeCompactClient interface { - Recv() (*VacuumVolumeCompactResponse, error) - grpc.ClientStream -} - -type volumeServerVacuumVolumeCompactClient struct { - grpc.ClientStream -} - -func (x *volumeServerVacuumVolumeCompactClient) Recv() (*VacuumVolumeCompactResponse, error) { - m := new(VacuumVolumeCompactResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VacuumVolumeCommit(ctx context.Context, in *VacuumVolumeCommitRequest, opts ...grpc.CallOption) (*VacuumVolumeCommitResponse, error) { - out := new(VacuumVolumeCommitResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VacuumVolumeCommit", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VacuumVolumeCleanup(ctx context.Context, in *VacuumVolumeCleanupRequest, opts ...grpc.CallOption) (*VacuumVolumeCleanupResponse, error) { - out := new(VacuumVolumeCleanupResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VacuumVolumeCleanup", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) { - out := new(DeleteCollectionResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/DeleteCollection", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) AllocateVolume(ctx context.Context, in *AllocateVolumeRequest, opts ...grpc.CallOption) (*AllocateVolumeResponse, error) { - out := new(AllocateVolumeResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/AllocateVolume", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeSyncStatus(ctx context.Context, in *VolumeSyncStatusRequest, opts ...grpc.CallOption) (*VolumeSyncStatusResponse, error) { - out := new(VolumeSyncStatusResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeSyncStatus", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeIncrementalCopy(ctx context.Context, in *VolumeIncrementalCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeIncrementalCopyClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[1], "/volume_server_pb.VolumeServer/VolumeIncrementalCopy", opts...) - if err != nil { - return nil, err - } - x := &volumeServerVolumeIncrementalCopyClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_VolumeIncrementalCopyClient interface { - Recv() (*VolumeIncrementalCopyResponse, error) - grpc.ClientStream -} - -type volumeServerVolumeIncrementalCopyClient struct { - grpc.ClientStream -} - -func (x *volumeServerVolumeIncrementalCopyClient) Recv() (*VolumeIncrementalCopyResponse, error) { - m := new(VolumeIncrementalCopyResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VolumeMount(ctx context.Context, in *VolumeMountRequest, opts ...grpc.CallOption) (*VolumeMountResponse, error) { - out := new(VolumeMountResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeMount", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeUnmount(ctx context.Context, in *VolumeUnmountRequest, opts ...grpc.CallOption) (*VolumeUnmountResponse, error) { - out := new(VolumeUnmountResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeUnmount", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeDelete(ctx context.Context, in *VolumeDeleteRequest, opts ...grpc.CallOption) (*VolumeDeleteResponse, error) { - out := new(VolumeDeleteResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeDelete", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeMarkReadonly(ctx context.Context, in *VolumeMarkReadonlyRequest, opts ...grpc.CallOption) (*VolumeMarkReadonlyResponse, error) { - out := new(VolumeMarkReadonlyResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeMarkReadonly", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeMarkWritable(ctx context.Context, in *VolumeMarkWritableRequest, opts ...grpc.CallOption) (*VolumeMarkWritableResponse, error) { - out := new(VolumeMarkWritableResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeMarkWritable", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeConfigure(ctx context.Context, in *VolumeConfigureRequest, opts ...grpc.CallOption) (*VolumeConfigureResponse, error) { - out := new(VolumeConfigureResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeConfigure", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeStatus(ctx context.Context, in *VolumeStatusRequest, opts ...grpc.CallOption) (*VolumeStatusResponse, error) { - out := new(VolumeStatusResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeStatus", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeCopy(ctx context.Context, in *VolumeCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeCopyClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[2], "/volume_server_pb.VolumeServer/VolumeCopy", opts...) - if err != nil { - return nil, err - } - x := &volumeServerVolumeCopyClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_VolumeCopyClient interface { - Recv() (*VolumeCopyResponse, error) - grpc.ClientStream -} - -type volumeServerVolumeCopyClient struct { - grpc.ClientStream -} - -func (x *volumeServerVolumeCopyClient) Recv() (*VolumeCopyResponse, error) { - m := new(VolumeCopyResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) ReadVolumeFileStatus(ctx context.Context, in *ReadVolumeFileStatusRequest, opts ...grpc.CallOption) (*ReadVolumeFileStatusResponse, error) { - out := new(ReadVolumeFileStatusResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/ReadVolumeFileStatus", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) CopyFile(ctx context.Context, in *CopyFileRequest, opts ...grpc.CallOption) (VolumeServer_CopyFileClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[3], "/volume_server_pb.VolumeServer/CopyFile", opts...) - if err != nil { - return nil, err - } - x := &volumeServerCopyFileClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_CopyFileClient interface { - Recv() (*CopyFileResponse, error) - grpc.ClientStream -} - -type volumeServerCopyFileClient struct { - grpc.ClientStream -} - -func (x *volumeServerCopyFileClient) Recv() (*CopyFileResponse, error) { - m := new(CopyFileResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) ReadNeedleBlob(ctx context.Context, in *ReadNeedleBlobRequest, opts ...grpc.CallOption) (*ReadNeedleBlobResponse, error) { - out := new(ReadNeedleBlobResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/ReadNeedleBlob", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) WriteNeedleBlob(ctx context.Context, in *WriteNeedleBlobRequest, opts ...grpc.CallOption) (*WriteNeedleBlobResponse, error) { - out := new(WriteNeedleBlobResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/WriteNeedleBlob", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) ReadAllNeedles(ctx context.Context, in *ReadAllNeedlesRequest, opts ...grpc.CallOption) (VolumeServer_ReadAllNeedlesClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[4], "/volume_server_pb.VolumeServer/ReadAllNeedles", opts...) - if err != nil { - return nil, err - } - x := &volumeServerReadAllNeedlesClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_ReadAllNeedlesClient interface { - Recv() (*ReadAllNeedlesResponse, error) - grpc.ClientStream -} - -type volumeServerReadAllNeedlesClient struct { - grpc.ClientStream -} - -func (x *volumeServerReadAllNeedlesClient) Recv() (*ReadAllNeedlesResponse, error) { - m := new(ReadAllNeedlesResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VolumeTailSender(ctx context.Context, in *VolumeTailSenderRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTailSenderClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[5], "/volume_server_pb.VolumeServer/VolumeTailSender", opts...) - if err != nil { - return nil, err - } - x := &volumeServerVolumeTailSenderClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_VolumeTailSenderClient interface { - Recv() (*VolumeTailSenderResponse, error) - grpc.ClientStream -} - -type volumeServerVolumeTailSenderClient struct { - grpc.ClientStream -} - -func (x *volumeServerVolumeTailSenderClient) Recv() (*VolumeTailSenderResponse, error) { - m := new(VolumeTailSenderResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VolumeTailReceiver(ctx context.Context, in *VolumeTailReceiverRequest, opts ...grpc.CallOption) (*VolumeTailReceiverResponse, error) { - out := new(VolumeTailReceiverResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeTailReceiver", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardsGenerate(ctx context.Context, in *VolumeEcShardsGenerateRequest, opts ...grpc.CallOption) (*VolumeEcShardsGenerateResponse, error) { - out := new(VolumeEcShardsGenerateResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsGenerate", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardsRebuild(ctx context.Context, in *VolumeEcShardsRebuildRequest, opts ...grpc.CallOption) (*VolumeEcShardsRebuildResponse, error) { - out := new(VolumeEcShardsRebuildResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsRebuild", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardsCopy(ctx context.Context, in *VolumeEcShardsCopyRequest, opts ...grpc.CallOption) (*VolumeEcShardsCopyResponse, error) { - out := new(VolumeEcShardsCopyResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsCopy", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardsDelete(ctx context.Context, in *VolumeEcShardsDeleteRequest, opts ...grpc.CallOption) (*VolumeEcShardsDeleteResponse, error) { - out := new(VolumeEcShardsDeleteResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsDelete", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardsMount(ctx context.Context, in *VolumeEcShardsMountRequest, opts ...grpc.CallOption) (*VolumeEcShardsMountResponse, error) { - out := new(VolumeEcShardsMountResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsMount", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardsUnmount(ctx context.Context, in *VolumeEcShardsUnmountRequest, opts ...grpc.CallOption) (*VolumeEcShardsUnmountResponse, error) { - out := new(VolumeEcShardsUnmountResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsUnmount", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardRead(ctx context.Context, in *VolumeEcShardReadRequest, opts ...grpc.CallOption) (VolumeServer_VolumeEcShardReadClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[6], "/volume_server_pb.VolumeServer/VolumeEcShardRead", opts...) - if err != nil { - return nil, err - } - x := &volumeServerVolumeEcShardReadClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_VolumeEcShardReadClient interface { - Recv() (*VolumeEcShardReadResponse, error) - grpc.ClientStream -} - -type volumeServerVolumeEcShardReadClient struct { - grpc.ClientStream -} - -func (x *volumeServerVolumeEcShardReadClient) Recv() (*VolumeEcShardReadResponse, error) { - m := new(VolumeEcShardReadResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VolumeEcBlobDelete(ctx context.Context, in *VolumeEcBlobDeleteRequest, opts ...grpc.CallOption) (*VolumeEcBlobDeleteResponse, error) { - out := new(VolumeEcBlobDeleteResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcBlobDelete", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeEcShardsToVolume(ctx context.Context, in *VolumeEcShardsToVolumeRequest, opts ...grpc.CallOption) (*VolumeEcShardsToVolumeResponse, error) { - out := new(VolumeEcShardsToVolumeResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsToVolume", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeTierMoveDatToRemote(ctx context.Context, in *VolumeTierMoveDatToRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatToRemoteClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[7], "/volume_server_pb.VolumeServer/VolumeTierMoveDatToRemote", opts...) - if err != nil { - return nil, err - } - x := &volumeServerVolumeTierMoveDatToRemoteClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_VolumeTierMoveDatToRemoteClient interface { - Recv() (*VolumeTierMoveDatToRemoteResponse, error) - grpc.ClientStream -} - -type volumeServerVolumeTierMoveDatToRemoteClient struct { - grpc.ClientStream -} - -func (x *volumeServerVolumeTierMoveDatToRemoteClient) Recv() (*VolumeTierMoveDatToRemoteResponse, error) { - m := new(VolumeTierMoveDatToRemoteResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VolumeTierMoveDatFromRemote(ctx context.Context, in *VolumeTierMoveDatFromRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatFromRemoteClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[8], "/volume_server_pb.VolumeServer/VolumeTierMoveDatFromRemote", opts...) - if err != nil { - return nil, err - } - x := &volumeServerVolumeTierMoveDatFromRemoteClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_VolumeTierMoveDatFromRemoteClient interface { - Recv() (*VolumeTierMoveDatFromRemoteResponse, error) - grpc.ClientStream -} - -type volumeServerVolumeTierMoveDatFromRemoteClient struct { - grpc.ClientStream -} - -func (x *volumeServerVolumeTierMoveDatFromRemoteClient) Recv() (*VolumeTierMoveDatFromRemoteResponse, error) { - m := new(VolumeTierMoveDatFromRemoteResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VolumeServerStatus(ctx context.Context, in *VolumeServerStatusRequest, opts ...grpc.CallOption) (*VolumeServerStatusResponse, error) { - out := new(VolumeServerStatusResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeServerStatus", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) VolumeServerLeave(ctx context.Context, in *VolumeServerLeaveRequest, opts ...grpc.CallOption) (*VolumeServerLeaveResponse, error) { - out := new(VolumeServerLeaveResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeServerLeave", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) FetchAndWriteNeedle(ctx context.Context, in *FetchAndWriteNeedleRequest, opts ...grpc.CallOption) (*FetchAndWriteNeedleResponse, error) { - out := new(FetchAndWriteNeedleResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/FetchAndWriteNeedle", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *volumeServerClient) Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error) { - stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[9], "/volume_server_pb.VolumeServer/Query", opts...) - if err != nil { - return nil, err - } - x := &volumeServerQueryClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type VolumeServer_QueryClient interface { - Recv() (*QueriedStripe, error) - grpc.ClientStream -} - -type volumeServerQueryClient struct { - grpc.ClientStream -} - -func (x *volumeServerQueryClient) Recv() (*QueriedStripe, error) { - m := new(QueriedStripe) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *volumeServerClient) VolumeNeedleStatus(ctx context.Context, in *VolumeNeedleStatusRequest, opts ...grpc.CallOption) (*VolumeNeedleStatusResponse, error) { - out := new(VolumeNeedleStatusResponse) - err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeNeedleStatus", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -// VolumeServerServer is the server API for VolumeServer service. -type VolumeServerServer interface { - //Experts only: takes multiple fid parameters. This function does not propagate deletes to replicas. - BatchDelete(context.Context, *BatchDeleteRequest) (*BatchDeleteResponse, error) - VacuumVolumeCheck(context.Context, *VacuumVolumeCheckRequest) (*VacuumVolumeCheckResponse, error) - VacuumVolumeCompact(*VacuumVolumeCompactRequest, VolumeServer_VacuumVolumeCompactServer) error - VacuumVolumeCommit(context.Context, *VacuumVolumeCommitRequest) (*VacuumVolumeCommitResponse, error) - VacuumVolumeCleanup(context.Context, *VacuumVolumeCleanupRequest) (*VacuumVolumeCleanupResponse, error) - DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) - AllocateVolume(context.Context, *AllocateVolumeRequest) (*AllocateVolumeResponse, error) - VolumeSyncStatus(context.Context, *VolumeSyncStatusRequest) (*VolumeSyncStatusResponse, error) - VolumeIncrementalCopy(*VolumeIncrementalCopyRequest, VolumeServer_VolumeIncrementalCopyServer) error - VolumeMount(context.Context, *VolumeMountRequest) (*VolumeMountResponse, error) - VolumeUnmount(context.Context, *VolumeUnmountRequest) (*VolumeUnmountResponse, error) - VolumeDelete(context.Context, *VolumeDeleteRequest) (*VolumeDeleteResponse, error) - VolumeMarkReadonly(context.Context, *VolumeMarkReadonlyRequest) (*VolumeMarkReadonlyResponse, error) - VolumeMarkWritable(context.Context, *VolumeMarkWritableRequest) (*VolumeMarkWritableResponse, error) - VolumeConfigure(context.Context, *VolumeConfigureRequest) (*VolumeConfigureResponse, error) - VolumeStatus(context.Context, *VolumeStatusRequest) (*VolumeStatusResponse, error) - // copy the .idx .dat files, and mount this volume - VolumeCopy(*VolumeCopyRequest, VolumeServer_VolumeCopyServer) error - ReadVolumeFileStatus(context.Context, *ReadVolumeFileStatusRequest) (*ReadVolumeFileStatusResponse, error) - CopyFile(*CopyFileRequest, VolumeServer_CopyFileServer) error - ReadNeedleBlob(context.Context, *ReadNeedleBlobRequest) (*ReadNeedleBlobResponse, error) - WriteNeedleBlob(context.Context, *WriteNeedleBlobRequest) (*WriteNeedleBlobResponse, error) - ReadAllNeedles(*ReadAllNeedlesRequest, VolumeServer_ReadAllNeedlesServer) error - VolumeTailSender(*VolumeTailSenderRequest, VolumeServer_VolumeTailSenderServer) error - VolumeTailReceiver(context.Context, *VolumeTailReceiverRequest) (*VolumeTailReceiverResponse, error) - // erasure coding - VolumeEcShardsGenerate(context.Context, *VolumeEcShardsGenerateRequest) (*VolumeEcShardsGenerateResponse, error) - VolumeEcShardsRebuild(context.Context, *VolumeEcShardsRebuildRequest) (*VolumeEcShardsRebuildResponse, error) - VolumeEcShardsCopy(context.Context, *VolumeEcShardsCopyRequest) (*VolumeEcShardsCopyResponse, error) - VolumeEcShardsDelete(context.Context, *VolumeEcShardsDeleteRequest) (*VolumeEcShardsDeleteResponse, error) - VolumeEcShardsMount(context.Context, *VolumeEcShardsMountRequest) (*VolumeEcShardsMountResponse, error) - VolumeEcShardsUnmount(context.Context, *VolumeEcShardsUnmountRequest) (*VolumeEcShardsUnmountResponse, error) - VolumeEcShardRead(*VolumeEcShardReadRequest, VolumeServer_VolumeEcShardReadServer) error - VolumeEcBlobDelete(context.Context, *VolumeEcBlobDeleteRequest) (*VolumeEcBlobDeleteResponse, error) - VolumeEcShardsToVolume(context.Context, *VolumeEcShardsToVolumeRequest) (*VolumeEcShardsToVolumeResponse, error) - // tiered storage - VolumeTierMoveDatToRemote(*VolumeTierMoveDatToRemoteRequest, VolumeServer_VolumeTierMoveDatToRemoteServer) error - VolumeTierMoveDatFromRemote(*VolumeTierMoveDatFromRemoteRequest, VolumeServer_VolumeTierMoveDatFromRemoteServer) error - VolumeServerStatus(context.Context, *VolumeServerStatusRequest) (*VolumeServerStatusResponse, error) - VolumeServerLeave(context.Context, *VolumeServerLeaveRequest) (*VolumeServerLeaveResponse, error) - // remote storage - FetchAndWriteNeedle(context.Context, *FetchAndWriteNeedleRequest) (*FetchAndWriteNeedleResponse, error) - // <experimental> query - Query(*QueryRequest, VolumeServer_QueryServer) error - VolumeNeedleStatus(context.Context, *VolumeNeedleStatusRequest) (*VolumeNeedleStatusResponse, error) -} - -// UnimplementedVolumeServerServer can be embedded to have forward compatible implementations. -type UnimplementedVolumeServerServer struct { -} - -func (*UnimplementedVolumeServerServer) BatchDelete(context.Context, *BatchDeleteRequest) (*BatchDeleteResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method BatchDelete not implemented") -} -func (*UnimplementedVolumeServerServer) VacuumVolumeCheck(context.Context, *VacuumVolumeCheckRequest) (*VacuumVolumeCheckResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VacuumVolumeCheck not implemented") -} -func (*UnimplementedVolumeServerServer) VacuumVolumeCompact(*VacuumVolumeCompactRequest, VolumeServer_VacuumVolumeCompactServer) error { - return status.Errorf(codes.Unimplemented, "method VacuumVolumeCompact not implemented") -} -func (*UnimplementedVolumeServerServer) VacuumVolumeCommit(context.Context, *VacuumVolumeCommitRequest) (*VacuumVolumeCommitResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VacuumVolumeCommit not implemented") -} -func (*UnimplementedVolumeServerServer) VacuumVolumeCleanup(context.Context, *VacuumVolumeCleanupRequest) (*VacuumVolumeCleanupResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VacuumVolumeCleanup not implemented") -} -func (*UnimplementedVolumeServerServer) DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DeleteCollection not implemented") -} -func (*UnimplementedVolumeServerServer) AllocateVolume(context.Context, *AllocateVolumeRequest) (*AllocateVolumeResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method AllocateVolume not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeSyncStatus(context.Context, *VolumeSyncStatusRequest) (*VolumeSyncStatusResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeSyncStatus not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeIncrementalCopy(*VolumeIncrementalCopyRequest, VolumeServer_VolumeIncrementalCopyServer) error { - return status.Errorf(codes.Unimplemented, "method VolumeIncrementalCopy not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeMount(context.Context, *VolumeMountRequest) (*VolumeMountResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeMount not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeUnmount(context.Context, *VolumeUnmountRequest) (*VolumeUnmountResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeUnmount not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeDelete(context.Context, *VolumeDeleteRequest) (*VolumeDeleteResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeDelete not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeMarkReadonly(context.Context, *VolumeMarkReadonlyRequest) (*VolumeMarkReadonlyResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeMarkReadonly not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeMarkWritable(context.Context, *VolumeMarkWritableRequest) (*VolumeMarkWritableResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeMarkWritable not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeConfigure(context.Context, *VolumeConfigureRequest) (*VolumeConfigureResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeConfigure not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeStatus(context.Context, *VolumeStatusRequest) (*VolumeStatusResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeStatus not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeCopy(*VolumeCopyRequest, VolumeServer_VolumeCopyServer) error { - return status.Errorf(codes.Unimplemented, "method VolumeCopy not implemented") -} -func (*UnimplementedVolumeServerServer) ReadVolumeFileStatus(context.Context, *ReadVolumeFileStatusRequest) (*ReadVolumeFileStatusResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ReadVolumeFileStatus not implemented") -} -func (*UnimplementedVolumeServerServer) CopyFile(*CopyFileRequest, VolumeServer_CopyFileServer) error { - return status.Errorf(codes.Unimplemented, "method CopyFile not implemented") -} -func (*UnimplementedVolumeServerServer) ReadNeedleBlob(context.Context, *ReadNeedleBlobRequest) (*ReadNeedleBlobResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ReadNeedleBlob not implemented") -} -func (*UnimplementedVolumeServerServer) WriteNeedleBlob(context.Context, *WriteNeedleBlobRequest) (*WriteNeedleBlobResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method WriteNeedleBlob not implemented") -} -func (*UnimplementedVolumeServerServer) ReadAllNeedles(*ReadAllNeedlesRequest, VolumeServer_ReadAllNeedlesServer) error { - return status.Errorf(codes.Unimplemented, "method ReadAllNeedles not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeTailSender(*VolumeTailSenderRequest, VolumeServer_VolumeTailSenderServer) error { - return status.Errorf(codes.Unimplemented, "method VolumeTailSender not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeTailReceiver(context.Context, *VolumeTailReceiverRequest) (*VolumeTailReceiverResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeTailReceiver not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardsGenerate(context.Context, *VolumeEcShardsGenerateRequest) (*VolumeEcShardsGenerateResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsGenerate not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardsRebuild(context.Context, *VolumeEcShardsRebuildRequest) (*VolumeEcShardsRebuildResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsRebuild not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardsCopy(context.Context, *VolumeEcShardsCopyRequest) (*VolumeEcShardsCopyResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsCopy not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardsDelete(context.Context, *VolumeEcShardsDeleteRequest) (*VolumeEcShardsDeleteResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsDelete not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardsMount(context.Context, *VolumeEcShardsMountRequest) (*VolumeEcShardsMountResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsMount not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardsUnmount(context.Context, *VolumeEcShardsUnmountRequest) (*VolumeEcShardsUnmountResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsUnmount not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardRead(*VolumeEcShardReadRequest, VolumeServer_VolumeEcShardReadServer) error { - return status.Errorf(codes.Unimplemented, "method VolumeEcShardRead not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcBlobDelete(context.Context, *VolumeEcBlobDeleteRequest) (*VolumeEcBlobDeleteResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcBlobDelete not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeEcShardsToVolume(context.Context, *VolumeEcShardsToVolumeRequest) (*VolumeEcShardsToVolumeResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsToVolume not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeTierMoveDatToRemote(*VolumeTierMoveDatToRemoteRequest, VolumeServer_VolumeTierMoveDatToRemoteServer) error { - return status.Errorf(codes.Unimplemented, "method VolumeTierMoveDatToRemote not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeTierMoveDatFromRemote(*VolumeTierMoveDatFromRemoteRequest, VolumeServer_VolumeTierMoveDatFromRemoteServer) error { - return status.Errorf(codes.Unimplemented, "method VolumeTierMoveDatFromRemote not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeServerStatus(context.Context, *VolumeServerStatusRequest) (*VolumeServerStatusResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeServerStatus not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeServerLeave(context.Context, *VolumeServerLeaveRequest) (*VolumeServerLeaveResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeServerLeave not implemented") -} -func (*UnimplementedVolumeServerServer) FetchAndWriteNeedle(context.Context, *FetchAndWriteNeedleRequest) (*FetchAndWriteNeedleResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method FetchAndWriteNeedle not implemented") -} -func (*UnimplementedVolumeServerServer) Query(*QueryRequest, VolumeServer_QueryServer) error { - return status.Errorf(codes.Unimplemented, "method Query not implemented") -} -func (*UnimplementedVolumeServerServer) VolumeNeedleStatus(context.Context, *VolumeNeedleStatusRequest) (*VolumeNeedleStatusResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method VolumeNeedleStatus not implemented") -} - -func RegisterVolumeServerServer(s *grpc.Server, srv VolumeServerServer) { - s.RegisterService(&_VolumeServer_serviceDesc, srv) -} - -func _VolumeServer_BatchDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(BatchDeleteRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).BatchDelete(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/BatchDelete", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).BatchDelete(ctx, req.(*BatchDeleteRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VacuumVolumeCheck_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VacuumVolumeCheckRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VacuumVolumeCheck(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VacuumVolumeCheck", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VacuumVolumeCheck(ctx, req.(*VacuumVolumeCheckRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VacuumVolumeCompact_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(VacuumVolumeCompactRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).VacuumVolumeCompact(m, &volumeServerVacuumVolumeCompactServer{stream}) -} - -type VolumeServer_VacuumVolumeCompactServer interface { - Send(*VacuumVolumeCompactResponse) error - grpc.ServerStream -} - -type volumeServerVacuumVolumeCompactServer struct { - grpc.ServerStream -} - -func (x *volumeServerVacuumVolumeCompactServer) Send(m *VacuumVolumeCompactResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VacuumVolumeCommit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VacuumVolumeCommitRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VacuumVolumeCommit(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VacuumVolumeCommit", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VacuumVolumeCommit(ctx, req.(*VacuumVolumeCommitRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VacuumVolumeCleanup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VacuumVolumeCleanupRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VacuumVolumeCleanup(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VacuumVolumeCleanup", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VacuumVolumeCleanup(ctx, req.(*VacuumVolumeCleanupRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_DeleteCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DeleteCollectionRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).DeleteCollection(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/DeleteCollection", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).DeleteCollection(ctx, req.(*DeleteCollectionRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_AllocateVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AllocateVolumeRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).AllocateVolume(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/AllocateVolume", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).AllocateVolume(ctx, req.(*AllocateVolumeRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeSyncStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeSyncStatusRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeSyncStatus(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeSyncStatus", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeSyncStatus(ctx, req.(*VolumeSyncStatusRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeIncrementalCopy_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(VolumeIncrementalCopyRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).VolumeIncrementalCopy(m, &volumeServerVolumeIncrementalCopyServer{stream}) -} - -type VolumeServer_VolumeIncrementalCopyServer interface { - Send(*VolumeIncrementalCopyResponse) error - grpc.ServerStream -} - -type volumeServerVolumeIncrementalCopyServer struct { - grpc.ServerStream -} - -func (x *volumeServerVolumeIncrementalCopyServer) Send(m *VolumeIncrementalCopyResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VolumeMount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeMountRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeMount(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeMount", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeMount(ctx, req.(*VolumeMountRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeUnmount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeUnmountRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeUnmount(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeUnmount", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeUnmount(ctx, req.(*VolumeUnmountRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeDeleteRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeDelete(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeDelete", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeDelete(ctx, req.(*VolumeDeleteRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeMarkReadonly_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeMarkReadonlyRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeMarkReadonly(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeMarkReadonly", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeMarkReadonly(ctx, req.(*VolumeMarkReadonlyRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeMarkWritable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeMarkWritableRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeMarkWritable(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeMarkWritable", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeMarkWritable(ctx, req.(*VolumeMarkWritableRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeConfigure_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeConfigureRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeConfigure(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeConfigure", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeConfigure(ctx, req.(*VolumeConfigureRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeStatusRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeStatus(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeStatus", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeStatus(ctx, req.(*VolumeStatusRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeCopy_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(VolumeCopyRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).VolumeCopy(m, &volumeServerVolumeCopyServer{stream}) -} - -type VolumeServer_VolumeCopyServer interface { - Send(*VolumeCopyResponse) error - grpc.ServerStream -} - -type volumeServerVolumeCopyServer struct { - grpc.ServerStream -} - -func (x *volumeServerVolumeCopyServer) Send(m *VolumeCopyResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_ReadVolumeFileStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ReadVolumeFileStatusRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).ReadVolumeFileStatus(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/ReadVolumeFileStatus", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).ReadVolumeFileStatus(ctx, req.(*ReadVolumeFileStatusRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_CopyFile_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(CopyFileRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).CopyFile(m, &volumeServerCopyFileServer{stream}) -} - -type VolumeServer_CopyFileServer interface { - Send(*CopyFileResponse) error - grpc.ServerStream -} - -type volumeServerCopyFileServer struct { - grpc.ServerStream -} - -func (x *volumeServerCopyFileServer) Send(m *CopyFileResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_ReadNeedleBlob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ReadNeedleBlobRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).ReadNeedleBlob(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/ReadNeedleBlob", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).ReadNeedleBlob(ctx, req.(*ReadNeedleBlobRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_WriteNeedleBlob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(WriteNeedleBlobRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).WriteNeedleBlob(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/WriteNeedleBlob", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).WriteNeedleBlob(ctx, req.(*WriteNeedleBlobRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_ReadAllNeedles_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(ReadAllNeedlesRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).ReadAllNeedles(m, &volumeServerReadAllNeedlesServer{stream}) -} - -type VolumeServer_ReadAllNeedlesServer interface { - Send(*ReadAllNeedlesResponse) error - grpc.ServerStream -} - -type volumeServerReadAllNeedlesServer struct { - grpc.ServerStream -} - -func (x *volumeServerReadAllNeedlesServer) Send(m *ReadAllNeedlesResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VolumeTailSender_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(VolumeTailSenderRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).VolumeTailSender(m, &volumeServerVolumeTailSenderServer{stream}) -} - -type VolumeServer_VolumeTailSenderServer interface { - Send(*VolumeTailSenderResponse) error - grpc.ServerStream -} - -type volumeServerVolumeTailSenderServer struct { - grpc.ServerStream -} - -func (x *volumeServerVolumeTailSenderServer) Send(m *VolumeTailSenderResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VolumeTailReceiver_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeTailReceiverRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeTailReceiver(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeTailReceiver", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeTailReceiver(ctx, req.(*VolumeTailReceiverRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardsGenerate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcShardsGenerateRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcShardsGenerate(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsGenerate", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcShardsGenerate(ctx, req.(*VolumeEcShardsGenerateRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardsRebuild_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcShardsRebuildRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcShardsRebuild(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsRebuild", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcShardsRebuild(ctx, req.(*VolumeEcShardsRebuildRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardsCopy_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcShardsCopyRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcShardsCopy(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsCopy", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcShardsCopy(ctx, req.(*VolumeEcShardsCopyRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardsDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcShardsDeleteRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcShardsDelete(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsDelete", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcShardsDelete(ctx, req.(*VolumeEcShardsDeleteRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardsMount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcShardsMountRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcShardsMount(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsMount", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcShardsMount(ctx, req.(*VolumeEcShardsMountRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardsUnmount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcShardsUnmountRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcShardsUnmount(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsUnmount", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcShardsUnmount(ctx, req.(*VolumeEcShardsUnmountRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardRead_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(VolumeEcShardReadRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).VolumeEcShardRead(m, &volumeServerVolumeEcShardReadServer{stream}) -} - -type VolumeServer_VolumeEcShardReadServer interface { - Send(*VolumeEcShardReadResponse) error - grpc.ServerStream -} - -type volumeServerVolumeEcShardReadServer struct { - grpc.ServerStream -} - -func (x *volumeServerVolumeEcShardReadServer) Send(m *VolumeEcShardReadResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VolumeEcBlobDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcBlobDeleteRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcBlobDelete(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcBlobDelete", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcBlobDelete(ctx, req.(*VolumeEcBlobDeleteRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeEcShardsToVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeEcShardsToVolumeRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeEcShardsToVolume(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsToVolume", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeEcShardsToVolume(ctx, req.(*VolumeEcShardsToVolumeRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeTierMoveDatToRemote_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(VolumeTierMoveDatToRemoteRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).VolumeTierMoveDatToRemote(m, &volumeServerVolumeTierMoveDatToRemoteServer{stream}) -} - -type VolumeServer_VolumeTierMoveDatToRemoteServer interface { - Send(*VolumeTierMoveDatToRemoteResponse) error - grpc.ServerStream -} - -type volumeServerVolumeTierMoveDatToRemoteServer struct { - grpc.ServerStream -} - -func (x *volumeServerVolumeTierMoveDatToRemoteServer) Send(m *VolumeTierMoveDatToRemoteResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VolumeTierMoveDatFromRemote_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(VolumeTierMoveDatFromRemoteRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).VolumeTierMoveDatFromRemote(m, &volumeServerVolumeTierMoveDatFromRemoteServer{stream}) -} - -type VolumeServer_VolumeTierMoveDatFromRemoteServer interface { - Send(*VolumeTierMoveDatFromRemoteResponse) error - grpc.ServerStream -} - -type volumeServerVolumeTierMoveDatFromRemoteServer struct { - grpc.ServerStream -} - -func (x *volumeServerVolumeTierMoveDatFromRemoteServer) Send(m *VolumeTierMoveDatFromRemoteResponse) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VolumeServerStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeServerStatusRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeServerStatus(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeServerStatus", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeServerStatus(ctx, req.(*VolumeServerStatusRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_VolumeServerLeave_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeServerLeaveRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeServerLeave(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeServerLeave", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeServerLeave(ctx, req.(*VolumeServerLeaveRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_FetchAndWriteNeedle_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(FetchAndWriteNeedleRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).FetchAndWriteNeedle(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/FetchAndWriteNeedle", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).FetchAndWriteNeedle(ctx, req.(*FetchAndWriteNeedleRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _VolumeServer_Query_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(QueryRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(VolumeServerServer).Query(m, &volumeServerQueryServer{stream}) -} - -type VolumeServer_QueryServer interface { - Send(*QueriedStripe) error - grpc.ServerStream -} - -type volumeServerQueryServer struct { - grpc.ServerStream -} - -func (x *volumeServerQueryServer) Send(m *QueriedStripe) error { - return x.ServerStream.SendMsg(m) -} - -func _VolumeServer_VolumeNeedleStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(VolumeNeedleStatusRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(VolumeServerServer).VolumeNeedleStatus(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/volume_server_pb.VolumeServer/VolumeNeedleStatus", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(VolumeServerServer).VolumeNeedleStatus(ctx, req.(*VolumeNeedleStatusRequest)) - } - return interceptor(ctx, in, info, handler) -} - -var _VolumeServer_serviceDesc = grpc.ServiceDesc{ - ServiceName: "volume_server_pb.VolumeServer", - HandlerType: (*VolumeServerServer)(nil), - Methods: []grpc.MethodDesc{ - { - MethodName: "BatchDelete", - Handler: _VolumeServer_BatchDelete_Handler, - }, - { - MethodName: "VacuumVolumeCheck", - Handler: _VolumeServer_VacuumVolumeCheck_Handler, - }, - { - MethodName: "VacuumVolumeCommit", - Handler: _VolumeServer_VacuumVolumeCommit_Handler, - }, - { - MethodName: "VacuumVolumeCleanup", - Handler: _VolumeServer_VacuumVolumeCleanup_Handler, - }, - { - MethodName: "DeleteCollection", - Handler: _VolumeServer_DeleteCollection_Handler, - }, - { - MethodName: "AllocateVolume", - Handler: _VolumeServer_AllocateVolume_Handler, - }, - { - MethodName: "VolumeSyncStatus", - Handler: _VolumeServer_VolumeSyncStatus_Handler, - }, - { - MethodName: "VolumeMount", - Handler: _VolumeServer_VolumeMount_Handler, - }, - { - MethodName: "VolumeUnmount", - Handler: _VolumeServer_VolumeUnmount_Handler, - }, - { - MethodName: "VolumeDelete", - Handler: _VolumeServer_VolumeDelete_Handler, - }, - { - MethodName: "VolumeMarkReadonly", - Handler: _VolumeServer_VolumeMarkReadonly_Handler, - }, - { - MethodName: "VolumeMarkWritable", - Handler: _VolumeServer_VolumeMarkWritable_Handler, - }, - { - MethodName: "VolumeConfigure", - Handler: _VolumeServer_VolumeConfigure_Handler, - }, - { - MethodName: "VolumeStatus", - Handler: _VolumeServer_VolumeStatus_Handler, - }, - { - MethodName: "ReadVolumeFileStatus", - Handler: _VolumeServer_ReadVolumeFileStatus_Handler, - }, - { - MethodName: "ReadNeedleBlob", - Handler: _VolumeServer_ReadNeedleBlob_Handler, - }, - { - MethodName: "WriteNeedleBlob", - Handler: _VolumeServer_WriteNeedleBlob_Handler, - }, - { - MethodName: "VolumeTailReceiver", - Handler: _VolumeServer_VolumeTailReceiver_Handler, - }, - { - MethodName: "VolumeEcShardsGenerate", - Handler: _VolumeServer_VolumeEcShardsGenerate_Handler, - }, - { - MethodName: "VolumeEcShardsRebuild", - Handler: _VolumeServer_VolumeEcShardsRebuild_Handler, - }, - { - MethodName: "VolumeEcShardsCopy", - Handler: _VolumeServer_VolumeEcShardsCopy_Handler, - }, - { - MethodName: "VolumeEcShardsDelete", - Handler: _VolumeServer_VolumeEcShardsDelete_Handler, - }, - { - MethodName: "VolumeEcShardsMount", - Handler: _VolumeServer_VolumeEcShardsMount_Handler, - }, - { - MethodName: "VolumeEcShardsUnmount", - Handler: _VolumeServer_VolumeEcShardsUnmount_Handler, - }, - { - MethodName: "VolumeEcBlobDelete", - Handler: _VolumeServer_VolumeEcBlobDelete_Handler, - }, - { - MethodName: "VolumeEcShardsToVolume", - Handler: _VolumeServer_VolumeEcShardsToVolume_Handler, - }, - { - MethodName: "VolumeServerStatus", - Handler: _VolumeServer_VolumeServerStatus_Handler, - }, - { - MethodName: "VolumeServerLeave", - Handler: _VolumeServer_VolumeServerLeave_Handler, - }, - { - MethodName: "FetchAndWriteNeedle", - Handler: _VolumeServer_FetchAndWriteNeedle_Handler, - }, - { - MethodName: "VolumeNeedleStatus", - Handler: _VolumeServer_VolumeNeedleStatus_Handler, - }, - }, - Streams: []grpc.StreamDesc{ - { - StreamName: "VacuumVolumeCompact", - Handler: _VolumeServer_VacuumVolumeCompact_Handler, - ServerStreams: true, - }, - { - StreamName: "VolumeIncrementalCopy", - Handler: _VolumeServer_VolumeIncrementalCopy_Handler, - ServerStreams: true, - }, - { - StreamName: "VolumeCopy", - Handler: _VolumeServer_VolumeCopy_Handler, - ServerStreams: true, - }, - { - StreamName: "CopyFile", - Handler: _VolumeServer_CopyFile_Handler, - ServerStreams: true, - }, - { - StreamName: "ReadAllNeedles", - Handler: _VolumeServer_ReadAllNeedles_Handler, - ServerStreams: true, - }, - { - StreamName: "VolumeTailSender", - Handler: _VolumeServer_VolumeTailSender_Handler, - ServerStreams: true, - }, - { - StreamName: "VolumeEcShardRead", - Handler: _VolumeServer_VolumeEcShardRead_Handler, - ServerStreams: true, - }, - { - StreamName: "VolumeTierMoveDatToRemote", - Handler: _VolumeServer_VolumeTierMoveDatToRemote_Handler, - ServerStreams: true, - }, - { - StreamName: "VolumeTierMoveDatFromRemote", - Handler: _VolumeServer_VolumeTierMoveDatFromRemote_Handler, - ServerStreams: true, - }, - { - StreamName: "Query", - Handler: _VolumeServer_Query_Handler, - ServerStreams: true, - }, - }, - Metadata: "volume_server.proto", -} diff --git a/weed/pb/volume_server_pb/volume_server_grpc.pb.go b/weed/pb/volume_server_pb/volume_server_grpc.pb.go new file mode 100644 index 000000000..3334eb61b --- /dev/null +++ b/weed/pb/volume_server_pb/volume_server_grpc.pb.go @@ -0,0 +1,1788 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. + +package volume_server_pb + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// VolumeServerClient is the client API for VolumeServer service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type VolumeServerClient interface { + //Experts only: takes multiple fid parameters. This function does not propagate deletes to replicas. + BatchDelete(ctx context.Context, in *BatchDeleteRequest, opts ...grpc.CallOption) (*BatchDeleteResponse, error) + VacuumVolumeCheck(ctx context.Context, in *VacuumVolumeCheckRequest, opts ...grpc.CallOption) (*VacuumVolumeCheckResponse, error) + VacuumVolumeCompact(ctx context.Context, in *VacuumVolumeCompactRequest, opts ...grpc.CallOption) (VolumeServer_VacuumVolumeCompactClient, error) + VacuumVolumeCommit(ctx context.Context, in *VacuumVolumeCommitRequest, opts ...grpc.CallOption) (*VacuumVolumeCommitResponse, error) + VacuumVolumeCleanup(ctx context.Context, in *VacuumVolumeCleanupRequest, opts ...grpc.CallOption) (*VacuumVolumeCleanupResponse, error) + DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) + AllocateVolume(ctx context.Context, in *AllocateVolumeRequest, opts ...grpc.CallOption) (*AllocateVolumeResponse, error) + VolumeSyncStatus(ctx context.Context, in *VolumeSyncStatusRequest, opts ...grpc.CallOption) (*VolumeSyncStatusResponse, error) + VolumeIncrementalCopy(ctx context.Context, in *VolumeIncrementalCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeIncrementalCopyClient, error) + VolumeMount(ctx context.Context, in *VolumeMountRequest, opts ...grpc.CallOption) (*VolumeMountResponse, error) + VolumeUnmount(ctx context.Context, in *VolumeUnmountRequest, opts ...grpc.CallOption) (*VolumeUnmountResponse, error) + VolumeDelete(ctx context.Context, in *VolumeDeleteRequest, opts ...grpc.CallOption) (*VolumeDeleteResponse, error) + VolumeMarkReadonly(ctx context.Context, in *VolumeMarkReadonlyRequest, opts ...grpc.CallOption) (*VolumeMarkReadonlyResponse, error) + VolumeMarkWritable(ctx context.Context, in *VolumeMarkWritableRequest, opts ...grpc.CallOption) (*VolumeMarkWritableResponse, error) + VolumeConfigure(ctx context.Context, in *VolumeConfigureRequest, opts ...grpc.CallOption) (*VolumeConfigureResponse, error) + VolumeStatus(ctx context.Context, in *VolumeStatusRequest, opts ...grpc.CallOption) (*VolumeStatusResponse, error) + // copy the .idx .dat files, and mount this volume + VolumeCopy(ctx context.Context, in *VolumeCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeCopyClient, error) + ReadVolumeFileStatus(ctx context.Context, in *ReadVolumeFileStatusRequest, opts ...grpc.CallOption) (*ReadVolumeFileStatusResponse, error) + CopyFile(ctx context.Context, in *CopyFileRequest, opts ...grpc.CallOption) (VolumeServer_CopyFileClient, error) + ReadNeedleBlob(ctx context.Context, in *ReadNeedleBlobRequest, opts ...grpc.CallOption) (*ReadNeedleBlobResponse, error) + WriteNeedleBlob(ctx context.Context, in *WriteNeedleBlobRequest, opts ...grpc.CallOption) (*WriteNeedleBlobResponse, error) + ReadAllNeedles(ctx context.Context, in *ReadAllNeedlesRequest, opts ...grpc.CallOption) (VolumeServer_ReadAllNeedlesClient, error) + VolumeTailSender(ctx context.Context, in *VolumeTailSenderRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTailSenderClient, error) + VolumeTailReceiver(ctx context.Context, in *VolumeTailReceiverRequest, opts ...grpc.CallOption) (*VolumeTailReceiverResponse, error) + // erasure coding + VolumeEcShardsGenerate(ctx context.Context, in *VolumeEcShardsGenerateRequest, opts ...grpc.CallOption) (*VolumeEcShardsGenerateResponse, error) + VolumeEcShardsRebuild(ctx context.Context, in *VolumeEcShardsRebuildRequest, opts ...grpc.CallOption) (*VolumeEcShardsRebuildResponse, error) + VolumeEcShardsCopy(ctx context.Context, in *VolumeEcShardsCopyRequest, opts ...grpc.CallOption) (*VolumeEcShardsCopyResponse, error) + VolumeEcShardsDelete(ctx context.Context, in *VolumeEcShardsDeleteRequest, opts ...grpc.CallOption) (*VolumeEcShardsDeleteResponse, error) + VolumeEcShardsMount(ctx context.Context, in *VolumeEcShardsMountRequest, opts ...grpc.CallOption) (*VolumeEcShardsMountResponse, error) + VolumeEcShardsUnmount(ctx context.Context, in *VolumeEcShardsUnmountRequest, opts ...grpc.CallOption) (*VolumeEcShardsUnmountResponse, error) + VolumeEcShardRead(ctx context.Context, in *VolumeEcShardReadRequest, opts ...grpc.CallOption) (VolumeServer_VolumeEcShardReadClient, error) + VolumeEcBlobDelete(ctx context.Context, in *VolumeEcBlobDeleteRequest, opts ...grpc.CallOption) (*VolumeEcBlobDeleteResponse, error) + VolumeEcShardsToVolume(ctx context.Context, in *VolumeEcShardsToVolumeRequest, opts ...grpc.CallOption) (*VolumeEcShardsToVolumeResponse, error) + // tiered storage + VolumeTierMoveDatToRemote(ctx context.Context, in *VolumeTierMoveDatToRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatToRemoteClient, error) + VolumeTierMoveDatFromRemote(ctx context.Context, in *VolumeTierMoveDatFromRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatFromRemoteClient, error) + VolumeServerStatus(ctx context.Context, in *VolumeServerStatusRequest, opts ...grpc.CallOption) (*VolumeServerStatusResponse, error) + VolumeServerLeave(ctx context.Context, in *VolumeServerLeaveRequest, opts ...grpc.CallOption) (*VolumeServerLeaveResponse, error) + // remote storage + FetchAndWriteNeedle(ctx context.Context, in *FetchAndWriteNeedleRequest, opts ...grpc.CallOption) (*FetchAndWriteNeedleResponse, error) + // <experimental> query + Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error) + VolumeNeedleStatus(ctx context.Context, in *VolumeNeedleStatusRequest, opts ...grpc.CallOption) (*VolumeNeedleStatusResponse, error) +} + +type volumeServerClient struct { + cc grpc.ClientConnInterface +} + +func NewVolumeServerClient(cc grpc.ClientConnInterface) VolumeServerClient { + return &volumeServerClient{cc} +} + +func (c *volumeServerClient) BatchDelete(ctx context.Context, in *BatchDeleteRequest, opts ...grpc.CallOption) (*BatchDeleteResponse, error) { + out := new(BatchDeleteResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/BatchDelete", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VacuumVolumeCheck(ctx context.Context, in *VacuumVolumeCheckRequest, opts ...grpc.CallOption) (*VacuumVolumeCheckResponse, error) { + out := new(VacuumVolumeCheckResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VacuumVolumeCheck", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VacuumVolumeCompact(ctx context.Context, in *VacuumVolumeCompactRequest, opts ...grpc.CallOption) (VolumeServer_VacuumVolumeCompactClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[0], "/volume_server_pb.VolumeServer/VacuumVolumeCompact", opts...) + if err != nil { + return nil, err + } + x := &volumeServerVacuumVolumeCompactClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_VacuumVolumeCompactClient interface { + Recv() (*VacuumVolumeCompactResponse, error) + grpc.ClientStream +} + +type volumeServerVacuumVolumeCompactClient struct { + grpc.ClientStream +} + +func (x *volumeServerVacuumVolumeCompactClient) Recv() (*VacuumVolumeCompactResponse, error) { + m := new(VacuumVolumeCompactResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VacuumVolumeCommit(ctx context.Context, in *VacuumVolumeCommitRequest, opts ...grpc.CallOption) (*VacuumVolumeCommitResponse, error) { + out := new(VacuumVolumeCommitResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VacuumVolumeCommit", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VacuumVolumeCleanup(ctx context.Context, in *VacuumVolumeCleanupRequest, opts ...grpc.CallOption) (*VacuumVolumeCleanupResponse, error) { + out := new(VacuumVolumeCleanupResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VacuumVolumeCleanup", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) DeleteCollection(ctx context.Context, in *DeleteCollectionRequest, opts ...grpc.CallOption) (*DeleteCollectionResponse, error) { + out := new(DeleteCollectionResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/DeleteCollection", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) AllocateVolume(ctx context.Context, in *AllocateVolumeRequest, opts ...grpc.CallOption) (*AllocateVolumeResponse, error) { + out := new(AllocateVolumeResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/AllocateVolume", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeSyncStatus(ctx context.Context, in *VolumeSyncStatusRequest, opts ...grpc.CallOption) (*VolumeSyncStatusResponse, error) { + out := new(VolumeSyncStatusResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeSyncStatus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeIncrementalCopy(ctx context.Context, in *VolumeIncrementalCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeIncrementalCopyClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[1], "/volume_server_pb.VolumeServer/VolumeIncrementalCopy", opts...) + if err != nil { + return nil, err + } + x := &volumeServerVolumeIncrementalCopyClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_VolumeIncrementalCopyClient interface { + Recv() (*VolumeIncrementalCopyResponse, error) + grpc.ClientStream +} + +type volumeServerVolumeIncrementalCopyClient struct { + grpc.ClientStream +} + +func (x *volumeServerVolumeIncrementalCopyClient) Recv() (*VolumeIncrementalCopyResponse, error) { + m := new(VolumeIncrementalCopyResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VolumeMount(ctx context.Context, in *VolumeMountRequest, opts ...grpc.CallOption) (*VolumeMountResponse, error) { + out := new(VolumeMountResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeMount", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeUnmount(ctx context.Context, in *VolumeUnmountRequest, opts ...grpc.CallOption) (*VolumeUnmountResponse, error) { + out := new(VolumeUnmountResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeUnmount", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeDelete(ctx context.Context, in *VolumeDeleteRequest, opts ...grpc.CallOption) (*VolumeDeleteResponse, error) { + out := new(VolumeDeleteResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeDelete", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeMarkReadonly(ctx context.Context, in *VolumeMarkReadonlyRequest, opts ...grpc.CallOption) (*VolumeMarkReadonlyResponse, error) { + out := new(VolumeMarkReadonlyResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeMarkReadonly", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeMarkWritable(ctx context.Context, in *VolumeMarkWritableRequest, opts ...grpc.CallOption) (*VolumeMarkWritableResponse, error) { + out := new(VolumeMarkWritableResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeMarkWritable", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeConfigure(ctx context.Context, in *VolumeConfigureRequest, opts ...grpc.CallOption) (*VolumeConfigureResponse, error) { + out := new(VolumeConfigureResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeConfigure", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeStatus(ctx context.Context, in *VolumeStatusRequest, opts ...grpc.CallOption) (*VolumeStatusResponse, error) { + out := new(VolumeStatusResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeStatus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeCopy(ctx context.Context, in *VolumeCopyRequest, opts ...grpc.CallOption) (VolumeServer_VolumeCopyClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[2], "/volume_server_pb.VolumeServer/VolumeCopy", opts...) + if err != nil { + return nil, err + } + x := &volumeServerVolumeCopyClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_VolumeCopyClient interface { + Recv() (*VolumeCopyResponse, error) + grpc.ClientStream +} + +type volumeServerVolumeCopyClient struct { + grpc.ClientStream +} + +func (x *volumeServerVolumeCopyClient) Recv() (*VolumeCopyResponse, error) { + m := new(VolumeCopyResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) ReadVolumeFileStatus(ctx context.Context, in *ReadVolumeFileStatusRequest, opts ...grpc.CallOption) (*ReadVolumeFileStatusResponse, error) { + out := new(ReadVolumeFileStatusResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/ReadVolumeFileStatus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) CopyFile(ctx context.Context, in *CopyFileRequest, opts ...grpc.CallOption) (VolumeServer_CopyFileClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[3], "/volume_server_pb.VolumeServer/CopyFile", opts...) + if err != nil { + return nil, err + } + x := &volumeServerCopyFileClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_CopyFileClient interface { + Recv() (*CopyFileResponse, error) + grpc.ClientStream +} + +type volumeServerCopyFileClient struct { + grpc.ClientStream +} + +func (x *volumeServerCopyFileClient) Recv() (*CopyFileResponse, error) { + m := new(CopyFileResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) ReadNeedleBlob(ctx context.Context, in *ReadNeedleBlobRequest, opts ...grpc.CallOption) (*ReadNeedleBlobResponse, error) { + out := new(ReadNeedleBlobResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/ReadNeedleBlob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) WriteNeedleBlob(ctx context.Context, in *WriteNeedleBlobRequest, opts ...grpc.CallOption) (*WriteNeedleBlobResponse, error) { + out := new(WriteNeedleBlobResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/WriteNeedleBlob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) ReadAllNeedles(ctx context.Context, in *ReadAllNeedlesRequest, opts ...grpc.CallOption) (VolumeServer_ReadAllNeedlesClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[4], "/volume_server_pb.VolumeServer/ReadAllNeedles", opts...) + if err != nil { + return nil, err + } + x := &volumeServerReadAllNeedlesClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_ReadAllNeedlesClient interface { + Recv() (*ReadAllNeedlesResponse, error) + grpc.ClientStream +} + +type volumeServerReadAllNeedlesClient struct { + grpc.ClientStream +} + +func (x *volumeServerReadAllNeedlesClient) Recv() (*ReadAllNeedlesResponse, error) { + m := new(ReadAllNeedlesResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VolumeTailSender(ctx context.Context, in *VolumeTailSenderRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTailSenderClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[5], "/volume_server_pb.VolumeServer/VolumeTailSender", opts...) + if err != nil { + return nil, err + } + x := &volumeServerVolumeTailSenderClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_VolumeTailSenderClient interface { + Recv() (*VolumeTailSenderResponse, error) + grpc.ClientStream +} + +type volumeServerVolumeTailSenderClient struct { + grpc.ClientStream +} + +func (x *volumeServerVolumeTailSenderClient) Recv() (*VolumeTailSenderResponse, error) { + m := new(VolumeTailSenderResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VolumeTailReceiver(ctx context.Context, in *VolumeTailReceiverRequest, opts ...grpc.CallOption) (*VolumeTailReceiverResponse, error) { + out := new(VolumeTailReceiverResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeTailReceiver", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardsGenerate(ctx context.Context, in *VolumeEcShardsGenerateRequest, opts ...grpc.CallOption) (*VolumeEcShardsGenerateResponse, error) { + out := new(VolumeEcShardsGenerateResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsGenerate", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardsRebuild(ctx context.Context, in *VolumeEcShardsRebuildRequest, opts ...grpc.CallOption) (*VolumeEcShardsRebuildResponse, error) { + out := new(VolumeEcShardsRebuildResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsRebuild", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardsCopy(ctx context.Context, in *VolumeEcShardsCopyRequest, opts ...grpc.CallOption) (*VolumeEcShardsCopyResponse, error) { + out := new(VolumeEcShardsCopyResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsCopy", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardsDelete(ctx context.Context, in *VolumeEcShardsDeleteRequest, opts ...grpc.CallOption) (*VolumeEcShardsDeleteResponse, error) { + out := new(VolumeEcShardsDeleteResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsDelete", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardsMount(ctx context.Context, in *VolumeEcShardsMountRequest, opts ...grpc.CallOption) (*VolumeEcShardsMountResponse, error) { + out := new(VolumeEcShardsMountResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsMount", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardsUnmount(ctx context.Context, in *VolumeEcShardsUnmountRequest, opts ...grpc.CallOption) (*VolumeEcShardsUnmountResponse, error) { + out := new(VolumeEcShardsUnmountResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsUnmount", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardRead(ctx context.Context, in *VolumeEcShardReadRequest, opts ...grpc.CallOption) (VolumeServer_VolumeEcShardReadClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[6], "/volume_server_pb.VolumeServer/VolumeEcShardRead", opts...) + if err != nil { + return nil, err + } + x := &volumeServerVolumeEcShardReadClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_VolumeEcShardReadClient interface { + Recv() (*VolumeEcShardReadResponse, error) + grpc.ClientStream +} + +type volumeServerVolumeEcShardReadClient struct { + grpc.ClientStream +} + +func (x *volumeServerVolumeEcShardReadClient) Recv() (*VolumeEcShardReadResponse, error) { + m := new(VolumeEcShardReadResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VolumeEcBlobDelete(ctx context.Context, in *VolumeEcBlobDeleteRequest, opts ...grpc.CallOption) (*VolumeEcBlobDeleteResponse, error) { + out := new(VolumeEcBlobDeleteResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcBlobDelete", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeEcShardsToVolume(ctx context.Context, in *VolumeEcShardsToVolumeRequest, opts ...grpc.CallOption) (*VolumeEcShardsToVolumeResponse, error) { + out := new(VolumeEcShardsToVolumeResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsToVolume", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeTierMoveDatToRemote(ctx context.Context, in *VolumeTierMoveDatToRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatToRemoteClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[7], "/volume_server_pb.VolumeServer/VolumeTierMoveDatToRemote", opts...) + if err != nil { + return nil, err + } + x := &volumeServerVolumeTierMoveDatToRemoteClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_VolumeTierMoveDatToRemoteClient interface { + Recv() (*VolumeTierMoveDatToRemoteResponse, error) + grpc.ClientStream +} + +type volumeServerVolumeTierMoveDatToRemoteClient struct { + grpc.ClientStream +} + +func (x *volumeServerVolumeTierMoveDatToRemoteClient) Recv() (*VolumeTierMoveDatToRemoteResponse, error) { + m := new(VolumeTierMoveDatToRemoteResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VolumeTierMoveDatFromRemote(ctx context.Context, in *VolumeTierMoveDatFromRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatFromRemoteClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[8], "/volume_server_pb.VolumeServer/VolumeTierMoveDatFromRemote", opts...) + if err != nil { + return nil, err + } + x := &volumeServerVolumeTierMoveDatFromRemoteClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_VolumeTierMoveDatFromRemoteClient interface { + Recv() (*VolumeTierMoveDatFromRemoteResponse, error) + grpc.ClientStream +} + +type volumeServerVolumeTierMoveDatFromRemoteClient struct { + grpc.ClientStream +} + +func (x *volumeServerVolumeTierMoveDatFromRemoteClient) Recv() (*VolumeTierMoveDatFromRemoteResponse, error) { + m := new(VolumeTierMoveDatFromRemoteResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VolumeServerStatus(ctx context.Context, in *VolumeServerStatusRequest, opts ...grpc.CallOption) (*VolumeServerStatusResponse, error) { + out := new(VolumeServerStatusResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeServerStatus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) VolumeServerLeave(ctx context.Context, in *VolumeServerLeaveRequest, opts ...grpc.CallOption) (*VolumeServerLeaveResponse, error) { + out := new(VolumeServerLeaveResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeServerLeave", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) FetchAndWriteNeedle(ctx context.Context, in *FetchAndWriteNeedleRequest, opts ...grpc.CallOption) (*FetchAndWriteNeedleResponse, error) { + out := new(FetchAndWriteNeedleResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/FetchAndWriteNeedle", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *volumeServerClient) Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error) { + stream, err := c.cc.NewStream(ctx, &VolumeServer_ServiceDesc.Streams[9], "/volume_server_pb.VolumeServer/Query", opts...) + if err != nil { + return nil, err + } + x := &volumeServerQueryClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type VolumeServer_QueryClient interface { + Recv() (*QueriedStripe, error) + grpc.ClientStream +} + +type volumeServerQueryClient struct { + grpc.ClientStream +} + +func (x *volumeServerQueryClient) Recv() (*QueriedStripe, error) { + m := new(QueriedStripe) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *volumeServerClient) VolumeNeedleStatus(ctx context.Context, in *VolumeNeedleStatusRequest, opts ...grpc.CallOption) (*VolumeNeedleStatusResponse, error) { + out := new(VolumeNeedleStatusResponse) + err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeNeedleStatus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// VolumeServerServer is the server API for VolumeServer service. +// All implementations must embed UnimplementedVolumeServerServer +// for forward compatibility +type VolumeServerServer interface { + //Experts only: takes multiple fid parameters. This function does not propagate deletes to replicas. + BatchDelete(context.Context, *BatchDeleteRequest) (*BatchDeleteResponse, error) + VacuumVolumeCheck(context.Context, *VacuumVolumeCheckRequest) (*VacuumVolumeCheckResponse, error) + VacuumVolumeCompact(*VacuumVolumeCompactRequest, VolumeServer_VacuumVolumeCompactServer) error + VacuumVolumeCommit(context.Context, *VacuumVolumeCommitRequest) (*VacuumVolumeCommitResponse, error) + VacuumVolumeCleanup(context.Context, *VacuumVolumeCleanupRequest) (*VacuumVolumeCleanupResponse, error) + DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) + AllocateVolume(context.Context, *AllocateVolumeRequest) (*AllocateVolumeResponse, error) + VolumeSyncStatus(context.Context, *VolumeSyncStatusRequest) (*VolumeSyncStatusResponse, error) + VolumeIncrementalCopy(*VolumeIncrementalCopyRequest, VolumeServer_VolumeIncrementalCopyServer) error + VolumeMount(context.Context, *VolumeMountRequest) (*VolumeMountResponse, error) + VolumeUnmount(context.Context, *VolumeUnmountRequest) (*VolumeUnmountResponse, error) + VolumeDelete(context.Context, *VolumeDeleteRequest) (*VolumeDeleteResponse, error) + VolumeMarkReadonly(context.Context, *VolumeMarkReadonlyRequest) (*VolumeMarkReadonlyResponse, error) + VolumeMarkWritable(context.Context, *VolumeMarkWritableRequest) (*VolumeMarkWritableResponse, error) + VolumeConfigure(context.Context, *VolumeConfigureRequest) (*VolumeConfigureResponse, error) + VolumeStatus(context.Context, *VolumeStatusRequest) (*VolumeStatusResponse, error) + // copy the .idx .dat files, and mount this volume + VolumeCopy(*VolumeCopyRequest, VolumeServer_VolumeCopyServer) error + ReadVolumeFileStatus(context.Context, *ReadVolumeFileStatusRequest) (*ReadVolumeFileStatusResponse, error) + CopyFile(*CopyFileRequest, VolumeServer_CopyFileServer) error + ReadNeedleBlob(context.Context, *ReadNeedleBlobRequest) (*ReadNeedleBlobResponse, error) + WriteNeedleBlob(context.Context, *WriteNeedleBlobRequest) (*WriteNeedleBlobResponse, error) + ReadAllNeedles(*ReadAllNeedlesRequest, VolumeServer_ReadAllNeedlesServer) error + VolumeTailSender(*VolumeTailSenderRequest, VolumeServer_VolumeTailSenderServer) error + VolumeTailReceiver(context.Context, *VolumeTailReceiverRequest) (*VolumeTailReceiverResponse, error) + // erasure coding + VolumeEcShardsGenerate(context.Context, *VolumeEcShardsGenerateRequest) (*VolumeEcShardsGenerateResponse, error) + VolumeEcShardsRebuild(context.Context, *VolumeEcShardsRebuildRequest) (*VolumeEcShardsRebuildResponse, error) + VolumeEcShardsCopy(context.Context, *VolumeEcShardsCopyRequest) (*VolumeEcShardsCopyResponse, error) + VolumeEcShardsDelete(context.Context, *VolumeEcShardsDeleteRequest) (*VolumeEcShardsDeleteResponse, error) + VolumeEcShardsMount(context.Context, *VolumeEcShardsMountRequest) (*VolumeEcShardsMountResponse, error) + VolumeEcShardsUnmount(context.Context, *VolumeEcShardsUnmountRequest) (*VolumeEcShardsUnmountResponse, error) + VolumeEcShardRead(*VolumeEcShardReadRequest, VolumeServer_VolumeEcShardReadServer) error + VolumeEcBlobDelete(context.Context, *VolumeEcBlobDeleteRequest) (*VolumeEcBlobDeleteResponse, error) + VolumeEcShardsToVolume(context.Context, *VolumeEcShardsToVolumeRequest) (*VolumeEcShardsToVolumeResponse, error) + // tiered storage + VolumeTierMoveDatToRemote(*VolumeTierMoveDatToRemoteRequest, VolumeServer_VolumeTierMoveDatToRemoteServer) error + VolumeTierMoveDatFromRemote(*VolumeTierMoveDatFromRemoteRequest, VolumeServer_VolumeTierMoveDatFromRemoteServer) error + VolumeServerStatus(context.Context, *VolumeServerStatusRequest) (*VolumeServerStatusResponse, error) + VolumeServerLeave(context.Context, *VolumeServerLeaveRequest) (*VolumeServerLeaveResponse, error) + // remote storage + FetchAndWriteNeedle(context.Context, *FetchAndWriteNeedleRequest) (*FetchAndWriteNeedleResponse, error) + // <experimental> query + Query(*QueryRequest, VolumeServer_QueryServer) error + VolumeNeedleStatus(context.Context, *VolumeNeedleStatusRequest) (*VolumeNeedleStatusResponse, error) + mustEmbedUnimplementedVolumeServerServer() +} + +// UnimplementedVolumeServerServer must be embedded to have forward compatible implementations. +type UnimplementedVolumeServerServer struct { +} + +func (UnimplementedVolumeServerServer) BatchDelete(context.Context, *BatchDeleteRequest) (*BatchDeleteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method BatchDelete not implemented") +} +func (UnimplementedVolumeServerServer) VacuumVolumeCheck(context.Context, *VacuumVolumeCheckRequest) (*VacuumVolumeCheckResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VacuumVolumeCheck not implemented") +} +func (UnimplementedVolumeServerServer) VacuumVolumeCompact(*VacuumVolumeCompactRequest, VolumeServer_VacuumVolumeCompactServer) error { + return status.Errorf(codes.Unimplemented, "method VacuumVolumeCompact not implemented") +} +func (UnimplementedVolumeServerServer) VacuumVolumeCommit(context.Context, *VacuumVolumeCommitRequest) (*VacuumVolumeCommitResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VacuumVolumeCommit not implemented") +} +func (UnimplementedVolumeServerServer) VacuumVolumeCleanup(context.Context, *VacuumVolumeCleanupRequest) (*VacuumVolumeCleanupResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VacuumVolumeCleanup not implemented") +} +func (UnimplementedVolumeServerServer) DeleteCollection(context.Context, *DeleteCollectionRequest) (*DeleteCollectionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteCollection not implemented") +} +func (UnimplementedVolumeServerServer) AllocateVolume(context.Context, *AllocateVolumeRequest) (*AllocateVolumeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AllocateVolume not implemented") +} +func (UnimplementedVolumeServerServer) VolumeSyncStatus(context.Context, *VolumeSyncStatusRequest) (*VolumeSyncStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeSyncStatus not implemented") +} +func (UnimplementedVolumeServerServer) VolumeIncrementalCopy(*VolumeIncrementalCopyRequest, VolumeServer_VolumeIncrementalCopyServer) error { + return status.Errorf(codes.Unimplemented, "method VolumeIncrementalCopy not implemented") +} +func (UnimplementedVolumeServerServer) VolumeMount(context.Context, *VolumeMountRequest) (*VolumeMountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeMount not implemented") +} +func (UnimplementedVolumeServerServer) VolumeUnmount(context.Context, *VolumeUnmountRequest) (*VolumeUnmountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeUnmount not implemented") +} +func (UnimplementedVolumeServerServer) VolumeDelete(context.Context, *VolumeDeleteRequest) (*VolumeDeleteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeDelete not implemented") +} +func (UnimplementedVolumeServerServer) VolumeMarkReadonly(context.Context, *VolumeMarkReadonlyRequest) (*VolumeMarkReadonlyResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeMarkReadonly not implemented") +} +func (UnimplementedVolumeServerServer) VolumeMarkWritable(context.Context, *VolumeMarkWritableRequest) (*VolumeMarkWritableResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeMarkWritable not implemented") +} +func (UnimplementedVolumeServerServer) VolumeConfigure(context.Context, *VolumeConfigureRequest) (*VolumeConfigureResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeConfigure not implemented") +} +func (UnimplementedVolumeServerServer) VolumeStatus(context.Context, *VolumeStatusRequest) (*VolumeStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeStatus not implemented") +} +func (UnimplementedVolumeServerServer) VolumeCopy(*VolumeCopyRequest, VolumeServer_VolumeCopyServer) error { + return status.Errorf(codes.Unimplemented, "method VolumeCopy not implemented") +} +func (UnimplementedVolumeServerServer) ReadVolumeFileStatus(context.Context, *ReadVolumeFileStatusRequest) (*ReadVolumeFileStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReadVolumeFileStatus not implemented") +} +func (UnimplementedVolumeServerServer) CopyFile(*CopyFileRequest, VolumeServer_CopyFileServer) error { + return status.Errorf(codes.Unimplemented, "method CopyFile not implemented") +} +func (UnimplementedVolumeServerServer) ReadNeedleBlob(context.Context, *ReadNeedleBlobRequest) (*ReadNeedleBlobResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReadNeedleBlob not implemented") +} +func (UnimplementedVolumeServerServer) WriteNeedleBlob(context.Context, *WriteNeedleBlobRequest) (*WriteNeedleBlobResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method WriteNeedleBlob not implemented") +} +func (UnimplementedVolumeServerServer) ReadAllNeedles(*ReadAllNeedlesRequest, VolumeServer_ReadAllNeedlesServer) error { + return status.Errorf(codes.Unimplemented, "method ReadAllNeedles not implemented") +} +func (UnimplementedVolumeServerServer) VolumeTailSender(*VolumeTailSenderRequest, VolumeServer_VolumeTailSenderServer) error { + return status.Errorf(codes.Unimplemented, "method VolumeTailSender not implemented") +} +func (UnimplementedVolumeServerServer) VolumeTailReceiver(context.Context, *VolumeTailReceiverRequest) (*VolumeTailReceiverResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeTailReceiver not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardsGenerate(context.Context, *VolumeEcShardsGenerateRequest) (*VolumeEcShardsGenerateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsGenerate not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardsRebuild(context.Context, *VolumeEcShardsRebuildRequest) (*VolumeEcShardsRebuildResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsRebuild not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardsCopy(context.Context, *VolumeEcShardsCopyRequest) (*VolumeEcShardsCopyResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsCopy not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardsDelete(context.Context, *VolumeEcShardsDeleteRequest) (*VolumeEcShardsDeleteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsDelete not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardsMount(context.Context, *VolumeEcShardsMountRequest) (*VolumeEcShardsMountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsMount not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardsUnmount(context.Context, *VolumeEcShardsUnmountRequest) (*VolumeEcShardsUnmountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsUnmount not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardRead(*VolumeEcShardReadRequest, VolumeServer_VolumeEcShardReadServer) error { + return status.Errorf(codes.Unimplemented, "method VolumeEcShardRead not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcBlobDelete(context.Context, *VolumeEcBlobDeleteRequest) (*VolumeEcBlobDeleteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcBlobDelete not implemented") +} +func (UnimplementedVolumeServerServer) VolumeEcShardsToVolume(context.Context, *VolumeEcShardsToVolumeRequest) (*VolumeEcShardsToVolumeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeEcShardsToVolume not implemented") +} +func (UnimplementedVolumeServerServer) VolumeTierMoveDatToRemote(*VolumeTierMoveDatToRemoteRequest, VolumeServer_VolumeTierMoveDatToRemoteServer) error { + return status.Errorf(codes.Unimplemented, "method VolumeTierMoveDatToRemote not implemented") +} +func (UnimplementedVolumeServerServer) VolumeTierMoveDatFromRemote(*VolumeTierMoveDatFromRemoteRequest, VolumeServer_VolumeTierMoveDatFromRemoteServer) error { + return status.Errorf(codes.Unimplemented, "method VolumeTierMoveDatFromRemote not implemented") +} +func (UnimplementedVolumeServerServer) VolumeServerStatus(context.Context, *VolumeServerStatusRequest) (*VolumeServerStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeServerStatus not implemented") +} +func (UnimplementedVolumeServerServer) VolumeServerLeave(context.Context, *VolumeServerLeaveRequest) (*VolumeServerLeaveResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeServerLeave not implemented") +} +func (UnimplementedVolumeServerServer) FetchAndWriteNeedle(context.Context, *FetchAndWriteNeedleRequest) (*FetchAndWriteNeedleResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method FetchAndWriteNeedle not implemented") +} +func (UnimplementedVolumeServerServer) Query(*QueryRequest, VolumeServer_QueryServer) error { + return status.Errorf(codes.Unimplemented, "method Query not implemented") +} +func (UnimplementedVolumeServerServer) VolumeNeedleStatus(context.Context, *VolumeNeedleStatusRequest) (*VolumeNeedleStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method VolumeNeedleStatus not implemented") +} +func (UnimplementedVolumeServerServer) mustEmbedUnimplementedVolumeServerServer() {} + +// UnsafeVolumeServerServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to VolumeServerServer will +// result in compilation errors. +type UnsafeVolumeServerServer interface { + mustEmbedUnimplementedVolumeServerServer() +} + +func RegisterVolumeServerServer(s grpc.ServiceRegistrar, srv VolumeServerServer) { + s.RegisterService(&VolumeServer_ServiceDesc, srv) +} + +func _VolumeServer_BatchDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(BatchDeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).BatchDelete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/BatchDelete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).BatchDelete(ctx, req.(*BatchDeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VacuumVolumeCheck_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VacuumVolumeCheckRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VacuumVolumeCheck(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VacuumVolumeCheck", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VacuumVolumeCheck(ctx, req.(*VacuumVolumeCheckRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VacuumVolumeCompact_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(VacuumVolumeCompactRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).VacuumVolumeCompact(m, &volumeServerVacuumVolumeCompactServer{stream}) +} + +type VolumeServer_VacuumVolumeCompactServer interface { + Send(*VacuumVolumeCompactResponse) error + grpc.ServerStream +} + +type volumeServerVacuumVolumeCompactServer struct { + grpc.ServerStream +} + +func (x *volumeServerVacuumVolumeCompactServer) Send(m *VacuumVolumeCompactResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VacuumVolumeCommit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VacuumVolumeCommitRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VacuumVolumeCommit(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VacuumVolumeCommit", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VacuumVolumeCommit(ctx, req.(*VacuumVolumeCommitRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VacuumVolumeCleanup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VacuumVolumeCleanupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VacuumVolumeCleanup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VacuumVolumeCleanup", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VacuumVolumeCleanup(ctx, req.(*VacuumVolumeCleanupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_DeleteCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteCollectionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).DeleteCollection(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/DeleteCollection", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).DeleteCollection(ctx, req.(*DeleteCollectionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_AllocateVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AllocateVolumeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).AllocateVolume(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/AllocateVolume", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).AllocateVolume(ctx, req.(*AllocateVolumeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeSyncStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeSyncStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeSyncStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeSyncStatus", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeSyncStatus(ctx, req.(*VolumeSyncStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeIncrementalCopy_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(VolumeIncrementalCopyRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).VolumeIncrementalCopy(m, &volumeServerVolumeIncrementalCopyServer{stream}) +} + +type VolumeServer_VolumeIncrementalCopyServer interface { + Send(*VolumeIncrementalCopyResponse) error + grpc.ServerStream +} + +type volumeServerVolumeIncrementalCopyServer struct { + grpc.ServerStream +} + +func (x *volumeServerVolumeIncrementalCopyServer) Send(m *VolumeIncrementalCopyResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VolumeMount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeMountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeMount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeMount", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeMount(ctx, req.(*VolumeMountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeUnmount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeUnmountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeUnmount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeUnmount", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeUnmount(ctx, req.(*VolumeUnmountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeDeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeDelete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeDelete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeDelete(ctx, req.(*VolumeDeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeMarkReadonly_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeMarkReadonlyRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeMarkReadonly(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeMarkReadonly", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeMarkReadonly(ctx, req.(*VolumeMarkReadonlyRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeMarkWritable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeMarkWritableRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeMarkWritable(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeMarkWritable", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeMarkWritable(ctx, req.(*VolumeMarkWritableRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeConfigure_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeConfigureRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeConfigure(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeConfigure", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeConfigure(ctx, req.(*VolumeConfigureRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeStatus", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeStatus(ctx, req.(*VolumeStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeCopy_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(VolumeCopyRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).VolumeCopy(m, &volumeServerVolumeCopyServer{stream}) +} + +type VolumeServer_VolumeCopyServer interface { + Send(*VolumeCopyResponse) error + grpc.ServerStream +} + +type volumeServerVolumeCopyServer struct { + grpc.ServerStream +} + +func (x *volumeServerVolumeCopyServer) Send(m *VolumeCopyResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_ReadVolumeFileStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReadVolumeFileStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).ReadVolumeFileStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/ReadVolumeFileStatus", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).ReadVolumeFileStatus(ctx, req.(*ReadVolumeFileStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_CopyFile_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(CopyFileRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).CopyFile(m, &volumeServerCopyFileServer{stream}) +} + +type VolumeServer_CopyFileServer interface { + Send(*CopyFileResponse) error + grpc.ServerStream +} + +type volumeServerCopyFileServer struct { + grpc.ServerStream +} + +func (x *volumeServerCopyFileServer) Send(m *CopyFileResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_ReadNeedleBlob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReadNeedleBlobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).ReadNeedleBlob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/ReadNeedleBlob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).ReadNeedleBlob(ctx, req.(*ReadNeedleBlobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_WriteNeedleBlob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(WriteNeedleBlobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).WriteNeedleBlob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/WriteNeedleBlob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).WriteNeedleBlob(ctx, req.(*WriteNeedleBlobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_ReadAllNeedles_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(ReadAllNeedlesRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).ReadAllNeedles(m, &volumeServerReadAllNeedlesServer{stream}) +} + +type VolumeServer_ReadAllNeedlesServer interface { + Send(*ReadAllNeedlesResponse) error + grpc.ServerStream +} + +type volumeServerReadAllNeedlesServer struct { + grpc.ServerStream +} + +func (x *volumeServerReadAllNeedlesServer) Send(m *ReadAllNeedlesResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VolumeTailSender_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(VolumeTailSenderRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).VolumeTailSender(m, &volumeServerVolumeTailSenderServer{stream}) +} + +type VolumeServer_VolumeTailSenderServer interface { + Send(*VolumeTailSenderResponse) error + grpc.ServerStream +} + +type volumeServerVolumeTailSenderServer struct { + grpc.ServerStream +} + +func (x *volumeServerVolumeTailSenderServer) Send(m *VolumeTailSenderResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VolumeTailReceiver_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeTailReceiverRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeTailReceiver(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeTailReceiver", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeTailReceiver(ctx, req.(*VolumeTailReceiverRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardsGenerate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcShardsGenerateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcShardsGenerate(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsGenerate", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcShardsGenerate(ctx, req.(*VolumeEcShardsGenerateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardsRebuild_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcShardsRebuildRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcShardsRebuild(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsRebuild", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcShardsRebuild(ctx, req.(*VolumeEcShardsRebuildRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardsCopy_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcShardsCopyRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcShardsCopy(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsCopy", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcShardsCopy(ctx, req.(*VolumeEcShardsCopyRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardsDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcShardsDeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcShardsDelete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsDelete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcShardsDelete(ctx, req.(*VolumeEcShardsDeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardsMount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcShardsMountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcShardsMount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsMount", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcShardsMount(ctx, req.(*VolumeEcShardsMountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardsUnmount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcShardsUnmountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcShardsUnmount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsUnmount", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcShardsUnmount(ctx, req.(*VolumeEcShardsUnmountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardRead_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(VolumeEcShardReadRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).VolumeEcShardRead(m, &volumeServerVolumeEcShardReadServer{stream}) +} + +type VolumeServer_VolumeEcShardReadServer interface { + Send(*VolumeEcShardReadResponse) error + grpc.ServerStream +} + +type volumeServerVolumeEcShardReadServer struct { + grpc.ServerStream +} + +func (x *volumeServerVolumeEcShardReadServer) Send(m *VolumeEcShardReadResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VolumeEcBlobDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcBlobDeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcBlobDelete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcBlobDelete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcBlobDelete(ctx, req.(*VolumeEcBlobDeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeEcShardsToVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeEcShardsToVolumeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeEcShardsToVolume(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsToVolume", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeEcShardsToVolume(ctx, req.(*VolumeEcShardsToVolumeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeTierMoveDatToRemote_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(VolumeTierMoveDatToRemoteRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).VolumeTierMoveDatToRemote(m, &volumeServerVolumeTierMoveDatToRemoteServer{stream}) +} + +type VolumeServer_VolumeTierMoveDatToRemoteServer interface { + Send(*VolumeTierMoveDatToRemoteResponse) error + grpc.ServerStream +} + +type volumeServerVolumeTierMoveDatToRemoteServer struct { + grpc.ServerStream +} + +func (x *volumeServerVolumeTierMoveDatToRemoteServer) Send(m *VolumeTierMoveDatToRemoteResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VolumeTierMoveDatFromRemote_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(VolumeTierMoveDatFromRemoteRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).VolumeTierMoveDatFromRemote(m, &volumeServerVolumeTierMoveDatFromRemoteServer{stream}) +} + +type VolumeServer_VolumeTierMoveDatFromRemoteServer interface { + Send(*VolumeTierMoveDatFromRemoteResponse) error + grpc.ServerStream +} + +type volumeServerVolumeTierMoveDatFromRemoteServer struct { + grpc.ServerStream +} + +func (x *volumeServerVolumeTierMoveDatFromRemoteServer) Send(m *VolumeTierMoveDatFromRemoteResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VolumeServerStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeServerStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeServerStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeServerStatus", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeServerStatus(ctx, req.(*VolumeServerStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_VolumeServerLeave_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeServerLeaveRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeServerLeave(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeServerLeave", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeServerLeave(ctx, req.(*VolumeServerLeaveRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_FetchAndWriteNeedle_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FetchAndWriteNeedleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).FetchAndWriteNeedle(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/FetchAndWriteNeedle", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).FetchAndWriteNeedle(ctx, req.(*FetchAndWriteNeedleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _VolumeServer_Query_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(QueryRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VolumeServerServer).Query(m, &volumeServerQueryServer{stream}) +} + +type VolumeServer_QueryServer interface { + Send(*QueriedStripe) error + grpc.ServerStream +} + +type volumeServerQueryServer struct { + grpc.ServerStream +} + +func (x *volumeServerQueryServer) Send(m *QueriedStripe) error { + return x.ServerStream.SendMsg(m) +} + +func _VolumeServer_VolumeNeedleStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(VolumeNeedleStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VolumeServerServer).VolumeNeedleStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/volume_server_pb.VolumeServer/VolumeNeedleStatus", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VolumeServerServer).VolumeNeedleStatus(ctx, req.(*VolumeNeedleStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// VolumeServer_ServiceDesc is the grpc.ServiceDesc for VolumeServer service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var VolumeServer_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "volume_server_pb.VolumeServer", + HandlerType: (*VolumeServerServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "BatchDelete", + Handler: _VolumeServer_BatchDelete_Handler, + }, + { + MethodName: "VacuumVolumeCheck", + Handler: _VolumeServer_VacuumVolumeCheck_Handler, + }, + { + MethodName: "VacuumVolumeCommit", + Handler: _VolumeServer_VacuumVolumeCommit_Handler, + }, + { + MethodName: "VacuumVolumeCleanup", + Handler: _VolumeServer_VacuumVolumeCleanup_Handler, + }, + { + MethodName: "DeleteCollection", + Handler: _VolumeServer_DeleteCollection_Handler, + }, + { + MethodName: "AllocateVolume", + Handler: _VolumeServer_AllocateVolume_Handler, + }, + { + MethodName: "VolumeSyncStatus", + Handler: _VolumeServer_VolumeSyncStatus_Handler, + }, + { + MethodName: "VolumeMount", + Handler: _VolumeServer_VolumeMount_Handler, + }, + { + MethodName: "VolumeUnmount", + Handler: _VolumeServer_VolumeUnmount_Handler, + }, + { + MethodName: "VolumeDelete", + Handler: _VolumeServer_VolumeDelete_Handler, + }, + { + MethodName: "VolumeMarkReadonly", + Handler: _VolumeServer_VolumeMarkReadonly_Handler, + }, + { + MethodName: "VolumeMarkWritable", + Handler: _VolumeServer_VolumeMarkWritable_Handler, + }, + { + MethodName: "VolumeConfigure", + Handler: _VolumeServer_VolumeConfigure_Handler, + }, + { + MethodName: "VolumeStatus", + Handler: _VolumeServer_VolumeStatus_Handler, + }, + { + MethodName: "ReadVolumeFileStatus", + Handler: _VolumeServer_ReadVolumeFileStatus_Handler, + }, + { + MethodName: "ReadNeedleBlob", + Handler: _VolumeServer_ReadNeedleBlob_Handler, + }, + { + MethodName: "WriteNeedleBlob", + Handler: _VolumeServer_WriteNeedleBlob_Handler, + }, + { + MethodName: "VolumeTailReceiver", + Handler: _VolumeServer_VolumeTailReceiver_Handler, + }, + { + MethodName: "VolumeEcShardsGenerate", + Handler: _VolumeServer_VolumeEcShardsGenerate_Handler, + }, + { + MethodName: "VolumeEcShardsRebuild", + Handler: _VolumeServer_VolumeEcShardsRebuild_Handler, + }, + { + MethodName: "VolumeEcShardsCopy", + Handler: _VolumeServer_VolumeEcShardsCopy_Handler, + }, + { + MethodName: "VolumeEcShardsDelete", + Handler: _VolumeServer_VolumeEcShardsDelete_Handler, + }, + { + MethodName: "VolumeEcShardsMount", + Handler: _VolumeServer_VolumeEcShardsMount_Handler, + }, + { + MethodName: "VolumeEcShardsUnmount", + Handler: _VolumeServer_VolumeEcShardsUnmount_Handler, + }, + { + MethodName: "VolumeEcBlobDelete", + Handler: _VolumeServer_VolumeEcBlobDelete_Handler, + }, + { + MethodName: "VolumeEcShardsToVolume", + Handler: _VolumeServer_VolumeEcShardsToVolume_Handler, + }, + { + MethodName: "VolumeServerStatus", + Handler: _VolumeServer_VolumeServerStatus_Handler, + }, + { + MethodName: "VolumeServerLeave", + Handler: _VolumeServer_VolumeServerLeave_Handler, + }, + { + MethodName: "FetchAndWriteNeedle", + Handler: _VolumeServer_FetchAndWriteNeedle_Handler, + }, + { + MethodName: "VolumeNeedleStatus", + Handler: _VolumeServer_VolumeNeedleStatus_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "VacuumVolumeCompact", + Handler: _VolumeServer_VacuumVolumeCompact_Handler, + ServerStreams: true, + }, + { + StreamName: "VolumeIncrementalCopy", + Handler: _VolumeServer_VolumeIncrementalCopy_Handler, + ServerStreams: true, + }, + { + StreamName: "VolumeCopy", + Handler: _VolumeServer_VolumeCopy_Handler, + ServerStreams: true, + }, + { + StreamName: "CopyFile", + Handler: _VolumeServer_CopyFile_Handler, + ServerStreams: true, + }, + { + StreamName: "ReadAllNeedles", + Handler: _VolumeServer_ReadAllNeedles_Handler, + ServerStreams: true, + }, + { + StreamName: "VolumeTailSender", + Handler: _VolumeServer_VolumeTailSender_Handler, + ServerStreams: true, + }, + { + StreamName: "VolumeEcShardRead", + Handler: _VolumeServer_VolumeEcShardRead_Handler, + ServerStreams: true, + }, + { + StreamName: "VolumeTierMoveDatToRemote", + Handler: _VolumeServer_VolumeTierMoveDatToRemote_Handler, + ServerStreams: true, + }, + { + StreamName: "VolumeTierMoveDatFromRemote", + Handler: _VolumeServer_VolumeTierMoveDatFromRemote_Handler, + ServerStreams: true, + }, + { + StreamName: "Query", + Handler: _VolumeServer_Query_Handler, + ServerStreams: true, + }, + }, + Metadata: "volume_server.proto", +} diff --git a/weed/s3api/auth_credentials.go b/weed/s3api/auth_credentials.go index d29e8692f..0d46ad7ca 100644 --- a/weed/s3api/auth_credentials.go +++ b/weed/s3api/auth_credentials.go @@ -203,40 +203,51 @@ func (iam *IdentityAccessManagement) authRequest(r *http.Request, action Action) var identity *Identity var s3Err s3err.ErrorCode var found bool + var authType string switch getRequestAuthType(r) { case authTypeStreamingSigned: return identity, s3err.ErrNone case authTypeUnknown: glog.V(3).Infof("unknown auth type") + r.Header.Set(xhttp.AmzAuthType, "Unknown") return identity, s3err.ErrAccessDenied case authTypePresignedV2, authTypeSignedV2: glog.V(3).Infof("v2 auth type") identity, s3Err = iam.isReqAuthenticatedV2(r) + authType = "SigV2" case authTypeSigned, authTypePresigned: glog.V(3).Infof("v4 auth type") identity, s3Err = iam.reqSignatureV4Verify(r) + authType = "SigV4" case authTypePostPolicy: glog.V(3).Infof("post policy auth type") + r.Header.Set(xhttp.AmzAuthType, "PostPolicy") return identity, s3err.ErrNone case authTypeJWT: glog.V(3).Infof("jwt auth type") + r.Header.Set(xhttp.AmzAuthType, "Jwt") return identity, s3err.ErrNotImplemented case authTypeAnonymous: + authType = "Anonymous" identity, found = iam.lookupAnonymous() if !found { + r.Header.Set(xhttp.AmzAuthType, authType) return identity, s3err.ErrAccessDenied } default: return identity, s3err.ErrNotImplemented } + if len(authType) > 0 { + r.Header.Set(xhttp.AmzAuthType, authType) + } if s3Err != s3err.ErrNone { return identity, s3Err } glog.V(3).Infof("user name: %v actions: %v, action: %v", identity.Name, identity.Actions, action) - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) if !identity.canDo(action, bucket) { return identity, s3err.ErrAccessDenied @@ -250,33 +261,45 @@ func (iam *IdentityAccessManagement) authUser(r *http.Request) (*Identity, s3err var identity *Identity var s3Err s3err.ErrorCode var found bool + var authType string switch getRequestAuthType(r) { case authTypeStreamingSigned: return identity, s3err.ErrNone case authTypeUnknown: glog.V(3).Infof("unknown auth type") + r.Header.Set(xhttp.AmzAuthType, "Unknown") return identity, s3err.ErrAccessDenied case authTypePresignedV2, authTypeSignedV2: glog.V(3).Infof("v2 auth type") identity, s3Err = iam.isReqAuthenticatedV2(r) + authType = "SigV2" case authTypeSigned, authTypePresigned: glog.V(3).Infof("v4 auth type") identity, s3Err = iam.reqSignatureV4Verify(r) + authType = "SigV4" case authTypePostPolicy: glog.V(3).Infof("post policy auth type") + r.Header.Set(xhttp.AmzAuthType, "PostPolicy") return identity, s3err.ErrNone case authTypeJWT: glog.V(3).Infof("jwt auth type") + r.Header.Set(xhttp.AmzAuthType, "Jwt") return identity, s3err.ErrNotImplemented case authTypeAnonymous: + authType = "Anonymous" identity, found = iam.lookupAnonymous() if !found { + r.Header.Set(xhttp.AmzAuthType, authType) return identity, s3err.ErrAccessDenied } default: return identity, s3err.ErrNotImplemented } + if len(authType) > 0 { + r.Header.Set(xhttp.AmzAuthType, authType) + } + glog.V(3).Infof("auth error: %v", s3Err) if s3Err != s3err.ErrNone { return identity, s3Err diff --git a/weed/s3api/chunked_reader_v4.go b/weed/s3api/chunked_reader_v4.go index ec26f693a..5dd0648c6 100644 --- a/weed/s3api/chunked_reader_v4.go +++ b/weed/s3api/chunked_reader_v4.go @@ -24,6 +24,7 @@ import ( "crypto/sha256" "encoding/hex" "errors" + xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http" "github.com/chrislusf/seaweedfs/weed/s3api/s3err" "hash" "io" @@ -90,7 +91,7 @@ func (iam *IdentityAccessManagement) calculateSeedSignature(r *http.Request) (cr return nil, "", "", time.Time{}, s3err.ErrInvalidAccessKeyID } - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) if !identity.canDo("Write", bucket) { errCode = s3err.ErrAccessDenied return diff --git a/weed/s3api/filer_multipart.go b/weed/s3api/filer_multipart.go index d93ac63ea..1795ade93 100644 --- a/weed/s3api/filer_multipart.go +++ b/weed/s3api/filer_multipart.go @@ -142,7 +142,7 @@ func (s3a *S3ApiServer) completeMultipartUpload(input *s3.CompleteMultipartUploa output = &CompleteMultipartUploadResult{ CompleteMultipartUploadOutput: s3.CompleteMultipartUploadOutput{ - Location: aws.String(fmt.Sprintf("http://%s%s/%s", s3a.option.Filer.ToHttpAddress(), dirName, entryName)), + Location: aws.String(fmt.Sprintf("http://%s%s/%s", s3a.option.Filer.ToHttpAddress(), urlPathEscape(dirName), urlPathEscape(entryName))), Bucket: input.Bucket, ETag: aws.String("\"" + filer.ETagChunks(finalParts) + "\""), Key: objectKey(input.Key), diff --git a/weed/s3api/http/header.go b/weed/s3api/http/header.go index 6614b0af0..d63d50443 100644 --- a/weed/s3api/http/header.go +++ b/weed/s3api/http/header.go @@ -16,6 +16,12 @@ package http +import ( + "github.com/gorilla/mux" + "net/http" + "strings" +) + // Standard S3 HTTP request constants const ( // S3 storage class @@ -32,5 +38,26 @@ const ( // Non-Standard S3 HTTP request constants const ( AmzIdentityId = "s3-identity-id" + AmzAuthType = "s3-auth-type" AmzIsAdmin = "s3-is-admin" // only set to http request header as a context ) + +func GetBucketAndObject(r *http.Request) (bucket, object string) { + vars := mux.Vars(r) + bucket = vars["bucket"] + object = vars["object"] + if !strings.HasPrefix(object, "/") { + object = "/" + object + } + + return +} + +var PassThroughHeaders = map[string]string{ + "response-cache-control": "Cache-Control", + "response-content-disposition": "Content-Disposition", + "response-content-encoding": "Content-Encoding", + "response-content-language": "Content-Language", + "response-content-type": "Content-Type", + "response-expires": "Expires", +} diff --git a/weed/s3api/s3api_bucket_handlers.go b/weed/s3api/s3api_bucket_handlers.go index b932edbac..247e33104 100644 --- a/weed/s3api/s3api_bucket_handlers.go +++ b/weed/s3api/s3api_bucket_handlers.go @@ -78,7 +78,7 @@ func (s3a *S3ApiServer) ListBucketsHandler(w http.ResponseWriter, r *http.Reques func (s3a *S3ApiServer) PutBucketHandler(w http.ResponseWriter, r *http.Request) { - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("PutBucketHandler %s", bucket) // avoid duplicated buckets @@ -133,13 +133,12 @@ func (s3a *S3ApiServer) PutBucketHandler(w http.ResponseWriter, r *http.Request) s3err.WriteErrorResponse(w, r, s3err.ErrInternalError) return } - writeSuccessResponseEmpty(w, r) } func (s3a *S3ApiServer) DeleteBucketHandler(w http.ResponseWriter, r *http.Request) { - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("DeleteBucketHandler %s", bucket) if err := s3a.checkBucket(r, bucket); err != s3err.ErrNone { @@ -174,7 +173,7 @@ func (s3a *S3ApiServer) DeleteBucketHandler(w http.ResponseWriter, r *http.Reque func (s3a *S3ApiServer) HeadBucketHandler(w http.ResponseWriter, r *http.Request) { - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("HeadBucketHandler %s", bucket) if entry, err := s3a.getEntry(s3a.option.BucketsPath, bucket); entry == nil || err == filer_pb.ErrNotFound { @@ -219,7 +218,7 @@ func (s3a *S3ApiServer) hasAccess(r *http.Request, entry *filer_pb.Entry) bool { // https://docs.aws.amazon.com/AmazonS3/latest/API/API_GetBucketAcl.html func (s3a *S3ApiServer) GetBucketAclHandler(w http.ResponseWriter, r *http.Request) { // collect parameters - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("GetBucketAclHandler %s", bucket) if err := s3a.checkBucket(r, bucket); err != s3err.ErrNone { @@ -259,7 +258,7 @@ func (s3a *S3ApiServer) GetBucketAclHandler(w http.ResponseWriter, r *http.Reque // https://docs.aws.amazon.com/AmazonS3/latest/API/API_GetBucketLifecycleConfiguration.html func (s3a *S3ApiServer) GetBucketLifecycleConfigurationHandler(w http.ResponseWriter, r *http.Request) { // collect parameters - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("GetBucketAclHandler %s", bucket) if err := s3a.checkBucket(r, bucket); err != s3err.ErrNone { diff --git a/weed/s3api/s3api_handlers.go b/weed/s3api/s3api_handlers.go index 5bc94bb04..e42fb6c44 100644 --- a/weed/s3api/s3api_handlers.go +++ b/weed/s3api/s3api_handlers.go @@ -28,6 +28,7 @@ func (s3a *S3ApiServer) AdjustedUrl(location *filer_pb.Location) string { func writeSuccessResponseXML(w http.ResponseWriter, r *http.Request, response interface{}) { s3err.WriteXMLResponse(w, r, http.StatusOK, response) + s3err.PostLog(r, http.StatusOK, s3err.ErrNone) } func writeSuccessResponseEmpty(w http.ResponseWriter, r *http.Request) { diff --git a/weed/s3api/s3api_object_copy_handlers.go b/weed/s3api/s3api_object_copy_handlers.go index e2b191435..7756e1348 100644 --- a/weed/s3api/s3api_object_copy_handlers.go +++ b/weed/s3api/s3api_object_copy_handlers.go @@ -3,6 +3,7 @@ package s3api import ( "fmt" "github.com/chrislusf/seaweedfs/weed/glog" + xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http" "github.com/chrislusf/seaweedfs/weed/s3api/s3err" weed_server "github.com/chrislusf/seaweedfs/weed/server" "net/http" @@ -16,7 +17,7 @@ import ( func (s3a *S3ApiServer) CopyObjectHandler(w http.ResponseWriter, r *http.Request) { - dstBucket, dstObject := getBucketAndObject(r) + dstBucket, dstObject := xhttp.GetBucketAndObject(r) // Copy source path. cpSrcPath, err := url.QueryUnescape(r.Header.Get("X-Amz-Copy-Source")) @@ -69,9 +70,9 @@ func (s3a *S3ApiServer) CopyObjectHandler(w http.ResponseWriter, r *http.Request } dstUrl := fmt.Sprintf("http://%s%s/%s%s?collection=%s", - s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, dstBucket, dstObject, dstBucket) + s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, dstBucket, urlPathEscape(dstObject), dstBucket) srcUrl := fmt.Sprintf("http://%s%s/%s%s", - s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, srcBucket, srcObject) + s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, srcBucket, urlPathEscape(srcObject)) _, _, resp, err := util.DownloadFile(srcUrl, "") if err != nil { @@ -116,7 +117,7 @@ type CopyPartResult struct { func (s3a *S3ApiServer) CopyObjectPartHandler(w http.ResponseWriter, r *http.Request) { // https://docs.aws.amazon.com/AmazonS3/latest/dev/CopyingObjctsUsingRESTMPUapi.html // https://docs.aws.amazon.com/AmazonS3/latest/API/API_UploadPartCopy.html - dstBucket, _ := getBucketAndObject(r) + dstBucket, _ := xhttp.GetBucketAndObject(r) // Copy source path. cpSrcPath, err := url.QueryUnescape(r.Header.Get("X-Amz-Copy-Source")) @@ -154,7 +155,7 @@ func (s3a *S3ApiServer) CopyObjectPartHandler(w http.ResponseWriter, r *http.Req dstUrl := fmt.Sprintf("http://%s%s/%s/%04d.part?collection=%s", s3a.option.Filer.ToHttpAddress(), s3a.genUploadsFolder(dstBucket), uploadID, partID, dstBucket) srcUrl := fmt.Sprintf("http://%s%s/%s%s", - s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, srcBucket, srcObject) + s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, srcBucket, urlPathEscape(srcObject)) dataReader, err := util.ReadUrlAsReaderCloser(srcUrl, rangeHeader) if err != nil { diff --git a/weed/s3api/s3api_object_handlers.go b/weed/s3api/s3api_object_handlers.go index 4defe28da..2ac9c8102 100644 --- a/weed/s3api/s3api_object_handlers.go +++ b/weed/s3api/s3api_object_handlers.go @@ -16,10 +16,9 @@ import ( "github.com/chrislusf/seaweedfs/weed/filer" "github.com/pquerna/cachecontrol/cacheobject" + xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http" "github.com/chrislusf/seaweedfs/weed/s3api/s3err" - "github.com/gorilla/mux" - "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" weed_server "github.com/chrislusf/seaweedfs/weed/server" @@ -51,7 +50,7 @@ func (s3a *S3ApiServer) PutObjectHandler(w http.ResponseWriter, r *http.Request) // http://docs.aws.amazon.com/AmazonS3/latest/dev/UploadingObjects.html - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) glog.V(3).Infof("PutObjectHandler %s %s", bucket, object) _, err := validateContentMd5(r.Header) @@ -133,7 +132,7 @@ func urlPathEscape(object string) string { func (s3a *S3ApiServer) GetObjectHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) glog.V(3).Infof("GetObjectHandler %s %s", bucket, object) if strings.HasSuffix(r.URL.Path, "/") { @@ -145,34 +144,34 @@ func (s3a *S3ApiServer) GetObjectHandler(w http.ResponseWriter, r *http.Request) s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, bucket, urlPathEscape(object)) s3a.proxyToFiler(w, r, destUrl, passThroughResponse) - } func (s3a *S3ApiServer) HeadObjectHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) glog.V(3).Infof("HeadObjectHandler %s %s", bucket, object) destUrl := fmt.Sprintf("http://%s%s/%s%s", s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, bucket, urlPathEscape(object)) s3a.proxyToFiler(w, r, destUrl, passThroughResponse) - } func (s3a *S3ApiServer) DeleteObjectHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) glog.V(3).Infof("DeleteObjectHandler %s %s", bucket, object) destUrl := fmt.Sprintf("http://%s%s/%s%s?recursive=true", s3a.option.Filer.ToHttpAddress(), s3a.option.BucketsPath, bucket, urlPathEscape(object)) - s3a.proxyToFiler(w, r, destUrl, func(proxyResponse *http.Response, w http.ResponseWriter) { + s3a.proxyToFiler(w, r, destUrl, func(proxyResponse *http.Response, w http.ResponseWriter) (statusCode int) { + statusCode = http.StatusNoContent for k, v := range proxyResponse.Header { w.Header()[k] = v } - w.WriteHeader(http.StatusNoContent) + w.WriteHeader(statusCode) + return statusCode }) } @@ -210,7 +209,7 @@ type DeleteObjectsResponse struct { // DeleteMultipleObjectsHandler - Delete multiple objects func (s3a *S3ApiServer) DeleteMultipleObjectsHandler(w http.ResponseWriter, r *http.Request) { - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("DeleteMultipleObjectsHandler %s", bucket) deleteXMLBytes, err := io.ReadAll(r.Body) @@ -227,14 +226,17 @@ func (s3a *S3ApiServer) DeleteMultipleObjectsHandler(w http.ResponseWriter, r *h var deletedObjects []ObjectIdentifier var deleteErrors []DeleteError + var auditLog *s3err.AccessLog directoriesWithDeletion := make(map[string]int) + if s3err.Logger != nil { + auditLog = s3err.GetAccessLog(r, http.StatusNoContent, s3err.ErrNone) + } s3a.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { // delete file entries for _, object := range deleteObjects.Objects { - lastSeparator := strings.LastIndex(object.ObjectName, "/") parentDirectoryPath, entryName, isDeleteData, isRecursive := "", object.ObjectName, true, false if lastSeparator > 0 && lastSeparator+1 < len(object.ObjectName) { @@ -257,6 +259,10 @@ func (s3a *S3ApiServer) DeleteMultipleObjectsHandler(w http.ResponseWriter, r *h Key: object.ObjectName, }) } + if auditLog != nil { + auditLog.Key = entryName + s3err.PostAccessLog(*auditLog) + } } // purge empty folders, only checking folders with deletions @@ -300,16 +306,7 @@ func (s3a *S3ApiServer) doDeleteEmptyDirectories(client filer_pb.SeaweedFilerCli return } -var passThroughHeaders = []string{ - "response-cache-control", - "response-content-disposition", - "response-content-encoding", - "response-content-language", - "response-content-type", - "response-expires", -} - -func (s3a *S3ApiServer) proxyToFiler(w http.ResponseWriter, r *http.Request, destUrl string, responseFn func(proxyResponse *http.Response, w http.ResponseWriter)) { +func (s3a *S3ApiServer) proxyToFiler(w http.ResponseWriter, r *http.Request, destUrl string, responseFn func(proxyResponse *http.Response, w http.ResponseWriter) (statusCode int)) { glog.V(3).Infof("s3 proxying %s to %s", r.Method, destUrl) @@ -322,25 +319,14 @@ func (s3a *S3ApiServer) proxyToFiler(w http.ResponseWriter, r *http.Request, des } proxyReq.Header.Set("X-Forwarded-For", r.RemoteAddr) - - for header, values := range r.Header { - // handle s3 related headers - passed := false - for _, h := range passThroughHeaders { - if strings.ToLower(header) == h && len(values) > 0 { - proxyReq.Header.Add(header[len("response-"):], values[0]) - passed = true - break - } - } - if passed { - continue - } - // handle other headers - for _, value := range values { - proxyReq.Header.Add(header, value) + for k, v := range r.URL.Query() { + if _, ok := xhttp.PassThroughHeaders[strings.ToLower(k)]; ok { + proxyReq.Header[k] = v } } + for header, values := range r.Header { + proxyReq.Header[header] = values + } resp, postErr := client.Do(proxyReq) @@ -363,20 +349,23 @@ func (s3a *S3ApiServer) proxyToFiler(w http.ResponseWriter, r *http.Request, des } } - responseFn(resp, w) - + responseStatusCode := responseFn(resp, w) + s3err.PostLog(r, responseStatusCode, s3err.ErrNone) } -func passThroughResponse(proxyResponse *http.Response, w http.ResponseWriter) { +func passThroughResponse(proxyResponse *http.Response, w http.ResponseWriter) (statusCode int) { for k, v := range proxyResponse.Header { w.Header()[k] = v } if proxyResponse.Header.Get("Content-Range") != "" && proxyResponse.StatusCode == 200 { w.WriteHeader(http.StatusPartialContent) + statusCode = http.StatusPartialContent } else { - w.WriteHeader(proxyResponse.StatusCode) + statusCode = proxyResponse.StatusCode } + w.WriteHeader(statusCode) io.Copy(w, proxyResponse.Body) + return statusCode } func (s3a *S3ApiServer) putToFiler(r *http.Request, uploadUrl string, dataReader io.Reader) (etag string, code s3err.ErrorCode) { @@ -438,17 +427,6 @@ func setEtag(w http.ResponseWriter, etag string) { } } -func getBucketAndObject(r *http.Request) (bucket, object string) { - vars := mux.Vars(r) - bucket = vars["bucket"] - object = vars["object"] - if !strings.HasPrefix(object, "/") { - object = "/" + object - } - - return -} - func filerErrorToS3Error(errString string) s3err.ErrorCode { if strings.HasPrefix(errString, "existing ") && strings.HasSuffix(errString, "is a directory") { return s3err.ErrExistingObjectIsDirectory diff --git a/weed/s3api/s3api_object_handlers_postpolicy.go b/weed/s3api/s3api_object_handlers_postpolicy.go index 23027253e..b0b71b1de 100644 --- a/weed/s3api/s3api_object_handlers_postpolicy.go +++ b/weed/s3api/s3api_object_handlers_postpolicy.go @@ -142,6 +142,7 @@ func (s3a *S3ApiServer) PostPolicyBucketHandler(w http.ResponseWriter, r *http.R Location: w.Header().Get("Location"), } s3err.WriteXMLResponse(w, r, http.StatusCreated, resp) + s3err.PostLog(r, http.StatusCreated, s3err.ErrNone) case "200": s3err.WriteEmptyResponse(w, r, http.StatusOK) default: diff --git a/weed/s3api/s3api_object_multipart_handlers.go b/weed/s3api/s3api_object_multipart_handlers.go index 926e048a8..8cbaf9393 100644 --- a/weed/s3api/s3api_object_multipart_handlers.go +++ b/weed/s3api/s3api_object_multipart_handlers.go @@ -3,6 +3,7 @@ package s3api import ( "fmt" "github.com/chrislusf/seaweedfs/weed/glog" + xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http" "github.com/chrislusf/seaweedfs/weed/s3api/s3err" weed_server "github.com/chrislusf/seaweedfs/weed/server" "net/http" @@ -23,7 +24,7 @@ const ( // NewMultipartUploadHandler - New multipart upload. func (s3a *S3ApiServer) NewMultipartUploadHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) createMultipartUploadInput := &s3.CreateMultipartUploadInput{ Bucket: aws.String(bucket), @@ -55,7 +56,7 @@ func (s3a *S3ApiServer) NewMultipartUploadHandler(w http.ResponseWriter, r *http // CompleteMultipartUploadHandler - Completes multipart upload. func (s3a *S3ApiServer) CompleteMultipartUploadHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) // Get upload id. uploadID, _, _, _ := getObjectResources(r.URL.Query()) @@ -79,7 +80,7 @@ func (s3a *S3ApiServer) CompleteMultipartUploadHandler(w http.ResponseWriter, r // AbortMultipartUploadHandler - Aborts multipart upload. func (s3a *S3ApiServer) AbortMultipartUploadHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) // Get upload id. uploadID, _, _, _ := getObjectResources(r.URL.Query()) @@ -103,7 +104,7 @@ func (s3a *S3ApiServer) AbortMultipartUploadHandler(w http.ResponseWriter, r *ht // ListMultipartUploadsHandler - Lists multipart uploads. func (s3a *S3ApiServer) ListMultipartUploadsHandler(w http.ResponseWriter, r *http.Request) { - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) prefix, keyMarker, uploadIDMarker, delimiter, maxUploads, encodingType := getBucketMultipartResources(r.URL.Query()) if maxUploads < 0 { @@ -142,7 +143,7 @@ func (s3a *S3ApiServer) ListMultipartUploadsHandler(w http.ResponseWriter, r *ht // ListObjectPartsHandler - Lists object parts in a multipart upload. func (s3a *S3ApiServer) ListObjectPartsHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) uploadID, partNumberMarker, maxParts, _ := getObjectResources(r.URL.Query()) if partNumberMarker < 0 { @@ -175,7 +176,7 @@ func (s3a *S3ApiServer) ListObjectPartsHandler(w http.ResponseWriter, r *http.Re // PutObjectPartHandler - Put an object part in a multipart upload. func (s3a *S3ApiServer) PutObjectPartHandler(w http.ResponseWriter, r *http.Request) { - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) uploadID := r.URL.Query().Get("uploadId") exists, err := s3a.exists(s3a.genUploadsFolder(bucket), uploadID, true) diff --git a/weed/s3api/s3api_object_tagging_handlers.go b/weed/s3api/s3api_object_tagging_handlers.go index f97f32f0b..5c66fb961 100644 --- a/weed/s3api/s3api_object_tagging_handlers.go +++ b/weed/s3api/s3api_object_tagging_handlers.go @@ -3,6 +3,7 @@ package s3api import ( "encoding/xml" "fmt" + xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http" "io" "net/http" @@ -16,7 +17,7 @@ import ( // API reference: https://docs.aws.amazon.com/AmazonS3/latest/API/API_GetObjectTagging.html func (s3a *S3ApiServer) GetObjectTaggingHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) glog.V(3).Infof("GetObjectTaggingHandler %s %s", bucket, object) target := util.FullPath(fmt.Sprintf("%s/%s%s", s3a.option.BucketsPath, bucket, object)) @@ -42,7 +43,7 @@ func (s3a *S3ApiServer) GetObjectTaggingHandler(w http.ResponseWriter, r *http.R // API reference: https://docs.aws.amazon.com/AmazonS3/latest/API/API_PutObjectTagging.html func (s3a *S3ApiServer) PutObjectTaggingHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) glog.V(3).Infof("PutObjectTaggingHandler %s %s", bucket, object) target := util.FullPath(fmt.Sprintf("%s/%s%s", s3a.option.BucketsPath, bucket, object)) @@ -91,14 +92,14 @@ func (s3a *S3ApiServer) PutObjectTaggingHandler(w http.ResponseWriter, r *http.R } w.WriteHeader(http.StatusOK) - + s3err.PostLog(r, http.StatusOK, s3err.ErrNone) } // DeleteObjectTaggingHandler Delete object tagging // API reference: https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjectTagging.html func (s3a *S3ApiServer) DeleteObjectTaggingHandler(w http.ResponseWriter, r *http.Request) { - bucket, object := getBucketAndObject(r) + bucket, object := xhttp.GetBucketAndObject(r) glog.V(3).Infof("DeleteObjectTaggingHandler %s %s", bucket, object) target := util.FullPath(fmt.Sprintf("%s/%s%s", s3a.option.BucketsPath, bucket, object)) @@ -117,4 +118,5 @@ func (s3a *S3ApiServer) DeleteObjectTaggingHandler(w http.ResponseWriter, r *htt } w.WriteHeader(http.StatusNoContent) + s3err.PostLog(r, http.StatusNoContent, s3err.ErrNone) } diff --git a/weed/s3api/s3api_objects_list_handlers.go b/weed/s3api/s3api_objects_list_handlers.go index 20ab1d4d6..4decb5eac 100644 --- a/weed/s3api/s3api_objects_list_handlers.go +++ b/weed/s3api/s3api_objects_list_handlers.go @@ -39,7 +39,7 @@ func (s3a *S3ApiServer) ListObjectsV2Handler(w http.ResponseWriter, r *http.Requ // https://docs.aws.amazon.com/AmazonS3/latest/API/v2-RESTBucketGET.html // collect parameters - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("ListObjectsV2Handler %s", bucket) originalPrefix, continuationToken, startAfter, delimiter, _, maxKeys := getListObjectsV2Args(r.URL.Query()) @@ -95,7 +95,7 @@ func (s3a *S3ApiServer) ListObjectsV1Handler(w http.ResponseWriter, r *http.Requ // https://docs.aws.amazon.com/AmazonS3/latest/API/RESTBucketGET.html // collect parameters - bucket, _ := getBucketAndObject(r) + bucket, _ := xhttp.GetBucketAndObject(r) glog.V(3).Infof("ListObjectsV1Handler %s", bucket) originalPrefix, marker, delimiter, maxKeys := getListObjectsV1Args(r.URL.Query()) diff --git a/weed/s3api/s3api_server.go b/weed/s3api/s3api_server.go index 6f7767d66..1abf9259d 100644 --- a/weed/s3api/s3api_server.go +++ b/weed/s3api/s3api_server.go @@ -38,7 +38,6 @@ func NewS3ApiServer(router *mux.Router, option *S3ApiServerOption) (s3ApiServer s3ApiServer.registerRouter(router) go s3ApiServer.subscribeMetaEvents("s3", filer.IamConfigDirecotry+"/"+filer.IamIdentityFile, time.Now().UnixNano()) - return s3ApiServer, nil } @@ -132,7 +131,6 @@ func (s3a *S3ApiServer) registerRouter(router *mux.Router) { // DeleteBucketLifecycleConfiguration bucket.Methods("DELETE").HandlerFunc(s3a.iam.Auth(s3a.DeleteBucketLifecycleHandler, ACTION_WRITE)).Queries("lifecycle", "") - // ListObjectsV1 (Legacy) bucket.Methods("GET").HandlerFunc(track(s3a.iam.Auth(s3a.ListObjectsV1Handler, ACTION_LIST), "LIST")) diff --git a/weed/s3api/s3api_status_handlers.go b/weed/s3api/s3api_status_handlers.go index 2ee6c37b2..fafb6ac2f 100644 --- a/weed/s3api/s3api_status_handlers.go +++ b/weed/s3api/s3api_status_handlers.go @@ -1,8 +1,11 @@ package s3api -import "net/http" +import ( + "github.com/chrislusf/seaweedfs/weed/s3api/s3err" + "net/http" +) func (s3a *S3ApiServer) StatusHandler(w http.ResponseWriter, r *http.Request) { // write out the response code and content type header - writeSuccessResponseEmpty(w, r) + s3err.WriteResponse(w, r, http.StatusOK, []byte{}, "") } diff --git a/weed/s3api/s3err/audit_fluent.go b/weed/s3api/s3err/audit_fluent.go new file mode 100644 index 000000000..fcc5f9a0f --- /dev/null +++ b/weed/s3api/s3err/audit_fluent.go @@ -0,0 +1,183 @@ +package s3err + +import ( + "encoding/json" + "fmt" + "github.com/chrislusf/seaweedfs/weed/glog" + xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http" + "github.com/fluent/fluent-logger-golang/fluent" + "net/http" + "os" + "time" +) + +type AccessLogExtend struct { + AccessLog + AccessLogHTTP +} + +type AccessLog struct { + Bucket string `msg:"bucket" json:"bucket"` // awsexamplebucket1 + Time int64 `msg:"time" json:"time"` // [06/Feb/2019:00:00:38 +0000] + RemoteIP string `msg:"remote_ip" json:"remote_ip,omitempty"` // 192.0.2.3 + Requester string `msg:"requester" json:"requester,omitempty"` // IAM user id + RequestID string `msg:"request_id" json:"request_id,omitempty"` // 3E57427F33A59F07 + Operation string `msg:"operation" json:"operation,omitempty"` // REST.HTTP_method.resource_type REST.PUT.OBJECT + Key string `msg:"key" json:"key,omitempty"` // /photos/2019/08/puppy.jpg + ErrorCode string `msg:"error_code" json:"error_code,omitempty"` + HostId string `msg:"host_id" json:"host_id,omitempty"` + HostHeader string `msg:"host_header" json:"host_header,omitempty"` // s3.us-west-2.amazonaws.com + UserAgent string `msg:"user_agent" json:"user_agent,omitempty"` + HTTPStatus int `msg:"status" json:"status,omitempty"` + SignatureVersion string `msg:"signature_version" json:"signature_version,omitempty"` +} + +type AccessLogHTTP struct { + RequestURI string `json:"request_uri,omitempty"` // "GET /awsexamplebucket1/photos/2019/08/puppy.jpg?x-foo=bar HTTP/1.1" + BytesSent string `json:"bytes_sent,omitempty"` + ObjectSize string `json:"object_size,omitempty"` + TotalTime int `json:"total_time,omitempty"` + TurnAroundTime int `json:"turn_around_time,omitempty"` + Referer string `json:"Referer,omitempty"` + VersionId string `json:"version_id,omitempty"` + CipherSuite string `json:"cipher_suite,omitempty"` + AuthenticationType string `json:"auth_type,omitempty"` + TLSVersion string `json:"TLS_version,omitempty"` +} + +const tag = "s3.access" + +var ( + Logger *fluent.Fluent + hostname = os.Getenv("HOSTNAME") + environment = os.Getenv("ENVIRONMENT") +) + +func InitAuditLog(config string) { + configContent, readErr := os.ReadFile(config) + if readErr != nil { + glog.Errorf("fail to read fluent config %s : %v", config, readErr) + return + } + fluentConfig := &fluent.Config{} + if err := json.Unmarshal(configContent, fluentConfig); err != nil { + glog.Errorf("fail to parse fluent config %s : %v", string(configContent), err) + return + } + if len(fluentConfig.TagPrefix) == 0 && len(environment) > 0 { + fluentConfig.TagPrefix = environment + } + fluentConfig.Async = true + fluentConfig.AsyncResultCallback = func(data []byte, err error) { + if err != nil { + glog.Warning("Error while posting log: ", err) + } + } + var err error + Logger, err = fluent.New(*fluentConfig) + if err != nil { + glog.Errorf("fail to load fluent config: %v", err) + } +} + +func getREST(httpMetod string, resourceType string) string { + return fmt.Sprintf("REST.%s.%s", httpMetod, resourceType) +} + +func getResourceType(object string, query_key string, metod string) (string, bool) { + if object == "/" { + switch query_key { + case "delete": + return "BATCH.DELETE.OBJECT", true + case "tagging": + return getREST(metod, "OBJECTTAGGING"), true + case "lifecycle": + return getREST(metod, "LIFECYCLECONFIGURATION"), true + case "acl": + return getREST(metod, "ACCESSCONTROLPOLICY"), true + case "policy": + return getREST(metod, "BUCKETPOLICY"), true + default: + return getREST(metod, "BUCKET"), false + } + } else { + switch query_key { + case "tagging": + return getREST(metod, "OBJECTTAGGING"), true + default: + return getREST(metod, "OBJECT"), false + } + } +} + +func getOperation(object string, r *http.Request) string { + queries := r.URL.Query() + var operation string + var queryFound bool + for key, _ := range queries { + operation, queryFound = getResourceType(object, key, r.Method) + if queryFound { + return operation + } + } + if len(queries) == 0 { + operation, _ = getResourceType(object, "", r.Method) + } + return operation +} + +func GetAccessHttpLog(r *http.Request, statusCode int, s3errCode ErrorCode) AccessLogHTTP { + return AccessLogHTTP{ + RequestURI: r.RequestURI, + Referer: r.Header.Get("Referer"), + } +} + +func GetAccessLog(r *http.Request, HTTPStatusCode int, s3errCode ErrorCode) *AccessLog { + bucket, key := xhttp.GetBucketAndObject(r) + var errorCode string + if s3errCode != ErrNone { + errorCode = GetAPIError(s3errCode).Code + } + remoteIP := r.Header.Get("X-Real-IP") + if len(remoteIP) == 0 { + remoteIP = r.RemoteAddr + } + hostHeader := r.Header.Get("X-Forwarded-Host") + if len(hostHeader) == 0 { + hostHeader = r.Host + } + return &AccessLog{ + HostHeader: hostHeader, + RequestID: r.Header.Get("X-Request-ID"), + RemoteIP: remoteIP, + Requester: r.Header.Get(xhttp.AmzIdentityId), + SignatureVersion: r.Header.Get(xhttp.AmzAuthType), + UserAgent: r.Header.Get("user-agent"), + HostId: hostname, + Bucket: bucket, + HTTPStatus: HTTPStatusCode, + Time: time.Now().Unix(), + Key: key, + Operation: getOperation(key, r), + ErrorCode: errorCode, + } +} + +func PostLog(r *http.Request, HTTPStatusCode int, errorCode ErrorCode) { + if Logger == nil { + return + } + if err := Logger.Post(tag, *GetAccessLog(r, HTTPStatusCode, errorCode)); err != nil { + glog.Warning("Error while posting log: ", err) + } +} + +func PostAccessLog(log AccessLog) { + if Logger == nil || len(log.Key) == 0 { + return + } + if err := Logger.Post(tag, log); err != nil { + glog.Warning("Error while posting log: ", err) + } +} diff --git a/weed/s3api/s3err/error_handler.go b/weed/s3api/s3err/error_handler.go index 3cfdaafef..6753a1641 100644 --- a/weed/s3api/s3err/error_handler.go +++ b/weed/s3api/s3err/error_handler.go @@ -25,6 +25,7 @@ func WriteXMLResponse(w http.ResponseWriter, r *http.Request, statusCode int, re func WriteEmptyResponse(w http.ResponseWriter, r *http.Request, statusCode int) { WriteResponse(w, r, statusCode, []byte{}, mimeNone) + PostLog(r, statusCode, ErrNone) } func WriteErrorResponse(w http.ResponseWriter, r *http.Request, errorCode ErrorCode) { @@ -39,6 +40,7 @@ func WriteErrorResponse(w http.ResponseWriter, r *http.Request, errorCode ErrorC errorResponse := getRESTErrorResponse(apiError, r.URL.Path, bucket, object) encodedErrorResponse := EncodeXMLResponse(errorResponse) WriteResponse(w, r, apiError.HTTPStatusCode, encodedErrorResponse, MimeXML) + PostLog(r, apiError.HTTPStatusCode, errorCode) } func getRESTErrorResponse(err APIError, resource string, bucket, object string) RESTErrorResponse { diff --git a/weed/server/common.go b/weed/server/common.go index 16213689d..ba4d13456 100644 --- a/weed/server/common.go +++ b/weed/server/common.go @@ -5,6 +5,7 @@ import ( "encoding/json" "errors" "fmt" + xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http" "io" "io/fs" "mime/multipart" @@ -250,13 +251,16 @@ func handleStaticResources2(r *mux.Router) { r.PathPrefix("/seaweedfsstatic/").Handler(http.StripPrefix("/seaweedfsstatic", http.FileServer(http.FS(StaticFS)))) } -func adjustHeaderContentDisposition(w http.ResponseWriter, r *http.Request, filename string) { - responseContentDisposition := r.FormValue("response-content-disposition") - if responseContentDisposition != "" { - w.Header().Set("Content-Disposition", responseContentDisposition) - return +func adjustPassthroughHeaders(w http.ResponseWriter, r *http.Request, filename string) { + for header, values := range r.Header { + if normalizedHeader, ok := xhttp.PassThroughHeaders[strings.ToLower(header)]; ok { + w.Header()[normalizedHeader] = values + } } - if w.Header().Get("Content-Disposition") != "" { + adjustHeaderContentDisposition(w, r, filename) +} +func adjustHeaderContentDisposition(w http.ResponseWriter, r *http.Request, filename string) { + if contentDisposition := w.Header().Get("Content-Disposition"); contentDisposition != "" { return } if filename != "" { diff --git a/weed/server/filer_grpc_server_rename.go b/weed/server/filer_grpc_server_rename.go index a7f428848..62bcccabb 100644 --- a/weed/server/filer_grpc_server_rename.go +++ b/weed/server/filer_grpc_server_rename.go @@ -159,7 +159,7 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.Seawee } if stream != nil { if err := stream.Send(&filer_pb.StreamRenameEntryResponse{ - Directory: string(newParent), + Directory: string(oldParent), EventNotification: &filer_pb.EventNotification{ OldEntry: &filer_pb.Entry{ Name: entry.Name(), diff --git a/weed/server/filer_grpc_server_sub_meta.go b/weed/server/filer_grpc_server_sub_meta.go index 259879790..a0c53cae0 100644 --- a/weed/server/filer_grpc_server_sub_meta.go +++ b/weed/server/filer_grpc_server_sub_meta.go @@ -123,11 +123,11 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq return true }, eachLogEntryFn) if readInMemoryLogErr != nil { + time.Sleep(1127 * time.Millisecond) if readInMemoryLogErr == log_buffer.ResumeFromDiskError { continue } glog.Errorf("processed to %v: %v", lastReadTime, readInMemoryLogErr) - time.Sleep(1127 * time.Millisecond) if readInMemoryLogErr != log_buffer.ResumeError { break } diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go index e5f5a092e..cee15848a 100644 --- a/weed/server/filer_server.go +++ b/weed/server/filer_server.go @@ -3,6 +3,7 @@ package weed_server import ( "context" "fmt" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "net/http" "os" "sync" @@ -65,6 +66,7 @@ type FilerOption struct { } type FilerServer struct { + filer_pb.UnimplementedSeaweedFilerServer option *FilerOption secret security.SigningKey filer *filer.Filer diff --git a/weed/server/filer_server_handlers_read.go b/weed/server/filer_server_handlers_read.go index 86e4af586..ac6aea056 100644 --- a/weed/server/filer_server_handlers_read.go +++ b/weed/server/filer_server_handlers_read.go @@ -130,7 +130,7 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request) setEtag(w, etag) filename := entry.Name() - adjustHeaderContentDisposition(w, r, filename) + adjustPassthroughHeaders(w, r, filename) totalSize := int64(entry.Size()) diff --git a/weed/server/filer_server_handlers_write_autochunk.go b/weed/server/filer_server_handlers_write_autochunk.go index 80e623561..61d30372b 100644 --- a/weed/server/filer_server_handlers_write_autochunk.go +++ b/weed/server/filer_server_handlers_write_autochunk.go @@ -224,8 +224,13 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa entry.Extended = SaveAmzMetaData(r, entry.Extended, false) for k, v := range r.Header { - if len(v) > 0 && len(v[0]) > 0 && (strings.HasPrefix(k, needle.PairNamePrefix) || k == "Cache-Control" || k == "Expires" || k == "Content-Disposition") { - entry.Extended[k] = []byte(v[0]) + if len(v) > 0 && len(v[0]) > 0 { + if strings.HasPrefix(k, needle.PairNamePrefix) || k == "Cache-Control" || k == "Expires" || k == "Content-Disposition" { + entry.Extended[k] = []byte(v[0]) + } + if k == "Response-Content-Disposition" { + entry.Extended["Content-Disposition"] = []byte(v[0]) + } } } diff --git a/weed/server/master_grpc_server_admin.go b/weed/server/master_grpc_server_admin.go index 93c9e4e4e..983606476 100644 --- a/weed/server/master_grpc_server_admin.go +++ b/weed/server/master_grpc_server_admin.go @@ -62,6 +62,7 @@ type AdminLock struct { accessSecret int64 accessLockTime time.Time lastClient string + lastMessage string } type AdminLocks struct { @@ -75,15 +76,15 @@ func NewAdminLocks() *AdminLocks { } } -func (locks *AdminLocks) isLocked(lockName string) (clientName string, isLocked bool) { +func (locks *AdminLocks) isLocked(lockName string) (clientName string, message string, isLocked bool) { locks.RLock() defer locks.RUnlock() adminLock, found := locks.locks[lockName] if !found { - return "", false + return "", "", false } - glog.V(4).Infof("isLocked %v", adminLock.lastClient) - return adminLock.lastClient, adminLock.accessLockTime.Add(LockDuration).After(time.Now()) + glog.V(4).Infof("isLocked %v: %v", adminLock.lastClient, adminLock.lastMessage) + return adminLock.lastClient, adminLock.lastMessage, adminLock.accessLockTime.Add(LockDuration).After(time.Now()) } func (locks *AdminLocks) isValidToken(lockName string, ts time.Time, token int64) bool { @@ -117,7 +118,7 @@ func (locks *AdminLocks) deleteLock(lockName string) { func (ms *MasterServer) LeaseAdminToken(ctx context.Context, req *master_pb.LeaseAdminTokenRequest) (*master_pb.LeaseAdminTokenResponse, error) { resp := &master_pb.LeaseAdminTokenResponse{} - if lastClient, isLocked := ms.adminLocks.isLocked(req.LockName); isLocked { + if lastClient, lastMessage, isLocked := ms.adminLocks.isLocked(req.LockName); isLocked { glog.V(4).Infof("LeaseAdminToken %v", lastClient) if req.PreviousToken != 0 && ms.adminLocks.isValidToken(req.LockName, time.Unix(0, req.PreviousLockTime), req.PreviousToken) { // for renew @@ -126,7 +127,7 @@ func (ms *MasterServer) LeaseAdminToken(ctx context.Context, req *master_pb.Leas return resp, nil } // refuse since still locked - return resp, fmt.Errorf("already locked by " + lastClient) + return resp, fmt.Errorf("already locked by %v: %v", lastClient, lastMessage) } // for fresh lease request ts, token := ms.adminLocks.generateToken(req.LockName, req.ClientName) diff --git a/weed/server/master_server.go b/weed/server/master_server.go index 2b47fda11..cbc0aa337 100644 --- a/weed/server/master_server.go +++ b/weed/server/master_server.go @@ -49,6 +49,7 @@ type MasterOption struct { } type MasterServer struct { + master_pb.UnimplementedSeaweedServer option *MasterOption guard *security.Guard @@ -212,17 +213,7 @@ func (ms *MasterServer) startAdminScripts() { v := util.GetViper() adminScripts := v.GetString("master.maintenance.scripts") if adminScripts == "" { - adminScripts = ` - lock - ec.encode -fullPercent=95 -quietFor=1h - ec.rebuild -force - ec.balance -force - volume.deleteEmpty -quietFor=24h -force - volume.balance -force - volume.fix.replication - s3.clean.uploads -timeAgo=24h - unlock - ` + return } glog.V(0).Infof("adminScripts: %v", adminScripts) @@ -260,8 +251,8 @@ func (ms *MasterServer) startAdminScripts() { go func() { commandEnv.MasterClient.WaitUntilConnected() - c := time.Tick(time.Duration(sleepMinutes) * time.Minute) - for range c { + for { + time.Sleep(time.Duration(sleepMinutes) * time.Minute) if ms.Topo.IsLeader() { for _, line := range scriptLines { for _, c := range strings.Split(line, ";") { diff --git a/weed/server/volume_server.go b/weed/server/volume_server.go index 6b6692146..2551cc6e6 100644 --- a/weed/server/volume_server.go +++ b/weed/server/volume_server.go @@ -2,6 +2,7 @@ package weed_server import ( "github.com/chrislusf/seaweedfs/weed/pb" + "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb" "github.com/chrislusf/seaweedfs/weed/storage/types" "net/http" "sync" @@ -17,6 +18,7 @@ import ( ) type VolumeServer struct { + volume_server_pb.UnimplementedVolumeServerServer inFlightUploadDataSize int64 inFlightDownloadDataSize int64 concurrentUploadLimit int64 diff --git a/weed/server/volume_server_handlers_read.go b/weed/server/volume_server_handlers_read.go index 5d12108d3..5ce2278bf 100644 --- a/weed/server/volume_server_handlers_read.go +++ b/weed/server/volume_server_handlers_read.go @@ -29,8 +29,6 @@ var fileNameEscaper = strings.NewReplacer(`\`, `\\`, `"`, `\"`) func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request) { - glog.V(9).Info(r.Method + " " + r.URL.Path + " " + r.Header.Get("Range")) - stats.VolumeServerRequestCounter.WithLabelValues("get").Inc() start := time.Now() defer func() { stats.VolumeServerRequestHistogram.WithLabelValues("get").Observe(time.Since(start).Seconds()) }() @@ -301,7 +299,7 @@ func writeResponseContent(filename, mimeType string, rs io.ReadSeeker, w http.Re } w.Header().Set("Accept-Ranges", "bytes") - adjustHeaderContentDisposition(w, r, filename) + adjustPassthroughHeaders(w, r, filename) if r.Method == "HEAD" { w.Header().Set("Content-Length", strconv.FormatInt(totalSize, 10)) diff --git a/weed/shell/command_collection_delete.go b/weed/shell/command_collection_delete.go index da79b3437..8942c15da 100644 --- a/weed/shell/command_collection_delete.go +++ b/weed/shell/command_collection_delete.go @@ -36,7 +36,7 @@ func (c *commandCollectionDelete) Do(args []string, commandEnv *CommandEnv, writ return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_ec_balance.go b/weed/shell/command_ec_balance.go index e5438b9c7..6cd91119b 100644 --- a/weed/shell/command_ec_balance.go +++ b/weed/shell/command_ec_balance.go @@ -108,7 +108,7 @@ func (c *commandEcBalance) Do(args []string, commandEnv *CommandEnv, writer io.W return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_ec_decode.go b/weed/shell/command_ec_decode.go index c9f49745b..b2ca605c7 100644 --- a/weed/shell/command_ec_decode.go +++ b/weed/shell/command_ec_decode.go @@ -46,7 +46,7 @@ func (c *commandEcDecode) Do(args []string, commandEnv *CommandEnv, writer io.Wr return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_ec_encode.go b/weed/shell/command_ec_encode.go index 3606f42e3..6add14749 100644 --- a/weed/shell/command_ec_encode.go +++ b/weed/shell/command_ec_encode.go @@ -66,7 +66,7 @@ func (c *commandEcEncode) Do(args []string, commandEnv *CommandEnv, writer io.Wr return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_ec_rebuild.go b/weed/shell/command_ec_rebuild.go index c3f72ea91..409ec4329 100644 --- a/weed/shell/command_ec_rebuild.go +++ b/weed/shell/command_ec_rebuild.go @@ -64,7 +64,7 @@ func (c *commandEcRebuild) Do(args []string, commandEnv *CommandEnv, writer io.W return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_fs_lock_unlock.go b/weed/shell/command_lock_unlock.go index 33458bb6f..33458bb6f 100644 --- a/weed/shell/command_fs_lock_unlock.go +++ b/weed/shell/command_lock_unlock.go diff --git a/weed/shell/command_volume_balance.go b/weed/shell/command_volume_balance.go index 72c4c6db5..4c55de5fb 100644 --- a/weed/shell/command_volume_balance.go +++ b/weed/shell/command_volume_balance.go @@ -71,7 +71,7 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_check_disk.go b/weed/shell/command_volume_check_disk.go index bcc889136..643cccac3 100644 --- a/weed/shell/command_volume_check_disk.go +++ b/weed/shell/command_volume_check_disk.go @@ -51,7 +51,7 @@ func (c *commandVolumeCheckDisk) Do(args []string, commandEnv *CommandEnv, write return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } @@ -110,24 +110,24 @@ func (c *commandVolumeCheckDisk) syncTwoReplicas(aDB *needle_map.MemDb, bDB *nee // read index db if err := c.readIndexDatabase(aDB, a.info.Collection, a.info.Id, pb.NewServerAddressFromDataNode(a.location.dataNode), *verbose, writer); err != nil { - return err + return fmt.Errorf("readIndexDatabase %s volume %d: %v", a.location.dataNode, a.info.Id, err) } if err := c.readIndexDatabase(bDB, b.info.Collection, b.info.Id, pb.NewServerAddressFromDataNode(b.location.dataNode), *verbose, writer); err != nil { - return err + return fmt.Errorf("readIndexDatabase %s volume %d: %v", b.location.dataNode, b.info.Id, err) } // find and make up the differences - if aHasChanges, err = c.doVolumeCheckDisk(aDB, bDB, a, b, *verbose, writer, *applyChanges, *nonRepairThreshold); err != nil { - return err + if aHasChanges, err = c.doVolumeCheckDisk(bDB, aDB, b, a, *verbose, writer, *applyChanges, *nonRepairThreshold); err != nil { + return fmt.Errorf("doVolumeCheckDisk source:%s target:%s volume %d: %v", b.location.dataNode, a.location.dataNode, b.info.Id, err) } - if bHasChanges, err = c.doVolumeCheckDisk(bDB, aDB, b, a, *verbose, writer, *applyChanges, *nonRepairThreshold); err != nil { - return err + if bHasChanges, err = c.doVolumeCheckDisk(aDB, bDB, a, b, *verbose, writer, *applyChanges, *nonRepairThreshold); err != nil { + return fmt.Errorf("doVolumeCheckDisk source:%s target:%s volume %d: %v", a.location.dataNode, b.location.dataNode, a.info.Id, err) } } return nil } -func (c *commandVolumeCheckDisk) doVolumeCheckDisk(subtrahend, minuend *needle_map.MemDb, source, target *VolumeReplica, verbose bool, writer io.Writer, applyChanges bool, nonRepairThreshold float64) (hasChanges bool, err error) { +func (c *commandVolumeCheckDisk) doVolumeCheckDisk(minuend, subtrahend *needle_map.MemDb, source, target *VolumeReplica, verbose bool, writer io.Writer, applyChanges bool, nonRepairThreshold float64) (hasChanges bool, err error) { // find missing keys // hash join, can be more efficient diff --git a/weed/shell/command_volume_configure_replication.go b/weed/shell/command_volume_configure_replication.go index 7e9627b40..27cba618b 100644 --- a/weed/shell/command_volume_configure_replication.go +++ b/weed/shell/command_volume_configure_replication.go @@ -45,7 +45,7 @@ func (c *commandVolumeConfigureReplication) Do(args []string, commandEnv *Comman return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_copy.go b/weed/shell/command_volume_copy.go index b4dfbb78a..0a4ac5063 100644 --- a/weed/shell/command_volume_copy.go +++ b/weed/shell/command_volume_copy.go @@ -41,7 +41,7 @@ func (c *commandVolumeCopy) Do(args []string, commandEnv *CommandEnv, writer io. return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_delete.go b/weed/shell/command_volume_delete.go index 135ad7285..30d71ba9f 100644 --- a/weed/shell/command_volume_delete.go +++ b/weed/shell/command_volume_delete.go @@ -38,7 +38,7 @@ func (c *commandVolumeDelete) Do(args []string, commandEnv *CommandEnv, writer i return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_delete_empty.go b/weed/shell/command_volume_delete_empty.go index fdc4fcf6a..c98693147 100644 --- a/weed/shell/command_volume_delete_empty.go +++ b/weed/shell/command_volume_delete_empty.go @@ -40,7 +40,7 @@ func (c *commandVolumeDeleteEmpty) Do(args []string, commandEnv *CommandEnv, wri return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_fix_replication.go b/weed/shell/command_volume_fix_replication.go index c003eea91..2885ba11f 100644 --- a/weed/shell/command_volume_fix_replication.go +++ b/weed/shell/command_volume_fix_replication.go @@ -64,7 +64,7 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv, return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } @@ -89,7 +89,7 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv, } // find all under replicated volumes - var underReplicatedVolumeIds, overReplicatedVolumeIds []uint32 + var underReplicatedVolumeIds, overReplicatedVolumeIds, misplacedVolumeIds []uint32 for vid, replicas := range volumeReplicas { replica := replicas[0] replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replica.info.ReplicaPlacement)) @@ -98,11 +98,20 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv, } else if replicaPlacement.GetCopyCount() < len(replicas) { overReplicatedVolumeIds = append(overReplicatedVolumeIds, vid) fmt.Fprintf(writer, "volume %d replication %s, but over replicated %+d\n", replica.info.Id, replicaPlacement, len(replicas)) + } else if isMisplaced(replicas, replicaPlacement) { + misplacedVolumeIds = append(misplacedVolumeIds, vid) + fmt.Fprintf(writer, "volume %d replication %s is not well placed %+v\n", replica.info.Id, replicaPlacement, replicas) } } if len(overReplicatedVolumeIds) > 0 { - if err := c.fixOverReplicatedVolumes(commandEnv, writer, takeAction, overReplicatedVolumeIds, volumeReplicas, allLocations); err != nil { + if err := c.deleteOneVolume(commandEnv, writer, takeAction, overReplicatedVolumeIds, volumeReplicas, allLocations, pickOneReplicaToDelete); err != nil { + return err + } + } + + if len(misplacedVolumeIds) > 0 { + if err := c.deleteOneVolume(commandEnv, writer, takeAction, misplacedVolumeIds, volumeReplicas, allLocations, pickOneMisplacedVolume); err != nil { return err } } @@ -171,12 +180,14 @@ func collectVolumeReplicaLocations(topologyInfo *master_pb.TopologyInfo) (map[ui return volumeReplicas, allLocations } -func (c *commandVolumeFixReplication) fixOverReplicatedVolumes(commandEnv *CommandEnv, writer io.Writer, takeAction bool, overReplicatedVolumeIds []uint32, volumeReplicas map[uint32][]*VolumeReplica, allLocations []location) error { +type SelectOneVolumeFunc func(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) *VolumeReplica + +func (c *commandVolumeFixReplication) deleteOneVolume(commandEnv *CommandEnv, writer io.Writer, takeAction bool, overReplicatedVolumeIds []uint32, volumeReplicas map[uint32][]*VolumeReplica, allLocations []location, selectOneVolumeFn SelectOneVolumeFunc) error { for _, vid := range overReplicatedVolumeIds { replicas := volumeReplicas[vid] replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replicas[0].info.ReplicaPlacement)) - replica := pickOneReplicaToDelete(replicas, replicaPlacement) + replica := selectOneVolumeFn(replicas, replicaPlacement) // check collection name pattern if *c.collectionPattern != "" { @@ -495,3 +506,44 @@ func pickOneReplicaToDelete(replicas []*VolumeReplica, replicaPlacement *super_b return replicas[0] } + +// check and fix misplaced volumes + +func isMisplaced(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) bool { + + for i := 0; i < len(replicas); i++ { + others := otherThan(replicas, i) + if satisfyReplicaPlacement(replicaPlacement, others, *replicas[i].location) { + return false + } + } + + return true + +} + +func otherThan(replicas []*VolumeReplica, index int) (others []*VolumeReplica) { + for i := 0; i < len(replicas); i++ { + if index != i { + others = append(others, replicas[i]) + } + } + return +} + +func pickOneMisplacedVolume(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) (toDelete *VolumeReplica) { + + var deletionCandidates []*VolumeReplica + for i := 0; i < len(replicas); i++ { + others := otherThan(replicas, i) + if !isMisplaced(others, replicaPlacement) { + deletionCandidates = append(deletionCandidates, replicas[i]) + } + } + if len(deletionCandidates) > 0 { + return pickOneReplicaToDelete(deletionCandidates, replicaPlacement) + } + + return pickOneReplicaToDelete(replicas, replicaPlacement) + +} diff --git a/weed/shell/command_volume_fix_replication_test.go b/weed/shell/command_volume_fix_replication_test.go index 4d9cd8188..5212fd2ed 100644 --- a/weed/shell/command_volume_fix_replication_test.go +++ b/weed/shell/command_volume_fix_replication_test.go @@ -294,3 +294,141 @@ func runTests(tests []testcase, t *testing.T) { } } } + +func TestMisplacedChecking(t *testing.T) { + + var tests = []testcase{ + { + name: "test 001", + replication: "001", + replicas: []*VolumeReplica{ + { + location: &location{"dc1", "r1", &master_pb.DataNodeInfo{Id: "dn1"}}, + }, + { + location: &location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + }, + }, + expected: true, + }, + { + name: "test 010", + replication: "010", + replicas: []*VolumeReplica{ + { + location: &location{"dc1", "r1", &master_pb.DataNodeInfo{Id: "dn1"}}, + }, + { + location: &location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + }, + }, + expected: false, + }, + { + name: "test 011", + replication: "011", + replicas: []*VolumeReplica{ + { + location: &location{"dc1", "r1", &master_pb.DataNodeInfo{Id: "dn1"}}, + }, + { + location: &location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + }, + }, + expected: false, + }, + { + name: "test 110", + replication: "110", + replicas: []*VolumeReplica{ + { + location: &location{"dc1", "r1", &master_pb.DataNodeInfo{Id: "dn1"}}, + }, + { + location: &location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + }, + }, + expected: false, + }, + { + name: "test 100", + replication: "100", + replicas: []*VolumeReplica{ + { + location: &location{"dc1", "r1", &master_pb.DataNodeInfo{Id: "dn1"}}, + }, + { + location: &location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + }, + }, + expected: true, + }, + } + + for _, tt := range tests { + replicaPlacement, _ := super_block.NewReplicaPlacementFromString(tt.replication) + println("replication:", tt.replication, "expected", tt.expected, "name:", tt.name) + if isMisplaced(tt.replicas, replicaPlacement) != tt.expected { + t.Errorf("%s: expect %v %v %+v", + tt.name, tt.expected, tt.replication, tt.replicas) + } + } + +} + +func TestPickingMisplacedVolumeToDelete(t *testing.T) { + + var tests = []testcase{ + { + name: "test 001", + replication: "001", + replicas: []*VolumeReplica{ + { + location: &location{"dc1", "r1", &master_pb.DataNodeInfo{Id: "dn1"}}, + info: &master_pb.VolumeInformationMessage{ + Size: 100, + }, + }, + { + location: &location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + info: &master_pb.VolumeInformationMessage{ + Size: 99, + }, + }, + }, + possibleLocation: location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + }, + { + name: "test 100", + replication: "100", + replicas: []*VolumeReplica{ + { + location: &location{"dc1", "r1", &master_pb.DataNodeInfo{Id: "dn1"}}, + info: &master_pb.VolumeInformationMessage{ + Size: 100, + }, + }, + { + location: &location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + info: &master_pb.VolumeInformationMessage{ + Size: 99, + }, + }, + }, + possibleLocation: location{"dc1", "r2", &master_pb.DataNodeInfo{Id: "dn2"}}, + }, + } + + for _, tt := range tests { + replicaPlacement, _ := super_block.NewReplicaPlacementFromString(tt.replication) + println("replication:", tt.replication, "name:", tt.name) + if x := pickOneMisplacedVolume(tt.replicas, replicaPlacement); x.location.dataNode.Id != tt.possibleLocation.dataNode.Id { + t.Errorf("%s: picked %+v for replication %v", + tt.name, x.location.dataNode.Id, tt.replication) + } else { + t.Logf("%s: picked %+v %v", + tt.name, x.location.dataNode.Id, tt.replication) + } + } + +} diff --git a/weed/shell/command_volume_fsck.go b/weed/shell/command_volume_fsck.go index bd5241173..a7a981339 100644 --- a/weed/shell/command_volume_fsck.go +++ b/weed/shell/command_volume_fsck.go @@ -66,7 +66,7 @@ func (c *commandVolumeFsck) Do(args []string, commandEnv *CommandEnv, writer io. return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } @@ -212,7 +212,8 @@ func (c *commandVolumeFsck) findExtraChunksInVolumeServers(volumeIdToVInfo map[u if *applyPurging && len(orphanFileIds) > 0 { if vinfo.isEcVolume { - fmt.Fprintf(writer, "Skip purging for Erasure Coded volumes.\n") + fmt.Fprintf(writer, "Skip purging for Erasure Coded volume %d.\n", volumeId) + continue } if inUseCount == 0 { if err := deleteVolume(c.env.option.GrpcDialOption, needle.VolumeId(volumeId), vinfo.server); err != nil { diff --git a/weed/shell/command_volume_list.go b/weed/shell/command_volume_list.go index 9856de10b..531f7f675 100644 --- a/weed/shell/command_volume_list.go +++ b/weed/shell/command_volume_list.go @@ -2,6 +2,7 @@ package shell import ( "bytes" + "flag" "fmt" "github.com/chrislusf/seaweedfs/weed/pb/master_pb" "github.com/chrislusf/seaweedfs/weed/storage/erasure_coding" @@ -31,13 +32,19 @@ func (c *commandVolumeList) Help() string { func (c *commandVolumeList) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) { + volumeListCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError) + verbosityLevel := volumeListCommand.Int("v", 5, "verbose mode: 0, 1, 2, 3, 4, 5") + if err = volumeListCommand.Parse(args); err != nil { + return nil + } + // collect topology information topologyInfo, volumeSizeLimitMb, err := collectTopologyInfo(commandEnv) if err != nil { return err } - writeTopologyInfo(writer, topologyInfo, volumeSizeLimitMb) + writeTopologyInfo(writer, topologyInfo, volumeSizeLimitMb, *verbosityLevel) return nil } @@ -58,77 +65,83 @@ func diskInfoToString(diskInfo *master_pb.DiskInfo) string { return buf.String() } -func writeTopologyInfo(writer io.Writer, t *master_pb.TopologyInfo, volumeSizeLimitMb uint64) statistics { - fmt.Fprintf(writer, "Topology volumeSizeLimit:%d MB%s\n", volumeSizeLimitMb, diskInfosToString(t.DiskInfos)) +func writeTopologyInfo(writer io.Writer, t *master_pb.TopologyInfo, volumeSizeLimitMb uint64, verbosityLevel int) statistics { + output(verbosityLevel >= 0, writer, "Topology volumeSizeLimit:%d MB%s\n", volumeSizeLimitMb, diskInfosToString(t.DiskInfos)) sort.Slice(t.DataCenterInfos, func(i, j int) bool { return t.DataCenterInfos[i].Id < t.DataCenterInfos[j].Id }) var s statistics for _, dc := range t.DataCenterInfos { - s = s.plus(writeDataCenterInfo(writer, dc)) + s = s.plus(writeDataCenterInfo(writer, dc, verbosityLevel)) } - fmt.Fprintf(writer, "%+v \n", s) + output(verbosityLevel >= 0, writer, "%+v \n", s) return s } -func writeDataCenterInfo(writer io.Writer, t *master_pb.DataCenterInfo) statistics { - fmt.Fprintf(writer, " DataCenter %s%s\n", t.Id, diskInfosToString(t.DiskInfos)) +func writeDataCenterInfo(writer io.Writer, t *master_pb.DataCenterInfo, verbosityLevel int) statistics { + output(verbosityLevel >= 1, writer, " DataCenter %s%s\n", t.Id, diskInfosToString(t.DiskInfos)) var s statistics sort.Slice(t.RackInfos, func(i, j int) bool { return t.RackInfos[i].Id < t.RackInfos[j].Id }) for _, r := range t.RackInfos { - s = s.plus(writeRackInfo(writer, r)) + s = s.plus(writeRackInfo(writer, r, verbosityLevel)) } - fmt.Fprintf(writer, " DataCenter %s %+v \n", t.Id, s) + output(verbosityLevel >= 1, writer, " DataCenter %s %+v \n", t.Id, s) return s } -func writeRackInfo(writer io.Writer, t *master_pb.RackInfo) statistics { - fmt.Fprintf(writer, " Rack %s%s\n", t.Id, diskInfosToString(t.DiskInfos)) +func writeRackInfo(writer io.Writer, t *master_pb.RackInfo, verbosityLevel int) statistics { + output(verbosityLevel >= 2, writer, " Rack %s%s\n", t.Id, diskInfosToString(t.DiskInfos)) var s statistics sort.Slice(t.DataNodeInfos, func(i, j int) bool { return t.DataNodeInfos[i].Id < t.DataNodeInfos[j].Id }) for _, dn := range t.DataNodeInfos { - s = s.plus(writeDataNodeInfo(writer, dn)) + s = s.plus(writeDataNodeInfo(writer, dn, verbosityLevel)) } - fmt.Fprintf(writer, " Rack %s %+v \n", t.Id, s) + output(verbosityLevel >= 2, writer, " Rack %s %+v \n", t.Id, s) return s } -func writeDataNodeInfo(writer io.Writer, t *master_pb.DataNodeInfo) statistics { - fmt.Fprintf(writer, " DataNode %s%s\n", t.Id, diskInfosToString(t.DiskInfos)) +func writeDataNodeInfo(writer io.Writer, t *master_pb.DataNodeInfo, verbosityLevel int) statistics { + output(verbosityLevel >= 3, writer, " DataNode %s%s\n", t.Id, diskInfosToString(t.DiskInfos)) var s statistics for _, diskInfo := range t.DiskInfos { - s = s.plus(writeDiskInfo(writer, diskInfo)) + s = s.plus(writeDiskInfo(writer, diskInfo, verbosityLevel)) } - fmt.Fprintf(writer, " DataNode %s %+v \n", t.Id, s) + output(verbosityLevel >= 3, writer, " DataNode %s %+v \n", t.Id, s) return s } -func writeDiskInfo(writer io.Writer, t *master_pb.DiskInfo) statistics { +func writeDiskInfo(writer io.Writer, t *master_pb.DiskInfo, verbosityLevel int) statistics { var s statistics diskType := t.Type if diskType == "" { diskType = "hdd" } - fmt.Fprintf(writer, " Disk %s(%s)\n", diskType, diskInfoToString(t)) + output(verbosityLevel >= 4, writer, " Disk %s(%s)\n", diskType, diskInfoToString(t)) sort.Slice(t.VolumeInfos, func(i, j int) bool { return t.VolumeInfos[i].Id < t.VolumeInfos[j].Id }) for _, vi := range t.VolumeInfos { - s = s.plus(writeVolumeInformationMessage(writer, vi)) + s = s.plus(writeVolumeInformationMessage(writer, vi, verbosityLevel)) } for _, ecShardInfo := range t.EcShardInfos { - fmt.Fprintf(writer, " ec volume id:%v collection:%v shards:%v\n", ecShardInfo.Id, ecShardInfo.Collection, erasure_coding.ShardBits(ecShardInfo.EcIndexBits).ShardIds()) + output(verbosityLevel >= 5, writer, " ec volume id:%v collection:%v shards:%v\n", ecShardInfo.Id, ecShardInfo.Collection, erasure_coding.ShardBits(ecShardInfo.EcIndexBits).ShardIds()) } - fmt.Fprintf(writer, " Disk %s %+v \n", diskType, s) + output(verbosityLevel >= 4, writer, " Disk %s %+v \n", diskType, s) return s } -func writeVolumeInformationMessage(writer io.Writer, t *master_pb.VolumeInformationMessage) statistics { - fmt.Fprintf(writer, " volume %+v \n", t) +func writeVolumeInformationMessage(writer io.Writer, t *master_pb.VolumeInformationMessage, verbosityLevel int) statistics { + output(verbosityLevel >= 5, writer, " volume %+v \n", t) return newStatistics(t) } +func output(condition bool, w io.Writer, format string, a ...interface{}) { + if condition { + fmt.Fprintf(w, format, a...) + } +} + type statistics struct { Size uint64 FileCount uint64 diff --git a/weed/shell/command_volume_mark.go b/weed/shell/command_volume_mark.go index 7734ea9ce..53c2265b0 100644 --- a/weed/shell/command_volume_mark.go +++ b/weed/shell/command_volume_mark.go @@ -44,7 +44,7 @@ func (c *commandVolumeMark) Do(args []string, commandEnv *CommandEnv, writer io. markWritable = true } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_mount.go b/weed/shell/command_volume_mount.go index 385e43341..575051ffe 100644 --- a/weed/shell/command_volume_mount.go +++ b/weed/shell/command_volume_mount.go @@ -42,7 +42,7 @@ func (c *commandVolumeMount) Do(args []string, commandEnv *CommandEnv, writer io return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_move.go b/weed/shell/command_volume_move.go index ec71ba2b3..796f74264 100644 --- a/weed/shell/command_volume_move.go +++ b/weed/shell/command_volume_move.go @@ -59,7 +59,7 @@ func (c *commandVolumeMove) Do(args []string, commandEnv *CommandEnv, writer io. return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_server_evacuate.go b/weed/shell/command_volume_server_evacuate.go index 691d6d552..6a8aeab52 100644 --- a/weed/shell/command_volume_server_evacuate.go +++ b/weed/shell/command_volume_server_evacuate.go @@ -53,7 +53,7 @@ func (c *commandVolumeServerEvacuate) Do(args []string, commandEnv *CommandEnv, return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_server_leave.go b/weed/shell/command_volume_server_leave.go index b1c42f4fd..4daa589be 100644 --- a/weed/shell/command_volume_server_leave.go +++ b/weed/shell/command_volume_server_leave.go @@ -43,7 +43,7 @@ func (c *commandVolumeServerLeave) Do(args []string, commandEnv *CommandEnv, wri return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_tier_download.go b/weed/shell/command_volume_tier_download.go index 72331c8ce..57d3bf347 100644 --- a/weed/shell/command_volume_tier_download.go +++ b/weed/shell/command_volume_tier_download.go @@ -50,7 +50,7 @@ func (c *commandVolumeTierDownload) Do(args []string, commandEnv *CommandEnv, wr return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_tier_move.go b/weed/shell/command_volume_tier_move.go index 348f1799a..bdd90e196 100644 --- a/weed/shell/command_volume_tier_move.go +++ b/weed/shell/command_volume_tier_move.go @@ -57,7 +57,7 @@ func (c *commandVolumeTierMove) Do(args []string, commandEnv *CommandEnv, writer return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_tier_upload.go b/weed/shell/command_volume_tier_upload.go index 72f5347b0..a22fe92a1 100644 --- a/weed/shell/command_volume_tier_upload.go +++ b/weed/shell/command_volume_tier_upload.go @@ -68,7 +68,7 @@ func (c *commandVolumeTierUpload) Do(args []string, commandEnv *CommandEnv, writ return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_unmount.go b/weed/shell/command_volume_unmount.go index 065cbf227..d5cb9f07c 100644 --- a/weed/shell/command_volume_unmount.go +++ b/weed/shell/command_volume_unmount.go @@ -42,7 +42,7 @@ func (c *commandVolumeUnmount) Do(args []string, commandEnv *CommandEnv, writer return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/command_volume_vacuum.go b/weed/shell/command_volume_vacuum.go index ecd4d7756..2e09a8c1b 100644 --- a/weed/shell/command_volume_vacuum.go +++ b/weed/shell/command_volume_vacuum.go @@ -35,7 +35,7 @@ func (c *commandVacuum) Do(args []string, commandEnv *CommandEnv, writer io.Writ return nil } - if err = commandEnv.confirmIsLocked(); err != nil { + if err = commandEnv.confirmIsLocked(args); err != nil { return } diff --git a/weed/shell/commands.go b/weed/shell/commands.go index 6b614c159..02c0af59e 100644 --- a/weed/shell/commands.go +++ b/weed/shell/commands.go @@ -70,11 +70,12 @@ func (ce *CommandEnv) isDirectory(path string) bool { } -func (ce *CommandEnv) confirmIsLocked() error { +func (ce *CommandEnv) confirmIsLocked(args []string) error { if ce.locker.IsLocking() { return nil } + ce.locker.SetMessage(fmt.Sprintf("%v", args)) return fmt.Errorf("need to run \"lock\" first to continue") diff --git a/weed/storage/disk_location.go b/weed/storage/disk_location.go index a32a0093d..af4ec1eb4 100644 --- a/weed/storage/disk_location.go +++ b/weed/storage/disk_location.go @@ -84,7 +84,7 @@ func getValidVolumeName(basename string) string { return "" } -func (l *DiskLocation) loadExistingVolume(dirEntry os.DirEntry, needleMapKind NeedleMapKind) bool { +func (l *DiskLocation) loadExistingVolume(dirEntry os.DirEntry, needleMapKind NeedleMapKind, skipIfEcVolumesExists bool) bool { basename := dirEntry.Name() if dirEntry.IsDir() { return false @@ -94,9 +94,11 @@ func (l *DiskLocation) loadExistingVolume(dirEntry os.DirEntry, needleMapKind Ne return false } - // skip ec volumes - if util.FileExists(l.Directory + "/" + volumeName + ".ecx") { - return false + // skip if ec volumes exists + if skipIfEcVolumesExists { + if util.FileExists(l.Directory + "/" + volumeName + ".ecx") { + return false + } } // check for incomplete volume @@ -166,7 +168,7 @@ func (l *DiskLocation) concurrentLoadingVolumes(needleMapKind NeedleMapKind, con go func() { defer wg.Done() for fi := range task_queue { - _ = l.loadExistingVolume(fi, needleMapKind) + _ = l.loadExistingVolume(fi, needleMapKind, true) } }() } @@ -246,7 +248,7 @@ func (l *DiskLocation) deleteVolumeById(vid needle.VolumeId) (found bool, e erro func (l *DiskLocation) LoadVolume(vid needle.VolumeId, needleMapKind NeedleMapKind) bool { if fileInfo, found := l.LocateVolume(vid); found { - return l.loadExistingVolume(fileInfo, needleMapKind) + return l.loadExistingVolume(fileInfo, needleMapKind, false) } return false } @@ -332,9 +334,12 @@ func (l *DiskLocation) Close() { } func (l *DiskLocation) LocateVolume(vid needle.VolumeId) (os.DirEntry, bool) { + // println("LocateVolume", vid, "on", l.Directory) if dirEntries, err := os.ReadDir(l.Directory); err == nil { for _, entry := range dirEntries { + // println("checking", entry.Name(), "...") volId, _, err := volumeIdFromFileName(entry.Name()) + // println("volId", volId, "err", err) if vid == volId && err == nil { return entry, true } diff --git a/weed/storage/volume_loading.go b/weed/storage/volume_loading.go index 0cf603ad8..2dc75d5ee 100644 --- a/weed/storage/volume_loading.go +++ b/weed/storage/volume_loading.go @@ -83,6 +83,9 @@ func (v *Volume) load(alsoLoadIndex bool, createDatIfMissing bool, needleMapKind if alreadyHasSuperBlock { err = v.readSuperBlock() + if err == nil { + v.volumeInfo.Version = uint32(v.SuperBlock.Version) + } glog.V(0).Infof("readSuperBlock volume %d version %v", v.Id, v.SuperBlock.Version) if v.HasRemoteFile() { // maybe temporary network problem diff --git a/weed/topology/topology.go b/weed/topology/topology.go index ad440e244..207c89ad7 100644 --- a/weed/topology/topology.go +++ b/weed/topology/topology.go @@ -206,7 +206,7 @@ func (t *Topology) RegisterVolumeLayout(v storage.VolumeInfo, dn *DataNode) { vl.EnsureCorrectWritables(&v) } func (t *Topology) UnRegisterVolumeLayout(v storage.VolumeInfo, dn *DataNode) { - glog.Infof("removing volume info: %+v", v) + glog.Infof("removing volume info: %+v from %v", v, dn.id) diskType := types.ToDiskType(v.DiskType) volumeLayout := t.GetVolumeLayout(v.Collection, v.ReplicaPlacement, v.Ttl, diskType) volumeLayout.UnRegisterVolume(&v, dn) diff --git a/weed/util/constants.go b/weed/util/constants.go index 7b5c9ec66..8e178bc4a 100644 --- a/weed/util/constants.go +++ b/weed/util/constants.go @@ -5,7 +5,7 @@ import ( ) var ( - VERSION_NUMBER = fmt.Sprintf("%.02f", 2.80) + VERSION_NUMBER = fmt.Sprintf("%.02f", 2.82) VERSION = sizeLimit + " " + VERSION_NUMBER COMMIT = "" ) diff --git a/weed/util/http_util.go b/weed/util/http_util.go index f005e8d42..7bd6758eb 100644 --- a/weed/util/http_util.go +++ b/weed/util/http_util.go @@ -326,7 +326,7 @@ func ReadUrlAsStream(fileUrl string, cipherKey []byte, isContentGzipped bool, is return false, nil } if err != nil { - return false, err + return true, err } } diff --git a/weed/wdclient/exclusive_locks/exclusive_locker.go b/weed/wdclient/exclusive_locks/exclusive_locker.go index 2f033f36b..725fa307d 100644 --- a/weed/wdclient/exclusive_locks/exclusive_locker.go +++ b/weed/wdclient/exclusive_locks/exclusive_locker.go @@ -22,6 +22,7 @@ type ExclusiveLocker struct { isLocking bool masterClient *wdclient.MasterClient lockName string + message string } func NewExclusiveLocker(masterClient *wdclient.MasterClient, lockName string) *ExclusiveLocker { @@ -87,6 +88,7 @@ func (l *ExclusiveLocker) RequestLock(clientName string) { PreviousLockTime: atomic.LoadInt64(&l.lockTsNs), LockName: l.lockName, ClientName: clientName, + Message: l.message, }) if err == nil { atomic.StoreInt64(&l.token, resp.Token) @@ -123,3 +125,7 @@ func (l *ExclusiveLocker) ReleaseLock() { atomic.StoreInt64(&l.token, 0) atomic.StoreInt64(&l.lockTsNs, 0) } + +func (l *ExclusiveLocker) SetMessage(message string) { + l.message = message +} |
