aboutsummaryrefslogtreecommitdiff
path: root/weed
diff options
context:
space:
mode:
Diffstat (limited to 'weed')
-rw-r--r--weed/command/filer.go2
-rw-r--r--weed/command/filer_remote_gateway_buckets.go37
-rw-r--r--weed/command/filer_remote_sync.go7
-rw-r--r--weed/command/s3.go14
-rw-r--r--weed/command/scaffold/filer.toml8
-rw-r--r--weed/command/server.go2
-rw-r--r--weed/filer/filechunk_manifest.go9
-rw-r--r--weed/filer/mongodb/mongodb_store.go4
-rw-r--r--weed/filer/reader_at.go43
-rw-r--r--weed/filer/reader_pattern.go31
-rw-r--r--weed/filer/redis3/redis_sentinel_store.go49
-rw-r--r--weed/filesys/dir.go2
-rw-r--r--weed/filesys/dir_rename.go2
-rw-r--r--weed/filesys/dirty_pages_continuous.go21
-rw-r--r--weed/filesys/dirty_pages_temp_file.go35
-rw-r--r--weed/filesys/file.go2
-rw-r--r--weed/filesys/filehandle.go14
-rw-r--r--weed/filesys/page_writer/dirty_page_interval.go (renamed from weed/filesys/dirty_page_interval.go)2
-rw-r--r--weed/filesys/page_writer/dirty_page_interval_test.go (renamed from weed/filesys/dirty_page_interval_test.go)2
-rw-r--r--weed/filesys/page_writer/dirty_pages.go (renamed from weed/filesys/dirty_pages.go)4
-rw-r--r--weed/filesys/page_writer/dirty_pages_temp_interval.go (renamed from weed/filesys/dirty_pages_temp_interval.go)55
-rw-r--r--weed/filesys/page_writer/writer_pattern.go31
-rw-r--r--weed/filesys/wfs.go5
-rw-r--r--weed/filesys/wfs_write.go4
-rw-r--r--weed/messaging/broker/broker_server.go2
-rw-r--r--weed/pb/Makefile12
-rw-r--r--weed/pb/filer_pb/filer.pb.go954
-rw-r--r--weed/pb/filer_pb/filer_grpc.pb.go962
-rw-r--r--weed/pb/iam_pb/iam.pb.go53
-rw-r--r--weed/pb/iam_pb/iam_grpc.pb.go62
-rw-r--r--weed/pb/master.proto1
-rw-r--r--weed/pb/master_pb/master.pb.go831
-rw-r--r--weed/pb/master_pb/master_grpc.pb.go634
-rw-r--r--weed/pb/messaging_pb/messaging.pb.go338
-rw-r--r--weed/pb/messaging_pb/messaging_grpc.pb.go346
-rw-r--r--weed/pb/remote_pb/remote.pb.go9
-rw-r--r--weed/pb/volume_server_pb/volume_server.pb.go1780
-rw-r--r--weed/pb/volume_server_pb/volume_server_grpc.pb.go1788
-rw-r--r--weed/s3api/auth_credentials.go25
-rw-r--r--weed/s3api/chunked_reader_v4.go3
-rw-r--r--weed/s3api/filer_multipart.go2
-rw-r--r--weed/s3api/http/header.go27
-rw-r--r--weed/s3api/s3api_bucket_handlers.go11
-rw-r--r--weed/s3api/s3api_handlers.go1
-rw-r--r--weed/s3api/s3api_object_copy_handlers.go11
-rw-r--r--weed/s3api/s3api_object_handlers.go86
-rw-r--r--weed/s3api/s3api_object_handlers_postpolicy.go1
-rw-r--r--weed/s3api/s3api_object_multipart_handlers.go13
-rw-r--r--weed/s3api/s3api_object_tagging_handlers.go10
-rw-r--r--weed/s3api/s3api_objects_list_handlers.go4
-rw-r--r--weed/s3api/s3api_server.go2
-rw-r--r--weed/s3api/s3api_status_handlers.go7
-rw-r--r--weed/s3api/s3err/audit_fluent.go183
-rw-r--r--weed/s3api/s3err/error_handler.go2
-rw-r--r--weed/server/common.go16
-rw-r--r--weed/server/filer_grpc_server_rename.go2
-rw-r--r--weed/server/filer_grpc_server_sub_meta.go2
-rw-r--r--weed/server/filer_server.go2
-rw-r--r--weed/server/filer_server_handlers_read.go2
-rw-r--r--weed/server/filer_server_handlers_write_autochunk.go9
-rw-r--r--weed/server/master_grpc_server_admin.go13
-rw-r--r--weed/server/master_server.go17
-rw-r--r--weed/server/volume_server.go2
-rw-r--r--weed/server/volume_server_handlers_read.go4
-rw-r--r--weed/shell/command_collection_delete.go2
-rw-r--r--weed/shell/command_ec_balance.go2
-rw-r--r--weed/shell/command_ec_decode.go2
-rw-r--r--weed/shell/command_ec_encode.go2
-rw-r--r--weed/shell/command_ec_rebuild.go2
-rw-r--r--weed/shell/command_lock_unlock.go (renamed from weed/shell/command_fs_lock_unlock.go)0
-rw-r--r--weed/shell/command_volume_balance.go2
-rw-r--r--weed/shell/command_volume_check_disk.go16
-rw-r--r--weed/shell/command_volume_configure_replication.go2
-rw-r--r--weed/shell/command_volume_copy.go2
-rw-r--r--weed/shell/command_volume_delete.go2
-rw-r--r--weed/shell/command_volume_delete_empty.go2
-rw-r--r--weed/shell/command_volume_fix_replication.go62
-rw-r--r--weed/shell/command_volume_fix_replication_test.go138
-rw-r--r--weed/shell/command_volume_fsck.go5
-rw-r--r--weed/shell/command_volume_list.go61
-rw-r--r--weed/shell/command_volume_mark.go2
-rw-r--r--weed/shell/command_volume_mount.go2
-rw-r--r--weed/shell/command_volume_move.go2
-rw-r--r--weed/shell/command_volume_server_evacuate.go2
-rw-r--r--weed/shell/command_volume_server_leave.go2
-rw-r--r--weed/shell/command_volume_tier_download.go2
-rw-r--r--weed/shell/command_volume_tier_move.go2
-rw-r--r--weed/shell/command_volume_tier_upload.go2
-rw-r--r--weed/shell/command_volume_unmount.go2
-rw-r--r--weed/shell/command_volume_vacuum.go2
-rw-r--r--weed/shell/commands.go3
-rw-r--r--weed/storage/disk_location.go17
-rw-r--r--weed/storage/volume_loading.go3
-rw-r--r--weed/topology/topology.go2
-rw-r--r--weed/util/constants.go2
-rw-r--r--weed/util/http_util.go2
-rw-r--r--weed/wdclient/exclusive_locks/exclusive_locker.go6
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
+}