diff --git a/other/java/client/src/main/proto/filer.proto b/other/java/client/src/main/proto/filer.proto index 803a2ce32..1a5cfe79b 100644 --- a/other/java/client/src/main/proto/filer.proto +++ b/other/java/client/src/main/proto/filer.proto @@ -92,6 +92,12 @@ message ListEntriesResponse { Entry entry = 1; } +message RemoteEntry { + int64 last_modified_at = 1; + int64 size = 2; + string e_tag = 3; + string storage_name = 4; +} message Entry { string name = 1; bool is_directory = 2; @@ -102,12 +108,7 @@ message Entry { int32 hard_link_counter = 8; // only exists in hard link meta data bytes content = 9; // if not empty, the file content - message Remote { - int64 last_modified_at = 1; - int64 size = 2; - string e_tag = 3; - } - Remote remote = 10; + RemoteEntry remote_entry = 10; } message FullEntry { @@ -343,7 +344,9 @@ message LocateBrokerResponse { repeated Resource resources = 2; } +///////////////////////// // Key-Value operations +///////////////////////// message KvGetRequest { bytes key = 1; } @@ -359,7 +362,9 @@ message KvPutResponse { string error = 1; } +///////////////////////// // path-based configurations +///////////////////////// message FilerConf { int32 version = 1; message PathConf { @@ -375,6 +380,9 @@ message FilerConf { repeated PathConf locations = 2; } +///////////////////////// +// Remote Storage related +///////////////////////// message RemoteConf { string type = 1; string name = 2; @@ -383,3 +391,12 @@ message RemoteConf { string s3_region = 6; string s3_endpoint = 7; } + +message RemoteStorageMapping { + map mappings = 1; +} +message RemoteStorageLocation { + string name = 1; + string bucket = 2; + string path = 3; +} diff --git a/unmaintained/load_test/load_test_meta_tail/load_test_meta_tail.go b/unmaintained/load_test/load_test_meta_tail/load_test_meta_tail.go index 53cb2f912..c521ce33e 100644 --- a/unmaintained/load_test/load_test_meta_tail/load_test_meta_tail.go +++ b/unmaintained/load_test/load_test_meta_tail/load_test_meta_tail.go @@ -1,14 +1,12 @@ package main import ( - "context" "flag" "fmt" "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/pb" "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "google.golang.org/grpc" - "io" "strconv" "time" ) @@ -74,38 +72,9 @@ func startGenerateMetadata() { func startSubscribeMetadata(eachEntryFunc func(event *filer_pb.SubscribeMetadataResponse) error) { - lastTsNs := int64(0) + tailErr := pb.FollowMetadata(*tailFiler, grpc.WithInsecure(), "tail", + *dir, 0, 0, eachEntryFunc, false) - tailErr := pb.WithFilerClient(*tailFiler, grpc.WithInsecure(), func(client filer_pb.SeaweedFilerClient) error { - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ - ClientName: "tail", - PathPrefix: *dir, - SinceNs: lastTsNs, - }) - if err != nil { - return fmt.Errorf("listen: %v", err) - } - - for { - resp, listenErr := stream.Recv() - if listenErr == io.EOF { - return nil - } - if listenErr != nil { - return listenErr - } - if err = eachEntryFunc(resp); err != nil { - glog.V(0).Infof("tail last record:%+v", time.Unix(0, lastTsNs)) - return err - } - lastTsNs = resp.TsNs - } - - }) if tailErr != nil { fmt.Printf("tail %s: %v\n", *tailFiler, tailErr) } diff --git a/weed/Makefile b/weed/Makefile index c82735a0e..ad1f0b6f9 100644 --- a/weed/Makefile +++ b/weed/Makefile @@ -37,3 +37,7 @@ debug_s3: debug_filer_copy: go build -gcflags="all=-N -l" dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec weed -- -v=4 filer.backup -filer=localhost:8888 -filerProxy -timeAgo=10h + +debug_filer_remote_sync: + go build -gcflags="all=-N -l" + dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec weed -- -v=4 filer.remote.sync -filer="localhost:8888" -dir=/buckets/b2 -timeAgo=10000h diff --git a/weed/command/command.go b/weed/command/command.go index 9ae93fe61..02de2bd35 100644 --- a/weed/command/command.go +++ b/weed/command/command.go @@ -21,6 +21,7 @@ var Commands = []*Command{ cmdFilerCopy, cmdFilerMetaBackup, cmdFilerMetaTail, + cmdFilerRemoteSynchronize, cmdFilerReplicate, cmdFilerSynchronize, cmdFix, diff --git a/weed/command/filer.go b/weed/command/filer.go index 4fd2f9c72..ddee0852c 100644 --- a/weed/command/filer.go +++ b/weed/command/filer.go @@ -3,7 +3,6 @@ package command import ( "fmt" "net/http" - _ "net/http/pprof" "os" "strconv" "strings" diff --git a/weed/command/filer_backup.go b/weed/command/filer_backup.go index fc4dd8298..2828ccb39 100644 --- a/weed/command/filer_backup.go +++ b/weed/command/filer_backup.go @@ -1,16 +1,13 @@ package command import ( - "context" "fmt" "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/pb" - "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "github.com/chrislusf/seaweedfs/weed/replication/source" "github.com/chrislusf/seaweedfs/weed/security" "github.com/chrislusf/seaweedfs/weed/util" "google.golang.org/grpc" - "io" "time" ) @@ -110,48 +107,12 @@ func doFilerBackup(grpcDialOption grpc.DialOption, backupOption *FilerBackupOpti processEventFn := genProcessFunction(sourcePath, targetPath, dataSink, debug) - return pb.WithFilerClient(sourceFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ - ClientName: "backup_" + dataSink.GetName(), - PathPrefix: sourcePath, - SinceNs: startFrom.UnixNano(), - }) - if err != nil { - return fmt.Errorf("listen: %v", err) - } - - var counter int64 - var lastWriteTime time.Time - for { - resp, listenErr := stream.Recv() - - if listenErr == io.EOF { - return nil - } - if listenErr != nil { - return listenErr - } - - if err := processEventFn(resp); err != nil { - return fmt.Errorf("processEventFn: %v", err) - } - - counter++ - if lastWriteTime.Add(3 * time.Second).Before(time.Now()) { - glog.V(0).Infof("backup %s progressed to %v %0.2f/sec", sourceFiler, time.Unix(0, resp.TsNs), float64(counter)/float64(3)) - counter = 0 - lastWriteTime = time.Now() - if err := setOffset(grpcDialOption, sourceFiler, BackupKeyPrefix, int32(sinkId), resp.TsNs); err != nil { - return fmt.Errorf("setOffset: %v", err) - } - } - - } - + processEventFnWithOffset := pb.AddOffsetFunc(processEventFn, 3 * time.Second, func(counter int64, lastTsNs int64) error { + glog.V(0).Infof("backup %s progressed to %v %0.2f/sec", sourceFiler, time.Unix(0, lastTsNs), float64(counter)/float64(3)) + return setOffset(grpcDialOption, sourceFiler, BackupKeyPrefix, int32(sinkId), lastTsNs) }) + return pb.FollowMetadata(sourceFiler, grpcDialOption, "backup_" + dataSink.GetName(), + sourcePath, startFrom.UnixNano(), 0, processEventFnWithOffset, false) + } diff --git a/weed/command/filer_meta_backup.go b/weed/command/filer_meta_backup.go index 28bd367e7..108e76566 100644 --- a/weed/command/filer_meta_backup.go +++ b/weed/command/filer_meta_backup.go @@ -7,7 +7,6 @@ import ( "github.com/chrislusf/seaweedfs/weed/glog" "github.com/spf13/viper" "google.golang.org/grpc" - "io" "reflect" "time" @@ -190,48 +189,15 @@ func (metaBackup *FilerMetaBackupOptions) streamMetadataBackup() error { return nil } - tailErr := pb.WithFilerClient(*metaBackup.filerAddress, metaBackup.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ - ClientName: "meta_backup", - PathPrefix: *metaBackup.filerDirectory, - SinceNs: startTime.UnixNano(), - }) - if err != nil { - return fmt.Errorf("listen: %v", err) - } - - var counter int64 - var lastWriteTime time.Time - for { - resp, listenErr := stream.Recv() - if listenErr == io.EOF { - return nil - } - if listenErr != nil { - return listenErr - } - if err = eachEntryFunc(resp); err != nil { - return err - } - - counter++ - if lastWriteTime.Add(3 * time.Second).Before(time.Now()) { - glog.V(0).Infof("meta backup %s progressed to %v %0.2f/sec", *metaBackup.filerAddress, time.Unix(0, resp.TsNs), float64(counter)/float64(3)) - counter = 0 - lastWriteTime = time.Now() - if err2 := metaBackup.setOffset(lastWriteTime); err2 != nil { - return err2 - } - } - - } - + processEventFnWithOffset := pb.AddOffsetFunc(eachEntryFunc, 3 * time.Second, func(counter int64, lastTsNs int64) error { + lastTime := time.Unix(0, lastTsNs) + glog.V(0).Infof("meta backup %s progressed to %v %0.2f/sec", *metaBackup.filerAddress, lastTime, float64(counter)/float64(3)) + return metaBackup.setOffset(lastTime) }) - return tailErr + + return pb.FollowMetadata(*metaBackup.filerAddress, metaBackup.grpcDialOption, "meta_backup", + *metaBackup.filerDirectory, startTime.UnixNano(), 0, processEventFnWithOffset, false) + } func (metaBackup *FilerMetaBackupOptions) getOffset() (lastWriteTime time.Time, err error) { diff --git a/weed/command/filer_meta_tail.go b/weed/command/filer_meta_tail.go index 76699bb5e..28c0db99b 100644 --- a/weed/command/filer_meta_tail.go +++ b/weed/command/filer_meta_tail.go @@ -3,16 +3,15 @@ package command import ( "context" "fmt" + "github.com/chrislusf/seaweedfs/weed/pb" "github.com/golang/protobuf/jsonpb" jsoniter "github.com/json-iterator/go" "github.com/olivere/elastic/v7" - "io" "os" "path/filepath" "strings" "time" - "github.com/chrislusf/seaweedfs/weed/pb" "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "github.com/chrislusf/seaweedfs/weed/security" "github.com/chrislusf/seaweedfs/weed/util" @@ -104,37 +103,18 @@ func runFilerMetaTail(cmd *Command, args []string) bool { } } - tailErr := pb.WithFilerClient(*tailFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ - ClientName: "tail", - PathPrefix: *tailTarget, - SinceNs: time.Now().Add(-*tailStart).UnixNano(), - }) - if err != nil { - return fmt.Errorf("listen: %v", err) - } - - for { - resp, listenErr := stream.Recv() - if listenErr == io.EOF { + tailErr := pb.FollowMetadata(*tailFiler, grpcDialOption, "tail", + *tailTarget, time.Now().Add(-*tailStart).UnixNano(), 0, + func(resp *filer_pb.SubscribeMetadataResponse) error { + if !shouldPrint(resp) { return nil } - if listenErr != nil { - return listenErr - } - if !shouldPrint(resp) { - continue - } - if err = eachEntryFunc(resp); err != nil { + if err := eachEntryFunc(resp); err != nil { return err } - } + return nil + }, false) - }) if tailErr != nil { fmt.Printf("tail %s: %v\n", *tailFiler, tailErr) } diff --git a/weed/command/filer_remote_sync.go b/weed/command/filer_remote_sync.go new file mode 100644 index 000000000..4afb7c091 --- /dev/null +++ b/weed/command/filer_remote_sync.go @@ -0,0 +1,260 @@ +package command + +import ( + "context" + "fmt" + "github.com/chrislusf/seaweedfs/weed/filer" + "github.com/chrislusf/seaweedfs/weed/glog" + "github.com/chrislusf/seaweedfs/weed/pb" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "github.com/chrislusf/seaweedfs/weed/remote_storage" + "github.com/chrislusf/seaweedfs/weed/replication/source" + "github.com/chrislusf/seaweedfs/weed/security" + "github.com/chrislusf/seaweedfs/weed/util" + "github.com/golang/protobuf/proto" + "google.golang.org/grpc" + "strings" + "time" +) + +type RemoteSyncOptions struct { + filerAddress *string + grpcDialOption grpc.DialOption + readChunkFromFiler *bool + debug *bool + timeAgo *time.Duration + dir *string +} + +const ( + RemoteSyncKeyPrefix = "remote.sync." +) + +var _ = filer_pb.FilerClient(&RemoteSyncOptions{}) + +func (option *RemoteSyncOptions) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error { + return pb.WithFilerClient(*option.filerAddress, option.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { + return fn(client) + }) +} +func (option *RemoteSyncOptions) AdjustedUrl(location *filer_pb.Location) string { + return location.Url +} + +var ( + remoteSyncOptions RemoteSyncOptions +) + +func init() { + cmdFilerRemoteSynchronize.Run = runFilerRemoteSynchronize // break init cycle + 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.debug = cmdFilerRemoteSynchronize.Flag.Bool("debug", false, "debug mode to print out filer updated remote files") + 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\"") +} + +var cmdFilerRemoteSynchronize = &Command{ + UsageLine: "filer.remote.sync -filer=: -dir=/mount/s3_on_cloud", + Short: "resumeable continuously write back updates to remote storage if the directory is mounted to the remote storage", + Long: `resumeable continuously write back updates to remote storage if the directory is mounted to the remote storage + + filer.remote.sync listens on filer update events. + If any mounted remote file is updated, it will fetch the updated content, + and write to the remote storage. +`, +} + +func runFilerRemoteSynchronize(cmd *Command, args []string) bool { + + util.LoadConfiguration("security", false) + grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client") + remoteSyncOptions.grpcDialOption = grpcDialOption + + // read filer remote storage mount mappings + mappings, readErr := filer.ReadMountMappings(grpcDialOption, *remoteSyncOptions.filerAddress) + if readErr != nil { + fmt.Printf("read mount mapping: %v", readErr) + return false + } + + filerSource := &source.FilerSource{} + filerSource.DoInitialize( + *remoteSyncOptions.filerAddress, + pb.ServerToGrpcAddress(*remoteSyncOptions.filerAddress), + "/", // does not matter + *remoteSyncOptions.readChunkFromFiler, + ) + + var found bool + for dir, remoteStorageMountLocation := range mappings.Mappings { + if *remoteSyncOptions.dir == dir { + found = true + storageConf, readErr := filer.ReadRemoteStorageConf(grpcDialOption, *remoteSyncOptions.filerAddress, remoteStorageMountLocation.Name) + if readErr != nil { + fmt.Printf("read remote storage configuration for %s: %v", dir, readErr) + continue + } + fmt.Printf("synchronize %s to remote storage...\n", *remoteSyncOptions.dir) + if err := util.Retry("filer.remote.sync "+dir, func() error { + return followUpdatesAndUploadToRemote(&remoteSyncOptions, filerSource, dir, storageConf, remoteStorageMountLocation) + }); err != nil { + fmt.Printf("synchronize %s: %v\n", *remoteSyncOptions.dir, err) + } + break + } + } + if !found { + fmt.Printf("directory %s is not mounted to any remote storage\n", *remoteSyncOptions.dir) + return false + } + + return true +} + +func followUpdatesAndUploadToRemote(option *RemoteSyncOptions, filerSource *source.FilerSource, mountedDir string, remoteStorage *filer_pb.RemoteConf, remoteStorageMountLocation *filer_pb.RemoteStorageLocation) error { + + dirHash := util.HashStringToLong(mountedDir) + + // 1. specified by timeAgo + // 2. last offset timestamp for this directory + // 3. directory creation time + var lastOffsetTs time.Time + if *option.timeAgo == 0 { + mountedDirEntry, err := filer_pb.GetEntry(option, util.FullPath(mountedDir)) + if err != nil { + return fmt.Errorf("lookup %s: %v", mountedDir, err) + } + + lastOffsetTsNs, err := getOffset(option.grpcDialOption, *option.filerAddress, RemoteSyncKeyPrefix, int32(dirHash)) + if err == nil && mountedDirEntry.Attributes.Crtime < lastOffsetTsNs/1000000 { + lastOffsetTs = time.Unix(0, lastOffsetTsNs) + glog.V(0).Infof("resume from %v", lastOffsetTs) + } else { + lastOffsetTs = time.Unix(mountedDirEntry.Attributes.Crtime, 0) + } + } else { + lastOffsetTs = time.Now().Add(-*option.timeAgo) + } + + client, err := remote_storage.GetRemoteStorage(remoteStorage) + if err != nil { + return err + } + + eachEntryFunc := func(resp *filer_pb.SubscribeMetadataResponse) error { + message := resp.EventNotification + if message.OldEntry == nil && message.NewEntry == nil { + return nil + } + if message.OldEntry == nil && message.NewEntry != nil { + if len(message.NewEntry.Chunks) == 0 { + return nil + } + fmt.Printf("create: %+v\n", resp) + if !shouldSendToRemote(message.NewEntry) { + fmt.Printf("skipping creating: %+v\n", resp) + return nil + } + dest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(message.NewParentPath, message.NewEntry.Name), remoteStorageMountLocation) + reader := filer.NewChunkStreamReader(filerSource, message.NewEntry.Chunks) + remoteEntry, writeErr := client.WriteFile(dest, message.NewEntry, reader) + if writeErr != nil { + return writeErr + } + return updateLocalEntry(&remoteSyncOptions, message.NewParentPath, message.NewEntry, remoteEntry) + } + if message.OldEntry != nil && message.NewEntry == nil { + fmt.Printf("delete: %+v\n", resp) + dest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(resp.Directory, message.OldEntry.Name), remoteStorageMountLocation) + return client.DeleteFile(dest) + } + if message.OldEntry != nil && message.NewEntry != nil { + oldDest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(resp.Directory, message.OldEntry.Name), remoteStorageMountLocation) + dest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(message.NewParentPath, message.NewEntry.Name), remoteStorageMountLocation) + if !shouldSendToRemote(message.NewEntry) { + fmt.Printf("skipping updating: %+v\n", resp) + return nil + } + if resp.Directory == message.NewParentPath && message.OldEntry.Name == message.NewEntry.Name { + if isSameChunks(message.OldEntry.Chunks, message.NewEntry.Chunks) { + fmt.Printf("update meta: %+v\n", resp) + return client.UpdateFileMetadata(dest, message.NewEntry) + } + } + fmt.Printf("update: %+v\n", resp) + if err := client.DeleteFile(oldDest); err != nil { + return err + } + reader := filer.NewChunkStreamReader(filerSource, message.NewEntry.Chunks) + remoteEntry, writeErr := client.WriteFile(dest, message.NewEntry, reader) + if writeErr != nil { + return writeErr + } + return updateLocalEntry(&remoteSyncOptions, message.NewParentPath, message.NewEntry, remoteEntry) + } + + return nil + } + + processEventFnWithOffset := pb.AddOffsetFunc(eachEntryFunc, 3*time.Second, func(counter int64, lastTsNs int64) error { + lastTime := time.Unix(0, lastTsNs) + glog.V(0).Infof("remote sync %s progressed to %v %0.2f/sec", *option.filerAddress, lastTime, float64(counter)/float64(3)) + return setOffset(option.grpcDialOption, *option.filerAddress, RemoteSyncKeyPrefix, int32(dirHash), lastTsNs) + }) + + return pb.FollowMetadata(*option.filerAddress, option.grpcDialOption, + "filer.remote.sync", mountedDir, lastOffsetTs.UnixNano(), 0, processEventFnWithOffset, false) +} + +func toRemoteStorageLocation(mountDir, sourcePath util.FullPath, remoteMountLocation *filer_pb.RemoteStorageLocation) *filer_pb.RemoteStorageLocation { + var dest string + source := string(sourcePath[len(mountDir):]) + if strings.HasSuffix(remoteMountLocation.Path, "/") { + dest = remoteMountLocation.Path + source[1:] + } else { + dest = remoteMountLocation.Path + source + } + return &filer_pb.RemoteStorageLocation{ + Name: remoteMountLocation.Name, + Bucket: remoteMountLocation.Bucket, + Path: dest, + } +} + +func isSameChunks(a, b []*filer_pb.FileChunk) bool { + if len(a) != len(b) { + return false + } + for i := 0; i < len(a); i++ { + x, y := a[i], b[i] + if !proto.Equal(x, y) { + return false + } + } + return true +} + +func shouldSendToRemote(entry *filer_pb.Entry) bool { + if entry.RemoteEntry == nil { + return true + } + if entry.RemoteEntry.Size != int64(filer.FileSize(entry)) { + return true + } + if entry.RemoteEntry.LastModifiedAt < entry.Attributes.Mtime { + return true + } + return false +} + +func updateLocalEntry(filerClient filer_pb.FilerClient, dir string, entry *filer_pb.Entry, remoteEntry *filer_pb.RemoteEntry) error { + entry.RemoteEntry = remoteEntry + return filerClient.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { + _, err := client.UpdateEntry(context.Background(), &filer_pb.UpdateEntryRequest{ + Directory: dir, + Entry: entry, + }) + return err + }) +} \ No newline at end of file diff --git a/weed/command/filer_replication.go b/weed/command/filer_replication.go index 885c95540..bf0a3e140 100644 --- a/weed/command/filer_replication.go +++ b/weed/command/filer_replication.go @@ -7,12 +7,6 @@ import ( "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/replication" "github.com/chrislusf/seaweedfs/weed/replication/sink" - _ "github.com/chrislusf/seaweedfs/weed/replication/sink/azuresink" - _ "github.com/chrislusf/seaweedfs/weed/replication/sink/b2sink" - _ "github.com/chrislusf/seaweedfs/weed/replication/sink/filersink" - _ "github.com/chrislusf/seaweedfs/weed/replication/sink/gcssink" - _ "github.com/chrislusf/seaweedfs/weed/replication/sink/localsink" - _ "github.com/chrislusf/seaweedfs/weed/replication/sink/s3sink" "github.com/chrislusf/seaweedfs/weed/replication/sub" "github.com/chrislusf/seaweedfs/weed/util" ) diff --git a/weed/command/filer_sync.go b/weed/command/filer_sync.go index 7cfc8a7fe..a20f17201 100644 --- a/weed/command/filer_sync.go +++ b/weed/command/filer_sync.go @@ -15,7 +15,6 @@ import ( "github.com/chrislusf/seaweedfs/weed/util" "github.com/chrislusf/seaweedfs/weed/util/grace" "google.golang.org/grpc" - "io" "strings" "time" ) @@ -166,50 +165,14 @@ func doSubscribeFilerMetaChanges(grpcDialOption grpc.DialOption, sourceFiler, so return persistEventFn(resp) } - return pb.WithFilerClient(sourceFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ - ClientName: "syncTo_" + targetFiler, - PathPrefix: sourcePath, - SinceNs: sourceFilerOffsetTsNs, - Signature: targetFilerSignature, - }) - if err != nil { - return fmt.Errorf("listen: %v", err) - } - - var counter int64 - var lastWriteTime time.Time - for { - resp, listenErr := stream.Recv() - if listenErr == io.EOF { - return nil - } - if listenErr != nil { - return listenErr - } - - if err := processEventFn(resp); err != nil { - return err - } - - counter++ - if lastWriteTime.Add(3 * time.Second).Before(time.Now()) { - glog.V(0).Infof("sync %s to %s progressed to %v %0.2f/sec", sourceFiler, targetFiler, time.Unix(0, resp.TsNs), float64(counter)/float64(3)) - counter = 0 - lastWriteTime = time.Now() - if err := setOffset(grpcDialOption, targetFiler, SyncKeyPrefix, sourceFilerSignature, resp.TsNs); err != nil { - return err - } - } - - } - + processEventFnWithOffset := pb.AddOffsetFunc(processEventFn, 3 * time.Second, func(counter int64, lastTsNs int64) error { + glog.V(0).Infof("sync %s to %s progressed to %v %0.2f/sec", sourceFiler, targetFiler, time.Unix(0, lastTsNs), float64(counter)/float64(3)) + return setOffset(grpcDialOption, targetFiler, SyncKeyPrefix, sourceFilerSignature, lastTsNs) }) + return pb.FollowMetadata(sourceFiler, grpcDialOption, "syncTo_" + targetFiler, + sourcePath, sourceFilerOffsetTsNs, targetFilerSignature, processEventFnWithOffset, false) + } const ( diff --git a/weed/command/imports.go b/weed/command/imports.go new file mode 100644 index 000000000..d7ade1379 --- /dev/null +++ b/weed/command/imports.go @@ -0,0 +1,31 @@ +package command + +import ( + _ "net/http/pprof" + + _ "github.com/chrislusf/seaweedfs/weed/remote_storage/s3" + + _ "github.com/chrislusf/seaweedfs/weed/replication/sink/azuresink" + _ "github.com/chrislusf/seaweedfs/weed/replication/sink/b2sink" + _ "github.com/chrislusf/seaweedfs/weed/replication/sink/filersink" + _ "github.com/chrislusf/seaweedfs/weed/replication/sink/gcssink" + _ "github.com/chrislusf/seaweedfs/weed/replication/sink/localsink" + _ "github.com/chrislusf/seaweedfs/weed/replication/sink/s3sink" + + _ "github.com/chrislusf/seaweedfs/weed/filer/cassandra" + _ "github.com/chrislusf/seaweedfs/weed/filer/elastic/v7" + _ "github.com/chrislusf/seaweedfs/weed/filer/etcd" + _ "github.com/chrislusf/seaweedfs/weed/filer/hbase" + _ "github.com/chrislusf/seaweedfs/weed/filer/leveldb" + _ "github.com/chrislusf/seaweedfs/weed/filer/leveldb2" + _ "github.com/chrislusf/seaweedfs/weed/filer/leveldb3" + _ "github.com/chrislusf/seaweedfs/weed/filer/mongodb" + _ "github.com/chrislusf/seaweedfs/weed/filer/mysql" + _ "github.com/chrislusf/seaweedfs/weed/filer/mysql2" + _ "github.com/chrislusf/seaweedfs/weed/filer/postgres" + _ "github.com/chrislusf/seaweedfs/weed/filer/postgres2" + _ "github.com/chrislusf/seaweedfs/weed/filer/redis" + _ "github.com/chrislusf/seaweedfs/weed/filer/redis2" + _ "github.com/chrislusf/seaweedfs/weed/filer/sqlite" + +) \ No newline at end of file diff --git a/weed/command/shell.go b/weed/command/shell.go index c9976e809..4a9f4b027 100644 --- a/weed/command/shell.go +++ b/weed/command/shell.go @@ -55,6 +55,7 @@ func runShell(command *Command, args []string) bool { var err error shellOptions.FilerHost, shellOptions.FilerPort, err = util.ParseHostPort(*shellInitialFiler) + shellOptions.FilerAddress = *shellInitialFiler if err != nil { fmt.Printf("failed to parse filer %s: %v\n", *shellInitialFiler, err) return false diff --git a/weed/filer/entry.go b/weed/filer/entry.go index ede58a384..7673365fb 100644 --- a/weed/filer/entry.go +++ b/weed/filer/entry.go @@ -42,7 +42,7 @@ type Entry struct { HardLinkId HardLinkId HardLinkCounter int32 Content []byte - Remote *filer_pb.Entry_Remote + Remote *filer_pb.RemoteEntry } func (entry *Entry) Size() uint64 { @@ -78,7 +78,7 @@ func (entry *Entry) ToExistingProtoEntry(message *filer_pb.Entry) { message.HardLinkId = entry.HardLinkId message.HardLinkCounter = entry.HardLinkCounter message.Content = entry.Content - message.Remote = entry.Remote + message.RemoteEntry = entry.Remote } func FromPbEntryToExistingEntry(message *filer_pb.Entry, fsEntry *Entry) { @@ -88,7 +88,7 @@ func FromPbEntryToExistingEntry(message *filer_pb.Entry, fsEntry *Entry) { fsEntry.HardLinkId = HardLinkId(message.HardLinkId) fsEntry.HardLinkCounter = message.HardLinkCounter fsEntry.Content = message.Content - fsEntry.Remote = message.Remote + fsEntry.Remote = message.RemoteEntry } func (entry *Entry) ToProtoFullEntry() *filer_pb.FullEntry { diff --git a/weed/filer/filer.go b/weed/filer/filer.go index 862f98496..1a20abefc 100644 --- a/weed/filer/filer.go +++ b/weed/filer/filer.go @@ -42,6 +42,7 @@ type Filer struct { MetaAggregator *MetaAggregator Signature int32 FilerConf *FilerConf + RemoteStorage *FilerRemoteStorage } func NewFiler(masters []string, grpcDialOption grpc.DialOption, @@ -51,6 +52,7 @@ func NewFiler(masters []string, grpcDialOption grpc.DialOption, fileIdDeletionQueue: util.NewUnboundedQueue(), GrpcDialOption: grpcDialOption, FilerConf: NewFilerConf(), + RemoteStorage: NewFilerRemoteStorage(), } f.LocalMetaLogBuffer = log_buffer.NewLogBuffer("local", LogFlushInterval, f.logFlushFunc, notifyFn) f.metaLogCollection = collection diff --git a/weed/filer/filer_on_meta_event.go b/weed/filer/filer_on_meta_event.go index 5717b2b09..34ac5321a 100644 --- a/weed/filer/filer_on_meta_event.go +++ b/weed/filer/filer_on_meta_event.go @@ -12,6 +12,7 @@ import ( // onMetadataChangeEvent is triggered after filer processed change events from local or remote filers func (f *Filer) onMetadataChangeEvent(event *filer_pb.SubscribeMetadataResponse) { f.maybeReloadFilerConfiguration(event) + f.maybeReloadRemoteStorageConfigurationAndMapping(event) f.onBucketEvents(event) } @@ -84,3 +85,16 @@ func (f *Filer) LoadFilerConf() { } f.FilerConf = fc } + +//////////////////////////////////// +// load and maintain remote storages +//////////////////////////////////// +func (f *Filer) LoadRemoteStorageConfAndMapping() { + if err := f.RemoteStorage.LoadRemoteStorageConfigurationsAndMapping(f); err != nil { + glog.Errorf("read remote conf and mapping: %v", err) + return + } +} +func (f *Filer) maybeReloadRemoteStorageConfigurationAndMapping(event *filer_pb.SubscribeMetadataResponse) { + // FIXME add reloading +} diff --git a/weed/filer/filer_remote_storage.go b/weed/filer/filer_remote_storage.go new file mode 100644 index 000000000..573dcf3e7 --- /dev/null +++ b/weed/filer/filer_remote_storage.go @@ -0,0 +1,182 @@ +package filer + +import ( + "context" + "fmt" + "github.com/chrislusf/seaweedfs/weed/pb" + "github.com/chrislusf/seaweedfs/weed/remote_storage" + "github.com/chrislusf/seaweedfs/weed/util" + "github.com/golang/protobuf/proto" + "google.golang.org/grpc" + "math" + "strings" + + "github.com/chrislusf/seaweedfs/weed/glog" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "github.com/viant/ptrie" +) + +const REMOTE_STORAGE_CONF_SUFFIX = ".conf" +const REMOTE_STORAGE_MOUNT_FILE = "mount.mapping" + +type FilerRemoteStorage struct { + rules ptrie.Trie + storageNameToConf map[string]*filer_pb.RemoteConf +} + +func NewFilerRemoteStorage() (rs *FilerRemoteStorage) { + rs = &FilerRemoteStorage{ + rules: ptrie.New(), + storageNameToConf: make(map[string]*filer_pb.RemoteConf), + } + return rs +} + +func (rs *FilerRemoteStorage) LoadRemoteStorageConfigurationsAndMapping(filer *Filer) (err error) { + // execute this on filer + + entries, _, err := filer.ListDirectoryEntries(context.Background(), DirectoryEtcRemote, "", false, math.MaxInt64, "", "", "") + if err != nil { + if err == filer_pb.ErrNotFound { + return nil + } + glog.Errorf("read remote storage %s: %v", DirectoryEtcRemote, err) + return + } + + for _, entry := range entries { + if entry.Name() == REMOTE_STORAGE_MOUNT_FILE { + if err := rs.loadRemoteStorageMountMapping(entry.Content); err != nil { + return err + } + continue + } + if !strings.HasSuffix(entry.Name(), REMOTE_STORAGE_CONF_SUFFIX) { + return nil + } + conf := &filer_pb.RemoteConf{} + if err := proto.Unmarshal(entry.Content, conf); err != nil { + return fmt.Errorf("unmarshal %s/%s: %v", DirectoryEtcRemote, entry.Name(), err) + } + rs.storageNameToConf[conf.Name] = conf + } + return nil +} + +func (rs *FilerRemoteStorage) loadRemoteStorageMountMapping(data []byte) (err error) { + mappings := &filer_pb.RemoteStorageMapping{} + if err := proto.Unmarshal(data, mappings); err != nil { + return fmt.Errorf("unmarshal %s/%s: %v", DirectoryEtcRemote, REMOTE_STORAGE_MOUNT_FILE, err) + } + for dir, storageLocation := range mappings.Mappings { + rs.mapDirectoryToRemoteStorage(util.FullPath(dir), storageLocation) + } + return nil +} + +func (rs *FilerRemoteStorage) mapDirectoryToRemoteStorage(dir util.FullPath, loc *filer_pb.RemoteStorageLocation) { + rs.rules.Put([]byte(dir+"/"), loc) +} + +func (rs *FilerRemoteStorage) FindMountDirectory(p util.FullPath) (mountDir util.FullPath, remoteLocation *filer_pb.RemoteStorageLocation) { + rs.rules.MatchPrefix([]byte(p), func(key []byte, value interface{}) bool { + mountDir = util.FullPath(string(key[:len(key)-1])) + remoteLocation = value.(*filer_pb.RemoteStorageLocation) + return true + }) + return +} + +func (rs *FilerRemoteStorage) FindRemoteStorageClient(p util.FullPath) (client remote_storage.RemoteStorageClient, remoteConf *filer_pb.RemoteConf, found bool) { + var storageLocation *filer_pb.RemoteStorageLocation + rs.rules.MatchPrefix([]byte(p), func(key []byte, value interface{}) bool { + storageLocation = value.(*filer_pb.RemoteStorageLocation) + return true + }) + + if storageLocation == nil { + found = false + return + } + + return rs.GetRemoteStorageClient(storageLocation.Name) +} + +func (rs *FilerRemoteStorage) GetRemoteStorageClient(storageName string) (client remote_storage.RemoteStorageClient, remoteConf *filer_pb.RemoteConf, found bool) { + remoteConf, found = rs.storageNameToConf[storageName] + if !found { + return + } + + var err error + if client, err = remote_storage.GetRemoteStorage(remoteConf); err == nil { + found = true + return + } + return +} + +func UnmarshalRemoteStorageMappings(oldContent []byte) (mappings *filer_pb.RemoteStorageMapping, err error) { + mappings = &filer_pb.RemoteStorageMapping{ + Mappings: make(map[string]*filer_pb.RemoteStorageLocation), + } + if len(oldContent) > 0 { + if err = proto.Unmarshal(oldContent, mappings); err != nil { + glog.Warningf("unmarshal existing mappings: %v", err) + } + } + return +} + +func AddRemoteStorageMapping(oldContent []byte, dir string, storageLocation *filer_pb.RemoteStorageLocation) (newContent []byte, err error) { + mappings, unmarshalErr := UnmarshalRemoteStorageMappings(oldContent) + if unmarshalErr != nil { + // skip + } + + // set the new mapping + mappings.Mappings[dir] = storageLocation + + if newContent, err = proto.Marshal(mappings); err != nil { + return oldContent, fmt.Errorf("marshal mappings: %v", err) + } + + return +} + + +func ReadMountMappings(grpcDialOption grpc.DialOption, filerAddress string) (mappings *filer_pb.RemoteStorageMapping, readErr error) { + var oldContent []byte + if readErr = pb.WithFilerClient(filerAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { + oldContent, readErr = ReadInsideFiler(client, DirectoryEtcRemote, REMOTE_STORAGE_MOUNT_FILE) + return readErr + }); readErr != nil { + return nil, readErr + } + + mappings, readErr = UnmarshalRemoteStorageMappings(oldContent) + if readErr != nil { + return nil, fmt.Errorf("unmarshal mappings: %v", readErr) + } + + return +} + +func ReadRemoteStorageConf(grpcDialOption grpc.DialOption, filerAddress string, storageName string) (conf *filer_pb.RemoteConf, readErr error) { + var oldContent []byte + if readErr = pb.WithFilerClient(filerAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { + oldContent, readErr = ReadInsideFiler(client, DirectoryEtcRemote, storageName+REMOTE_STORAGE_CONF_SUFFIX) + return readErr + }); readErr != nil { + return nil, readErr + } + + // unmarshal storage configuration + conf = &filer_pb.RemoteConf{} + if unMarshalErr := proto.Unmarshal(oldContent, conf); unMarshalErr != nil { + readErr = fmt.Errorf("unmarshal %s/%s: %v", DirectoryEtcRemote, storageName+REMOTE_STORAGE_CONF_SUFFIX, unMarshalErr) + return + } + + return +} diff --git a/weed/filer/filer_remote_storage_test.go b/weed/filer/filer_remote_storage_test.go new file mode 100644 index 000000000..427cd5a8a --- /dev/null +++ b/weed/filer/filer_remote_storage_test.go @@ -0,0 +1,34 @@ +package filer + +import ( + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "github.com/stretchr/testify/assert" + "testing" +) + +func TestFilerRemoteStorage_FindRemoteStorageClient(t *testing.T) { + conf := &filer_pb.RemoteConf{ + Name: "s7", + Type: "s3", + } + rs := NewFilerRemoteStorage() + rs.storageNameToConf[conf.Name] = conf + + rs.mapDirectoryToRemoteStorage("/a/b/c", &filer_pb.RemoteStorageLocation{ + Name: "s7", + Bucket: "some", + Path: "/dir", + }) + + _, _, found := rs.FindRemoteStorageClient("/a/b/c/d/e/f") + assert.Equal(t, true, found, "find storage client") + + _, _, found2 := rs.FindRemoteStorageClient("/a/b") + assert.Equal(t, false, found2, "should not find storage client") + + _, _, found3 := rs.FindRemoteStorageClient("/a/b/c") + assert.Equal(t, false, found3, "should not find storage client") + + _, _, found4 := rs.FindRemoteStorageClient("/a/b/cc") + assert.Equal(t, false, found4, "should not find storage client") +} \ No newline at end of file diff --git a/weed/filer/filer_search.go b/weed/filer/filer_search.go index 2ee29be25..2e0336da8 100644 --- a/weed/filer/filer_search.go +++ b/weed/filer/filer_search.go @@ -3,6 +3,7 @@ package filer import ( "context" "github.com/chrislusf/seaweedfs/weed/util" + "math" "path/filepath" "strings" ) @@ -27,6 +28,10 @@ func (f *Filer) ListDirectoryEntries(ctx context.Context, p util.FullPath, start return true }) + if limit == math.MaxInt64 { + limit = math.MaxInt64 - 1 + } + hasMore = int64(len(entries)) >= limit+1 if hasMore { entries = entries[:limit] diff --git a/weed/filer/read_remote.go b/weed/filer/read_remote.go new file mode 100644 index 000000000..d9423ceda --- /dev/null +++ b/weed/filer/read_remote.go @@ -0,0 +1,29 @@ +package filer + +import ( + "fmt" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" +) + +func (entry *Entry) IsInRemoteOnly() bool { + return len(entry.Chunks) == 0 && entry.Remote != nil && entry.Remote.Size > 0 +} + +func (f *Filer) ReadRemote(entry *Entry, offset int64, size int64) (data[]byte, err error) { + client, _, found := f.RemoteStorage.GetRemoteStorageClient(entry.Remote.StorageName) + if !found { + return nil, fmt.Errorf("remote storage %v not found", entry.Remote.StorageName) + } + + mountDir, remoteLoation := f.RemoteStorage.FindMountDirectory(entry.FullPath) + + remoteFullPath := remoteLoation.Path + string(entry.FullPath[len(mountDir):]) + + sourceLoc := &filer_pb.RemoteStorageLocation{ + Name: remoteLoation.Name, + Bucket: remoteLoation.Bucket, + Path: remoteFullPath, + } + + return client.ReadFile(sourceLoc, offset, size) +} diff --git a/weed/filer/stream.go b/weed/filer/stream.go index 3859f9a67..503e6b23f 100644 --- a/weed/filer/stream.go +++ b/weed/filer/stream.go @@ -91,6 +91,7 @@ func ReadAll(masterClient *wdclient.MasterClient, chunks []*filer_pb.FileChunk) type ChunkStreamReader struct { chunkViews []*ChunkView totalSize int64 + logicOffset int64 buffer []byte bufferOffset int64 bufferPos int @@ -137,8 +138,7 @@ func NewChunkStreamReader(filerClient filer_pb.FilerClient, chunks []*filer_pb.F } func (c *ChunkStreamReader) ReadAt(p []byte, off int64) (n int, err error) { - _, err = c.Seek(off, io.SeekStart) - if err != nil { + if err = c.prepareBufferFor(c.logicOffset); err != nil { return } return c.Read(p) @@ -151,12 +151,15 @@ func (c *ChunkStreamReader) Read(p []byte) (n int, err error) { return n, io.EOF } chunkView := c.chunkViews[c.nextChunkViewIndex] - c.fetchChunkToBuffer(chunkView) + if err = c.fetchChunkToBuffer(chunkView); err != nil { + return + } c.nextChunkViewIndex++ } t := copy(p[n:], c.buffer[c.bufferPos:]) c.bufferPos += t n += t + c.logicOffset += int64(t) } return } @@ -171,19 +174,26 @@ func (c *ChunkStreamReader) Seek(offset int64, whence int) (int64, error) { switch whence { case io.SeekStart: case io.SeekCurrent: - offset += c.bufferOffset + int64(c.bufferPos) + offset += c.logicOffset case io.SeekEnd: offset = c.totalSize + offset } if offset > c.totalSize { err = io.ErrUnexpectedEOF + } else { + c.logicOffset = offset } + return offset, err + +} + +func (c *ChunkStreamReader) prepareBufferFor(offset int64) (err error) { // stay in the same chunk if !c.isBufferEmpty() { if c.bufferOffset <= offset && offset < c.bufferOffset+int64(len(c.buffer)) { c.bufferPos = int(offset - c.bufferOffset) - return offset, nil + return nil } } @@ -192,23 +202,21 @@ func (c *ChunkStreamReader) Seek(offset int64, whence int) (int64, error) { return c.chunkViews[i].LogicOffset <= offset }) if currentChunkIndex == len(c.chunkViews) { - return 0, io.EOF + return io.EOF } // positioning within the new chunk chunk := c.chunkViews[currentChunkIndex] if chunk.LogicOffset <= offset && offset < chunk.LogicOffset+int64(chunk.Size) { if c.isBufferEmpty() || c.bufferOffset != chunk.LogicOffset { - c.fetchChunkToBuffer(chunk) + if err = c.fetchChunkToBuffer(chunk); err != nil { + return + } c.nextChunkViewIndex = currentChunkIndex + 1 } c.bufferPos = int(offset - c.bufferOffset) - } else { - return 0, io.ErrUnexpectedEOF } - - return offset, err - + return } func (c *ChunkStreamReader) fetchChunkToBuffer(chunkView *ChunkView) error { diff --git a/weed/filesys/meta_cache/meta_cache_subscribe.go b/weed/filesys/meta_cache/meta_cache_subscribe.go index 747ac3cb9..c650b8024 100644 --- a/weed/filesys/meta_cache/meta_cache_subscribe.go +++ b/weed/filesys/meta_cache/meta_cache_subscribe.go @@ -2,12 +2,9 @@ package meta_cache import ( "context" - "fmt" - "io" - "time" - "github.com/chrislusf/seaweedfs/weed/filer" "github.com/chrislusf/seaweedfs/weed/glog" + "github.com/chrislusf/seaweedfs/weed/pb" "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "github.com/chrislusf/seaweedfs/weed/util" ) @@ -62,38 +59,8 @@ func SubscribeMetaEvents(mc *MetaCache, selfSignature int32, client filer_pb.Fil } - for { - err := client.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ - ClientName: "mount", - PathPrefix: dir, - SinceNs: lastTsNs, - Signature: selfSignature, - }) - if err != nil { - return fmt.Errorf("subscribe: %v", err) - } + return util.Retry("followMetaUpdates", func() error { + return pb.WithFilerClientFollowMetadata(client, "mount", dir, lastTsNs, selfSignature, processEventFn, true) + }) - for { - resp, listenErr := stream.Recv() - if listenErr == io.EOF { - return nil - } - if listenErr != nil { - return listenErr - } - - if err := processEventFn(resp); err != nil { - glog.Fatalf("process %v: %v", resp, err) - } - lastTsNs = resp.TsNs - } - }) - if err != nil { - glog.Errorf("subscribing filer meta change: %v", err) - } - time.Sleep(time.Second) - } } diff --git a/weed/pb/filer.proto b/weed/pb/filer.proto index 803a2ce32..1a5cfe79b 100644 --- a/weed/pb/filer.proto +++ b/weed/pb/filer.proto @@ -92,6 +92,12 @@ message ListEntriesResponse { Entry entry = 1; } +message RemoteEntry { + int64 last_modified_at = 1; + int64 size = 2; + string e_tag = 3; + string storage_name = 4; +} message Entry { string name = 1; bool is_directory = 2; @@ -102,12 +108,7 @@ message Entry { int32 hard_link_counter = 8; // only exists in hard link meta data bytes content = 9; // if not empty, the file content - message Remote { - int64 last_modified_at = 1; - int64 size = 2; - string e_tag = 3; - } - Remote remote = 10; + RemoteEntry remote_entry = 10; } message FullEntry { @@ -343,7 +344,9 @@ message LocateBrokerResponse { repeated Resource resources = 2; } +///////////////////////// // Key-Value operations +///////////////////////// message KvGetRequest { bytes key = 1; } @@ -359,7 +362,9 @@ message KvPutResponse { string error = 1; } +///////////////////////// // path-based configurations +///////////////////////// message FilerConf { int32 version = 1; message PathConf { @@ -375,6 +380,9 @@ message FilerConf { repeated PathConf locations = 2; } +///////////////////////// +// Remote Storage related +///////////////////////// message RemoteConf { string type = 1; string name = 2; @@ -383,3 +391,12 @@ message RemoteConf { string s3_region = 6; string s3_endpoint = 7; } + +message RemoteStorageMapping { + map mappings = 1; +} +message RemoteStorageLocation { + string name = 1; + string bucket = 2; + string path = 3; +} diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go index 2724bc7b0..495afb21a 100644 --- a/weed/pb/filer_pb/filer.pb.go +++ b/weed/pb/filer_pb/filer.pb.go @@ -257,6 +257,77 @@ func (x *ListEntriesResponse) GetEntry() *Entry { return nil } +type RemoteEntry struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + LastModifiedAt int64 `protobuf:"varint,1,opt,name=last_modified_at,json=lastModifiedAt,proto3" json:"last_modified_at,omitempty"` + Size int64 `protobuf:"varint,2,opt,name=size,proto3" json:"size,omitempty"` + ETag string `protobuf:"bytes,3,opt,name=e_tag,json=eTag,proto3" json:"e_tag,omitempty"` + StorageName string `protobuf:"bytes,4,opt,name=storage_name,json=storageName,proto3" json:"storage_name,omitempty"` +} + +func (x *RemoteEntry) Reset() { + *x = RemoteEntry{} + if protoimpl.UnsafeEnabled { + mi := &file_filer_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RemoteEntry) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoteEntry) ProtoMessage() {} + +func (x *RemoteEntry) ProtoReflect() protoreflect.Message { + mi := &file_filer_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RemoteEntry.ProtoReflect.Descriptor instead. +func (*RemoteEntry) Descriptor() ([]byte, []int) { + return file_filer_proto_rawDescGZIP(), []int{4} +} + +func (x *RemoteEntry) GetLastModifiedAt() int64 { + if x != nil { + return x.LastModifiedAt + } + return 0 +} + +func (x *RemoteEntry) GetSize() int64 { + if x != nil { + return x.Size + } + return 0 +} + +func (x *RemoteEntry) GetETag() string { + if x != nil { + return x.ETag + } + return "" +} + +func (x *RemoteEntry) GetStorageName() string { + if x != nil { + return x.StorageName + } + return "" +} + type Entry struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -270,13 +341,13 @@ type Entry struct { HardLinkId []byte `protobuf:"bytes,7,opt,name=hard_link_id,json=hardLinkId,proto3" json:"hard_link_id,omitempty"` HardLinkCounter int32 `protobuf:"varint,8,opt,name=hard_link_counter,json=hardLinkCounter,proto3" json:"hard_link_counter,omitempty"` // only exists in hard link meta data Content []byte `protobuf:"bytes,9,opt,name=content,proto3" json:"content,omitempty"` // if not empty, the file content - Remote *Entry_Remote `protobuf:"bytes,10,opt,name=remote,proto3" json:"remote,omitempty"` + RemoteEntry *RemoteEntry `protobuf:"bytes,10,opt,name=remote_entry,json=remoteEntry,proto3" json:"remote_entry,omitempty"` } func (x *Entry) Reset() { *x = Entry{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[4] + mi := &file_filer_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -289,7 +360,7 @@ func (x *Entry) String() string { func (*Entry) ProtoMessage() {} func (x *Entry) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[4] + mi := &file_filer_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -302,7 +373,7 @@ func (x *Entry) ProtoReflect() protoreflect.Message { // Deprecated: Use Entry.ProtoReflect.Descriptor instead. func (*Entry) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{4} + return file_filer_proto_rawDescGZIP(), []int{5} } func (x *Entry) GetName() string { @@ -361,9 +432,9 @@ func (x *Entry) GetContent() []byte { return nil } -func (x *Entry) GetRemote() *Entry_Remote { +func (x *Entry) GetRemoteEntry() *RemoteEntry { if x != nil { - return x.Remote + return x.RemoteEntry } return nil } @@ -380,7 +451,7 @@ type FullEntry struct { func (x *FullEntry) Reset() { *x = FullEntry{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[5] + mi := &file_filer_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -393,7 +464,7 @@ func (x *FullEntry) String() string { func (*FullEntry) ProtoMessage() {} func (x *FullEntry) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[5] + mi := &file_filer_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -406,7 +477,7 @@ func (x *FullEntry) ProtoReflect() protoreflect.Message { // Deprecated: Use FullEntry.ProtoReflect.Descriptor instead. func (*FullEntry) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{5} + return file_filer_proto_rawDescGZIP(), []int{6} } func (x *FullEntry) GetDir() string { @@ -439,7 +510,7 @@ type EventNotification struct { func (x *EventNotification) Reset() { *x = EventNotification{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[6] + mi := &file_filer_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -452,7 +523,7 @@ func (x *EventNotification) String() string { func (*EventNotification) ProtoMessage() {} func (x *EventNotification) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[6] + mi := &file_filer_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -465,7 +536,7 @@ func (x *EventNotification) ProtoReflect() protoreflect.Message { // Deprecated: Use EventNotification.ProtoReflect.Descriptor instead. func (*EventNotification) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{6} + return file_filer_proto_rawDescGZIP(), []int{7} } func (x *EventNotification) GetOldEntry() *Entry { @@ -531,7 +602,7 @@ type FileChunk struct { func (x *FileChunk) Reset() { *x = FileChunk{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[7] + mi := &file_filer_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -544,7 +615,7 @@ func (x *FileChunk) String() string { func (*FileChunk) ProtoMessage() {} func (x *FileChunk) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[7] + mi := &file_filer_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -557,7 +628,7 @@ func (x *FileChunk) ProtoReflect() protoreflect.Message { // Deprecated: Use FileChunk.ProtoReflect.Descriptor instead. func (*FileChunk) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{7} + return file_filer_proto_rawDescGZIP(), []int{8} } func (x *FileChunk) GetFileId() string { @@ -648,7 +719,7 @@ type FileChunkManifest struct { func (x *FileChunkManifest) Reset() { *x = FileChunkManifest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[8] + mi := &file_filer_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -661,7 +732,7 @@ func (x *FileChunkManifest) String() string { func (*FileChunkManifest) ProtoMessage() {} func (x *FileChunkManifest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[8] + mi := &file_filer_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -674,7 +745,7 @@ func (x *FileChunkManifest) ProtoReflect() protoreflect.Message { // Deprecated: Use FileChunkManifest.ProtoReflect.Descriptor instead. func (*FileChunkManifest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{8} + return file_filer_proto_rawDescGZIP(), []int{9} } func (x *FileChunkManifest) GetChunks() []*FileChunk { @@ -697,7 +768,7 @@ type FileId struct { func (x *FileId) Reset() { *x = FileId{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[9] + mi := &file_filer_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -710,7 +781,7 @@ func (x *FileId) String() string { func (*FileId) ProtoMessage() {} func (x *FileId) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[9] + mi := &file_filer_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -723,7 +794,7 @@ func (x *FileId) ProtoReflect() protoreflect.Message { // Deprecated: Use FileId.ProtoReflect.Descriptor instead. func (*FileId) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{9} + return file_filer_proto_rawDescGZIP(), []int{10} } func (x *FileId) GetVolumeId() uint32 { @@ -772,7 +843,7 @@ type FuseAttributes struct { func (x *FuseAttributes) Reset() { *x = FuseAttributes{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[10] + mi := &file_filer_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -785,7 +856,7 @@ func (x *FuseAttributes) String() string { func (*FuseAttributes) ProtoMessage() {} func (x *FuseAttributes) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[10] + mi := &file_filer_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -798,7 +869,7 @@ func (x *FuseAttributes) ProtoReflect() protoreflect.Message { // Deprecated: Use FuseAttributes.ProtoReflect.Descriptor instead. func (*FuseAttributes) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{10} + return file_filer_proto_rawDescGZIP(), []int{11} } func (x *FuseAttributes) GetFileSize() uint64 { @@ -921,7 +992,7 @@ type CreateEntryRequest struct { func (x *CreateEntryRequest) Reset() { *x = CreateEntryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[11] + mi := &file_filer_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -934,7 +1005,7 @@ func (x *CreateEntryRequest) String() string { func (*CreateEntryRequest) ProtoMessage() {} func (x *CreateEntryRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[11] + mi := &file_filer_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -947,7 +1018,7 @@ func (x *CreateEntryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateEntryRequest.ProtoReflect.Descriptor instead. func (*CreateEntryRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{11} + return file_filer_proto_rawDescGZIP(), []int{12} } func (x *CreateEntryRequest) GetDirectory() string { @@ -996,7 +1067,7 @@ type CreateEntryResponse struct { func (x *CreateEntryResponse) Reset() { *x = CreateEntryResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[12] + mi := &file_filer_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1009,7 +1080,7 @@ func (x *CreateEntryResponse) String() string { func (*CreateEntryResponse) ProtoMessage() {} func (x *CreateEntryResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[12] + mi := &file_filer_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1022,7 +1093,7 @@ func (x *CreateEntryResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateEntryResponse.ProtoReflect.Descriptor instead. func (*CreateEntryResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{12} + return file_filer_proto_rawDescGZIP(), []int{13} } func (x *CreateEntryResponse) GetError() string { @@ -1046,7 +1117,7 @@ type UpdateEntryRequest struct { func (x *UpdateEntryRequest) Reset() { *x = UpdateEntryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[13] + mi := &file_filer_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1059,7 +1130,7 @@ func (x *UpdateEntryRequest) String() string { func (*UpdateEntryRequest) ProtoMessage() {} func (x *UpdateEntryRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[13] + mi := &file_filer_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1072,7 +1143,7 @@ func (x *UpdateEntryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateEntryRequest.ProtoReflect.Descriptor instead. func (*UpdateEntryRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{13} + return file_filer_proto_rawDescGZIP(), []int{14} } func (x *UpdateEntryRequest) GetDirectory() string { @@ -1112,7 +1183,7 @@ type UpdateEntryResponse struct { func (x *UpdateEntryResponse) Reset() { *x = UpdateEntryResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[14] + mi := &file_filer_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1125,7 +1196,7 @@ func (x *UpdateEntryResponse) String() string { func (*UpdateEntryResponse) ProtoMessage() {} func (x *UpdateEntryResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[14] + mi := &file_filer_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1138,7 +1209,7 @@ func (x *UpdateEntryResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateEntryResponse.ProtoReflect.Descriptor instead. func (*UpdateEntryResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{14} + return file_filer_proto_rawDescGZIP(), []int{15} } type AppendToEntryRequest struct { @@ -1154,7 +1225,7 @@ type AppendToEntryRequest struct { func (x *AppendToEntryRequest) Reset() { *x = AppendToEntryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[15] + mi := &file_filer_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1167,7 +1238,7 @@ func (x *AppendToEntryRequest) String() string { func (*AppendToEntryRequest) ProtoMessage() {} func (x *AppendToEntryRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[15] + mi := &file_filer_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1180,7 +1251,7 @@ func (x *AppendToEntryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AppendToEntryRequest.ProtoReflect.Descriptor instead. func (*AppendToEntryRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{15} + return file_filer_proto_rawDescGZIP(), []int{16} } func (x *AppendToEntryRequest) GetDirectory() string { @@ -1213,7 +1284,7 @@ type AppendToEntryResponse struct { func (x *AppendToEntryResponse) Reset() { *x = AppendToEntryResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[16] + mi := &file_filer_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1226,7 +1297,7 @@ func (x *AppendToEntryResponse) String() string { func (*AppendToEntryResponse) ProtoMessage() {} func (x *AppendToEntryResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[16] + mi := &file_filer_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1239,7 +1310,7 @@ func (x *AppendToEntryResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AppendToEntryResponse.ProtoReflect.Descriptor instead. func (*AppendToEntryResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{16} + return file_filer_proto_rawDescGZIP(), []int{17} } type DeleteEntryRequest struct { @@ -1260,7 +1331,7 @@ type DeleteEntryRequest struct { func (x *DeleteEntryRequest) Reset() { *x = DeleteEntryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[17] + mi := &file_filer_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1273,7 +1344,7 @@ func (x *DeleteEntryRequest) String() string { func (*DeleteEntryRequest) ProtoMessage() {} func (x *DeleteEntryRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[17] + mi := &file_filer_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1286,7 +1357,7 @@ func (x *DeleteEntryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteEntryRequest.ProtoReflect.Descriptor instead. func (*DeleteEntryRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{17} + return file_filer_proto_rawDescGZIP(), []int{18} } func (x *DeleteEntryRequest) GetDirectory() string { @@ -1349,7 +1420,7 @@ type DeleteEntryResponse struct { func (x *DeleteEntryResponse) Reset() { *x = DeleteEntryResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[18] + mi := &file_filer_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1362,7 +1433,7 @@ func (x *DeleteEntryResponse) String() string { func (*DeleteEntryResponse) ProtoMessage() {} func (x *DeleteEntryResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[18] + mi := &file_filer_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1375,7 +1446,7 @@ func (x *DeleteEntryResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteEntryResponse.ProtoReflect.Descriptor instead. func (*DeleteEntryResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{18} + return file_filer_proto_rawDescGZIP(), []int{19} } func (x *DeleteEntryResponse) GetError() string { @@ -1400,7 +1471,7 @@ type AtomicRenameEntryRequest struct { func (x *AtomicRenameEntryRequest) Reset() { *x = AtomicRenameEntryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[19] + mi := &file_filer_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1413,7 +1484,7 @@ func (x *AtomicRenameEntryRequest) String() string { func (*AtomicRenameEntryRequest) ProtoMessage() {} func (x *AtomicRenameEntryRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[19] + mi := &file_filer_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1426,7 +1497,7 @@ func (x *AtomicRenameEntryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AtomicRenameEntryRequest.ProtoReflect.Descriptor instead. func (*AtomicRenameEntryRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{19} + return file_filer_proto_rawDescGZIP(), []int{20} } func (x *AtomicRenameEntryRequest) GetOldDirectory() string { @@ -1473,7 +1544,7 @@ type AtomicRenameEntryResponse struct { func (x *AtomicRenameEntryResponse) Reset() { *x = AtomicRenameEntryResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[20] + mi := &file_filer_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1486,7 +1557,7 @@ func (x *AtomicRenameEntryResponse) String() string { func (*AtomicRenameEntryResponse) ProtoMessage() {} func (x *AtomicRenameEntryResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[20] + mi := &file_filer_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1499,7 +1570,7 @@ func (x *AtomicRenameEntryResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AtomicRenameEntryResponse.ProtoReflect.Descriptor instead. func (*AtomicRenameEntryResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{20} + return file_filer_proto_rawDescGZIP(), []int{21} } type AssignVolumeRequest struct { @@ -1520,7 +1591,7 @@ type AssignVolumeRequest struct { func (x *AssignVolumeRequest) Reset() { *x = AssignVolumeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[21] + mi := &file_filer_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1533,7 +1604,7 @@ func (x *AssignVolumeRequest) String() string { func (*AssignVolumeRequest) ProtoMessage() {} func (x *AssignVolumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[21] + mi := &file_filer_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1546,7 +1617,7 @@ func (x *AssignVolumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignVolumeRequest.ProtoReflect.Descriptor instead. func (*AssignVolumeRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{21} + return file_filer_proto_rawDescGZIP(), []int{22} } func (x *AssignVolumeRequest) GetCount() int32 { @@ -1623,7 +1694,7 @@ type AssignVolumeResponse struct { func (x *AssignVolumeResponse) Reset() { *x = AssignVolumeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[22] + mi := &file_filer_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1636,7 +1707,7 @@ func (x *AssignVolumeResponse) String() string { func (*AssignVolumeResponse) ProtoMessage() {} func (x *AssignVolumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[22] + mi := &file_filer_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1649,7 +1720,7 @@ func (x *AssignVolumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignVolumeResponse.ProtoReflect.Descriptor instead. func (*AssignVolumeResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{22} + return file_filer_proto_rawDescGZIP(), []int{23} } func (x *AssignVolumeResponse) GetFileId() string { @@ -1719,7 +1790,7 @@ type LookupVolumeRequest struct { func (x *LookupVolumeRequest) Reset() { *x = LookupVolumeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[23] + mi := &file_filer_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1732,7 +1803,7 @@ func (x *LookupVolumeRequest) String() string { func (*LookupVolumeRequest) ProtoMessage() {} func (x *LookupVolumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[23] + mi := &file_filer_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1745,7 +1816,7 @@ func (x *LookupVolumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVolumeRequest.ProtoReflect.Descriptor instead. func (*LookupVolumeRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{23} + return file_filer_proto_rawDescGZIP(), []int{24} } func (x *LookupVolumeRequest) GetVolumeIds() []string { @@ -1766,7 +1837,7 @@ type Locations struct { func (x *Locations) Reset() { *x = Locations{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[24] + mi := &file_filer_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1779,7 +1850,7 @@ func (x *Locations) String() string { func (*Locations) ProtoMessage() {} func (x *Locations) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[24] + mi := &file_filer_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1792,7 +1863,7 @@ func (x *Locations) ProtoReflect() protoreflect.Message { // Deprecated: Use Locations.ProtoReflect.Descriptor instead. func (*Locations) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{24} + return file_filer_proto_rawDescGZIP(), []int{25} } func (x *Locations) GetLocations() []*Location { @@ -1814,7 +1885,7 @@ type Location struct { func (x *Location) Reset() { *x = Location{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[25] + mi := &file_filer_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1827,7 +1898,7 @@ func (x *Location) String() string { func (*Location) ProtoMessage() {} func (x *Location) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[25] + mi := &file_filer_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1840,7 +1911,7 @@ func (x *Location) ProtoReflect() protoreflect.Message { // Deprecated: Use Location.ProtoReflect.Descriptor instead. func (*Location) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{25} + return file_filer_proto_rawDescGZIP(), []int{26} } func (x *Location) GetUrl() string { @@ -1868,7 +1939,7 @@ type LookupVolumeResponse struct { func (x *LookupVolumeResponse) Reset() { *x = LookupVolumeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[26] + mi := &file_filer_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1881,7 +1952,7 @@ func (x *LookupVolumeResponse) String() string { func (*LookupVolumeResponse) ProtoMessage() {} func (x *LookupVolumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[26] + mi := &file_filer_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1894,7 +1965,7 @@ func (x *LookupVolumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVolumeResponse.ProtoReflect.Descriptor instead. func (*LookupVolumeResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{26} + return file_filer_proto_rawDescGZIP(), []int{27} } func (x *LookupVolumeResponse) GetLocationsMap() map[string]*Locations { @@ -1915,7 +1986,7 @@ type Collection struct { func (x *Collection) Reset() { *x = Collection{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[27] + mi := &file_filer_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1928,7 +1999,7 @@ func (x *Collection) String() string { func (*Collection) ProtoMessage() {} func (x *Collection) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[27] + mi := &file_filer_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1941,7 +2012,7 @@ func (x *Collection) ProtoReflect() protoreflect.Message { // Deprecated: Use Collection.ProtoReflect.Descriptor instead. func (*Collection) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{27} + return file_filer_proto_rawDescGZIP(), []int{28} } func (x *Collection) GetName() string { @@ -1963,7 +2034,7 @@ type CollectionListRequest struct { func (x *CollectionListRequest) Reset() { *x = CollectionListRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[28] + mi := &file_filer_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1976,7 +2047,7 @@ func (x *CollectionListRequest) String() string { func (*CollectionListRequest) ProtoMessage() {} func (x *CollectionListRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[28] + mi := &file_filer_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1989,7 +2060,7 @@ func (x *CollectionListRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionListRequest.ProtoReflect.Descriptor instead. func (*CollectionListRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{28} + return file_filer_proto_rawDescGZIP(), []int{29} } func (x *CollectionListRequest) GetIncludeNormalVolumes() bool { @@ -2017,7 +2088,7 @@ type CollectionListResponse struct { func (x *CollectionListResponse) Reset() { *x = CollectionListResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[29] + mi := &file_filer_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2030,7 +2101,7 @@ func (x *CollectionListResponse) String() string { func (*CollectionListResponse) ProtoMessage() {} func (x *CollectionListResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[29] + mi := &file_filer_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2043,7 +2114,7 @@ func (x *CollectionListResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionListResponse.ProtoReflect.Descriptor instead. func (*CollectionListResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{29} + return file_filer_proto_rawDescGZIP(), []int{30} } func (x *CollectionListResponse) GetCollections() []*Collection { @@ -2064,7 +2135,7 @@ type DeleteCollectionRequest struct { func (x *DeleteCollectionRequest) Reset() { *x = DeleteCollectionRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[30] + mi := &file_filer_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2077,7 +2148,7 @@ func (x *DeleteCollectionRequest) String() string { func (*DeleteCollectionRequest) ProtoMessage() {} func (x *DeleteCollectionRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[30] + mi := &file_filer_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2090,7 +2161,7 @@ func (x *DeleteCollectionRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteCollectionRequest.ProtoReflect.Descriptor instead. func (*DeleteCollectionRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{30} + return file_filer_proto_rawDescGZIP(), []int{31} } func (x *DeleteCollectionRequest) GetCollection() string { @@ -2109,7 +2180,7 @@ type DeleteCollectionResponse struct { func (x *DeleteCollectionResponse) Reset() { *x = DeleteCollectionResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[31] + mi := &file_filer_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2122,7 +2193,7 @@ func (x *DeleteCollectionResponse) String() string { func (*DeleteCollectionResponse) ProtoMessage() {} func (x *DeleteCollectionResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[31] + mi := &file_filer_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2135,7 +2206,7 @@ func (x *DeleteCollectionResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteCollectionResponse.ProtoReflect.Descriptor instead. func (*DeleteCollectionResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{31} + return file_filer_proto_rawDescGZIP(), []int{32} } type StatisticsRequest struct { @@ -2152,7 +2223,7 @@ type StatisticsRequest struct { func (x *StatisticsRequest) Reset() { *x = StatisticsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[32] + mi := &file_filer_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2165,7 +2236,7 @@ func (x *StatisticsRequest) String() string { func (*StatisticsRequest) ProtoMessage() {} func (x *StatisticsRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[32] + mi := &file_filer_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2178,7 +2249,7 @@ func (x *StatisticsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use StatisticsRequest.ProtoReflect.Descriptor instead. func (*StatisticsRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{32} + return file_filer_proto_rawDescGZIP(), []int{33} } func (x *StatisticsRequest) GetReplication() string { @@ -2222,7 +2293,7 @@ type StatisticsResponse struct { func (x *StatisticsResponse) Reset() { *x = StatisticsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[33] + mi := &file_filer_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2235,7 +2306,7 @@ func (x *StatisticsResponse) String() string { func (*StatisticsResponse) ProtoMessage() {} func (x *StatisticsResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[33] + mi := &file_filer_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2248,7 +2319,7 @@ func (x *StatisticsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use StatisticsResponse.ProtoReflect.Descriptor instead. func (*StatisticsResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{33} + return file_filer_proto_rawDescGZIP(), []int{34} } func (x *StatisticsResponse) GetTotalSize() uint64 { @@ -2281,7 +2352,7 @@ type GetFilerConfigurationRequest struct { func (x *GetFilerConfigurationRequest) Reset() { *x = GetFilerConfigurationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[34] + mi := &file_filer_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2294,7 +2365,7 @@ func (x *GetFilerConfigurationRequest) String() string { func (*GetFilerConfigurationRequest) ProtoMessage() {} func (x *GetFilerConfigurationRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[34] + mi := &file_filer_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2307,7 +2378,7 @@ func (x *GetFilerConfigurationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetFilerConfigurationRequest.ProtoReflect.Descriptor instead. func (*GetFilerConfigurationRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{34} + return file_filer_proto_rawDescGZIP(), []int{35} } type GetFilerConfigurationResponse struct { @@ -2330,7 +2401,7 @@ type GetFilerConfigurationResponse struct { func (x *GetFilerConfigurationResponse) Reset() { *x = GetFilerConfigurationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[35] + mi := &file_filer_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2343,7 +2414,7 @@ func (x *GetFilerConfigurationResponse) String() string { func (*GetFilerConfigurationResponse) ProtoMessage() {} func (x *GetFilerConfigurationResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[35] + mi := &file_filer_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2356,7 +2427,7 @@ func (x *GetFilerConfigurationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetFilerConfigurationResponse.ProtoReflect.Descriptor instead. func (*GetFilerConfigurationResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{35} + return file_filer_proto_rawDescGZIP(), []int{36} } func (x *GetFilerConfigurationResponse) GetMasters() []string { @@ -2443,7 +2514,7 @@ type SubscribeMetadataRequest struct { func (x *SubscribeMetadataRequest) Reset() { *x = SubscribeMetadataRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[36] + mi := &file_filer_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2456,7 +2527,7 @@ func (x *SubscribeMetadataRequest) String() string { func (*SubscribeMetadataRequest) ProtoMessage() {} func (x *SubscribeMetadataRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[36] + mi := &file_filer_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2469,7 +2540,7 @@ func (x *SubscribeMetadataRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeMetadataRequest.ProtoReflect.Descriptor instead. func (*SubscribeMetadataRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{36} + return file_filer_proto_rawDescGZIP(), []int{37} } func (x *SubscribeMetadataRequest) GetClientName() string { @@ -2513,7 +2584,7 @@ type SubscribeMetadataResponse struct { func (x *SubscribeMetadataResponse) Reset() { *x = SubscribeMetadataResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[37] + mi := &file_filer_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2526,7 +2597,7 @@ func (x *SubscribeMetadataResponse) String() string { func (*SubscribeMetadataResponse) ProtoMessage() {} func (x *SubscribeMetadataResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[37] + mi := &file_filer_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2539,7 +2610,7 @@ func (x *SubscribeMetadataResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeMetadataResponse.ProtoReflect.Descriptor instead. func (*SubscribeMetadataResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{37} + return file_filer_proto_rawDescGZIP(), []int{38} } func (x *SubscribeMetadataResponse) GetDirectory() string { @@ -2576,7 +2647,7 @@ type LogEntry struct { func (x *LogEntry) Reset() { *x = LogEntry{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[38] + mi := &file_filer_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2589,7 +2660,7 @@ func (x *LogEntry) String() string { func (*LogEntry) ProtoMessage() {} func (x *LogEntry) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[38] + mi := &file_filer_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2602,7 +2673,7 @@ func (x *LogEntry) ProtoReflect() protoreflect.Message { // Deprecated: Use LogEntry.ProtoReflect.Descriptor instead. func (*LogEntry) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{38} + return file_filer_proto_rawDescGZIP(), []int{39} } func (x *LogEntry) GetTsNs() int64 { @@ -2639,7 +2710,7 @@ type KeepConnectedRequest struct { func (x *KeepConnectedRequest) Reset() { *x = KeepConnectedRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[39] + mi := &file_filer_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2652,7 +2723,7 @@ func (x *KeepConnectedRequest) String() string { func (*KeepConnectedRequest) ProtoMessage() {} func (x *KeepConnectedRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[39] + mi := &file_filer_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2665,7 +2736,7 @@ func (x *KeepConnectedRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use KeepConnectedRequest.ProtoReflect.Descriptor instead. func (*KeepConnectedRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{39} + return file_filer_proto_rawDescGZIP(), []int{40} } func (x *KeepConnectedRequest) GetName() string { @@ -2698,7 +2769,7 @@ type KeepConnectedResponse struct { func (x *KeepConnectedResponse) Reset() { *x = KeepConnectedResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[40] + mi := &file_filer_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2711,7 +2782,7 @@ func (x *KeepConnectedResponse) String() string { func (*KeepConnectedResponse) ProtoMessage() {} func (x *KeepConnectedResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[40] + mi := &file_filer_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2724,7 +2795,7 @@ func (x *KeepConnectedResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use KeepConnectedResponse.ProtoReflect.Descriptor instead. func (*KeepConnectedResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{40} + return file_filer_proto_rawDescGZIP(), []int{41} } type LocateBrokerRequest struct { @@ -2738,7 +2809,7 @@ type LocateBrokerRequest struct { func (x *LocateBrokerRequest) Reset() { *x = LocateBrokerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[41] + mi := &file_filer_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2751,7 +2822,7 @@ func (x *LocateBrokerRequest) String() string { func (*LocateBrokerRequest) ProtoMessage() {} func (x *LocateBrokerRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[41] + mi := &file_filer_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2764,7 +2835,7 @@ func (x *LocateBrokerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LocateBrokerRequest.ProtoReflect.Descriptor instead. func (*LocateBrokerRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{41} + return file_filer_proto_rawDescGZIP(), []int{42} } func (x *LocateBrokerRequest) GetResource() string { @@ -2786,7 +2857,7 @@ type LocateBrokerResponse struct { func (x *LocateBrokerResponse) Reset() { *x = LocateBrokerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[42] + mi := &file_filer_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2799,7 +2870,7 @@ func (x *LocateBrokerResponse) String() string { func (*LocateBrokerResponse) ProtoMessage() {} func (x *LocateBrokerResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[42] + mi := &file_filer_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2812,7 +2883,7 @@ func (x *LocateBrokerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LocateBrokerResponse.ProtoReflect.Descriptor instead. func (*LocateBrokerResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{42} + return file_filer_proto_rawDescGZIP(), []int{43} } func (x *LocateBrokerResponse) GetFound() bool { @@ -2829,7 +2900,9 @@ func (x *LocateBrokerResponse) GetResources() []*LocateBrokerResponse_Resource { return nil } +///////////////////////// // Key-Value operations +///////////////////////// type KvGetRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2841,7 +2914,7 @@ type KvGetRequest struct { func (x *KvGetRequest) Reset() { *x = KvGetRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[43] + mi := &file_filer_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2854,7 +2927,7 @@ func (x *KvGetRequest) String() string { func (*KvGetRequest) ProtoMessage() {} func (x *KvGetRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[43] + mi := &file_filer_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2867,7 +2940,7 @@ func (x *KvGetRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use KvGetRequest.ProtoReflect.Descriptor instead. func (*KvGetRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{43} + return file_filer_proto_rawDescGZIP(), []int{44} } func (x *KvGetRequest) GetKey() []byte { @@ -2889,7 +2962,7 @@ type KvGetResponse struct { func (x *KvGetResponse) Reset() { *x = KvGetResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[44] + mi := &file_filer_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2902,7 +2975,7 @@ func (x *KvGetResponse) String() string { func (*KvGetResponse) ProtoMessage() {} func (x *KvGetResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[44] + mi := &file_filer_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2915,7 +2988,7 @@ func (x *KvGetResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use KvGetResponse.ProtoReflect.Descriptor instead. func (*KvGetResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{44} + return file_filer_proto_rawDescGZIP(), []int{45} } func (x *KvGetResponse) GetValue() []byte { @@ -2944,7 +3017,7 @@ type KvPutRequest struct { func (x *KvPutRequest) Reset() { *x = KvPutRequest{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[45] + mi := &file_filer_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2957,7 +3030,7 @@ func (x *KvPutRequest) String() string { func (*KvPutRequest) ProtoMessage() {} func (x *KvPutRequest) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[45] + mi := &file_filer_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2970,7 +3043,7 @@ func (x *KvPutRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use KvPutRequest.ProtoReflect.Descriptor instead. func (*KvPutRequest) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{45} + return file_filer_proto_rawDescGZIP(), []int{46} } func (x *KvPutRequest) GetKey() []byte { @@ -2998,7 +3071,7 @@ type KvPutResponse struct { func (x *KvPutResponse) Reset() { *x = KvPutResponse{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[46] + mi := &file_filer_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3011,7 +3084,7 @@ func (x *KvPutResponse) String() string { func (*KvPutResponse) ProtoMessage() {} func (x *KvPutResponse) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[46] + mi := &file_filer_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3024,7 +3097,7 @@ func (x *KvPutResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use KvPutResponse.ProtoReflect.Descriptor instead. func (*KvPutResponse) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{46} + return file_filer_proto_rawDescGZIP(), []int{47} } func (x *KvPutResponse) GetError() string { @@ -3034,7 +3107,9 @@ func (x *KvPutResponse) GetError() string { return "" } +///////////////////////// // path-based configurations +///////////////////////// type FilerConf struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3047,7 +3122,7 @@ type FilerConf struct { func (x *FilerConf) Reset() { *x = FilerConf{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[47] + mi := &file_filer_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3060,7 +3135,7 @@ func (x *FilerConf) String() string { func (*FilerConf) ProtoMessage() {} func (x *FilerConf) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[47] + mi := &file_filer_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3073,7 +3148,7 @@ func (x *FilerConf) ProtoReflect() protoreflect.Message { // Deprecated: Use FilerConf.ProtoReflect.Descriptor instead. func (*FilerConf) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{47} + return file_filer_proto_rawDescGZIP(), []int{48} } func (x *FilerConf) GetVersion() int32 { @@ -3090,6 +3165,9 @@ func (x *FilerConf) GetLocations() []*FilerConf_PathConf { return nil } +///////////////////////// +// Remote Storage related +///////////////////////// type RemoteConf struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3106,7 +3184,7 @@ type RemoteConf struct { func (x *RemoteConf) Reset() { *x = RemoteConf{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[48] + mi := &file_filer_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3119,7 +3197,7 @@ func (x *RemoteConf) String() string { func (*RemoteConf) ProtoMessage() {} func (x *RemoteConf) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[48] + mi := &file_filer_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3132,7 +3210,7 @@ func (x *RemoteConf) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoteConf.ProtoReflect.Descriptor instead. func (*RemoteConf) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{48} + return file_filer_proto_rawDescGZIP(), []int{49} } func (x *RemoteConf) GetType() string { @@ -3177,18 +3255,16 @@ func (x *RemoteConf) GetS3Endpoint() string { return "" } -type Entry_Remote struct { +type RemoteStorageMapping struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - LastModifiedAt int64 `protobuf:"varint,1,opt,name=last_modified_at,json=lastModifiedAt,proto3" json:"last_modified_at,omitempty"` - Size int64 `protobuf:"varint,2,opt,name=size,proto3" json:"size,omitempty"` - ETag string `protobuf:"bytes,3,opt,name=e_tag,json=eTag,proto3" json:"e_tag,omitempty"` + Mappings map[string]*RemoteStorageLocation `protobuf:"bytes,1,rep,name=mappings,proto3" json:"mappings,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (x *Entry_Remote) Reset() { - *x = Entry_Remote{} +func (x *RemoteStorageMapping) Reset() { + *x = RemoteStorageMapping{} if protoimpl.UnsafeEnabled { mi := &file_filer_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -3196,13 +3272,13 @@ func (x *Entry_Remote) Reset() { } } -func (x *Entry_Remote) String() string { +func (x *RemoteStorageMapping) String() string { return protoimpl.X.MessageStringOf(x) } -func (*Entry_Remote) ProtoMessage() {} +func (*RemoteStorageMapping) ProtoMessage() {} -func (x *Entry_Remote) ProtoReflect() protoreflect.Message { +func (x *RemoteStorageMapping) ProtoReflect() protoreflect.Message { mi := &file_filer_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -3214,28 +3290,77 @@ func (x *Entry_Remote) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Entry_Remote.ProtoReflect.Descriptor instead. -func (*Entry_Remote) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{4, 1} +// Deprecated: Use RemoteStorageMapping.ProtoReflect.Descriptor instead. +func (*RemoteStorageMapping) Descriptor() ([]byte, []int) { + return file_filer_proto_rawDescGZIP(), []int{50} } -func (x *Entry_Remote) GetLastModifiedAt() int64 { +func (x *RemoteStorageMapping) GetMappings() map[string]*RemoteStorageLocation { if x != nil { - return x.LastModifiedAt + return x.Mappings } - return 0 + return nil } -func (x *Entry_Remote) GetSize() int64 { - if x != nil { - return x.Size +type RemoteStorageLocation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Bucket string `protobuf:"bytes,2,opt,name=bucket,proto3" json:"bucket,omitempty"` + Path string `protobuf:"bytes,3,opt,name=path,proto3" json:"path,omitempty"` +} + +func (x *RemoteStorageLocation) Reset() { + *x = RemoteStorageLocation{} + if protoimpl.UnsafeEnabled { + mi := &file_filer_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return 0 } -func (x *Entry_Remote) GetETag() string { +func (x *RemoteStorageLocation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoteStorageLocation) ProtoMessage() {} + +func (x *RemoteStorageLocation) ProtoReflect() protoreflect.Message { + mi := &file_filer_proto_msgTypes[51] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RemoteStorageLocation.ProtoReflect.Descriptor instead. +func (*RemoteStorageLocation) Descriptor() ([]byte, []int) { + return file_filer_proto_rawDescGZIP(), []int{51} +} + +func (x *RemoteStorageLocation) GetName() string { if x != nil { - return x.ETag + return x.Name + } + return "" +} + +func (x *RemoteStorageLocation) GetBucket() string { + if x != nil { + return x.Bucket + } + return "" +} + +func (x *RemoteStorageLocation) GetPath() string { + if x != nil { + return x.Path } return "" } @@ -3254,7 +3379,7 @@ type LocateBrokerResponse_Resource struct { func (x *LocateBrokerResponse_Resource) Reset() { *x = LocateBrokerResponse_Resource{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[52] + mi := &file_filer_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3267,7 +3392,7 @@ func (x *LocateBrokerResponse_Resource) String() string { func (*LocateBrokerResponse_Resource) ProtoMessage() {} func (x *LocateBrokerResponse_Resource) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[52] + mi := &file_filer_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3280,7 +3405,7 @@ func (x *LocateBrokerResponse_Resource) ProtoReflect() protoreflect.Message { // Deprecated: Use LocateBrokerResponse_Resource.ProtoReflect.Descriptor instead. func (*LocateBrokerResponse_Resource) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{42, 0} + return file_filer_proto_rawDescGZIP(), []int{43, 0} } func (x *LocateBrokerResponse_Resource) GetGrpcAddresses() string { @@ -3315,7 +3440,7 @@ type FilerConf_PathConf struct { func (x *FilerConf_PathConf) Reset() { *x = FilerConf_PathConf{} if protoimpl.UnsafeEnabled { - mi := &file_filer_proto_msgTypes[53] + mi := &file_filer_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3328,7 +3453,7 @@ func (x *FilerConf_PathConf) String() string { func (*FilerConf_PathConf) ProtoMessage() {} func (x *FilerConf_PathConf) ProtoReflect() protoreflect.Message { - mi := &file_filer_proto_msgTypes[53] + mi := &file_filer_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3341,7 +3466,7 @@ func (x *FilerConf_PathConf) ProtoReflect() protoreflect.Message { // Deprecated: Use FilerConf_PathConf.ProtoReflect.Descriptor instead. func (*FilerConf_PathConf) Descriptor() ([]byte, []int) { - return file_filer_proto_rawDescGZIP(), []int{47, 0} + return file_filer_proto_rawDescGZIP(), []int{48, 0} } func (x *FilerConf_PathConf) GetLocationPrefix() string { @@ -3429,508 +3554,529 @@ var file_filer_proto_rawDesc = []byte{ 0x22, 0x3c, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, - 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x22, 0x92, - 0x04, 0x0a, 0x05, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x21, 0x0a, 0x0c, - 0x69, 0x73, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, - 0x2b, 0x0a, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x13, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x43, - 0x68, 0x75, 0x6e, 0x6b, 0x52, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x12, 0x38, 0x0a, 0x0a, - 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x75, 0x73, 0x65, - 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x0a, 0x61, 0x74, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x39, 0x0a, 0x08, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, - 0x65, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, - 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, - 0x65, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, - 0x64, 0x12, 0x20, 0x0a, 0x0c, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x69, - 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x69, 0x6e, - 0x6b, 0x49, 0x64, 0x12, 0x2a, 0x0a, 0x11, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, - 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, - 0x68, 0x61, 0x72, 0x64, 0x4c, 0x69, 0x6e, 0x6b, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, - 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, - 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x12, 0x2e, 0x0a, 0x06, 0x72, 0x65, 0x6d, - 0x6f, 0x74, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, - 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, - 0x65, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x1a, 0x3b, 0x0a, 0x0d, 0x45, 0x78, 0x74, + 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x22, 0x83, + 0x01, 0x0a, 0x0b, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x28, + 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, + 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, + 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x41, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x13, 0x0a, 0x05, + 0x65, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x65, 0x54, 0x61, + 0x67, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbf, 0x03, 0x0a, 0x05, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x12, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x44, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x2b, 0x0a, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x52, 0x06, 0x63, 0x68, 0x75, 0x6e, + 0x6b, 0x73, 0x12, 0x38, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, + 0x62, 0x2e, 0x46, 0x75, 0x73, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x52, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x39, 0x0a, 0x08, + 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x2e, + 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x65, + 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x12, 0x20, 0x0a, 0x0c, 0x68, 0x61, 0x72, 0x64, 0x5f, + 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x68, + 0x61, 0x72, 0x64, 0x4c, 0x69, 0x6e, 0x6b, 0x49, 0x64, 0x12, 0x2a, 0x0a, 0x11, 0x68, 0x61, 0x72, + 0x64, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x69, 0x6e, 0x6b, 0x43, 0x6f, + 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x12, + 0x38, 0x0a, 0x0c, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x72, 0x65, + 0x6d, 0x6f, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x1a, 0x3b, 0x0a, 0x0d, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x5b, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, - 0x12, 0x28, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, - 0x64, 0x5f, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, - 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x41, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, - 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x13, - 0x0a, 0x05, 0x65, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x65, - 0x54, 0x61, 0x67, 0x22, 0x44, 0x0a, 0x09, 0x46, 0x75, 0x6c, 0x6c, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x64, - 0x69, 0x72, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x22, 0x8f, 0x02, 0x0a, 0x11, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x2c, 0x0a, 0x09, 0x6f, 0x6c, 0x64, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x08, 0x6f, 0x6c, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x2c, 0x0a, - 0x09, 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x08, 0x6e, 0x65, 0x77, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x64, - 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0c, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x73, - 0x12, 0x26, 0x0a, 0x0f, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x70, - 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6e, 0x65, 0x77, 0x50, 0x61, - 0x72, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12, 0x31, 0x0a, 0x15, 0x69, 0x73, 0x5f, 0x66, - 0x72, 0x6f, 0x6d, 0x5f, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, - 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x69, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x4f, - 0x74, 0x68, 0x65, 0x72, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x73, - 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x05, 0x52, - 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x22, 0xe6, 0x02, 0x0a, 0x09, - 0x46, 0x69, 0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x69, 0x6c, - 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x65, - 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, - 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x14, - 0x0a, 0x05, 0x6d, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6d, - 0x74, 0x69, 0x6d, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x65, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x65, 0x54, 0x61, 0x67, 0x12, 0x24, 0x0a, 0x0e, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x12, - 0x22, 0x0a, 0x03, 0x66, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x66, - 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x52, 0x03, - 0x66, 0x69, 0x64, 0x12, 0x2f, 0x0a, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x66, 0x69, - 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, - 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x52, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x46, 0x69, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x69, 0x70, 0x68, 0x65, 0x72, 0x5f, 0x6b, - 0x65, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x63, 0x69, 0x70, 0x68, 0x65, 0x72, - 0x4b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, - 0x73, 0x73, 0x65, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x43, 0x6f, - 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x73, 0x5f, 0x63, - 0x68, 0x75, 0x6e, 0x6b, 0x5f, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x0b, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x4d, 0x61, 0x6e, 0x69, - 0x66, 0x65, 0x73, 0x74, 0x22, 0x40, 0x0a, 0x11, 0x46, 0x69, 0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, - 0x6b, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x06, 0x63, 0x68, 0x75, - 0x6e, 0x6b, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x69, 0x6c, 0x65, - 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x52, 0x06, - 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x22, 0x58, 0x0a, 0x06, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x64, - 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, 0x12, 0x19, 0x0a, - 0x08, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x07, 0x66, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6f, 0x6b, - 0x69, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x07, 0x52, 0x06, 0x63, 0x6f, 0x6f, 0x6b, 0x69, 0x65, - 0x22, 0x9d, 0x03, 0x0a, 0x0e, 0x46, 0x75, 0x73, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, - 0x74, 0x65, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x6d, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x05, 0x6d, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6d, - 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4d, - 0x6f, 0x64, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, - 0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x03, 0x67, 0x69, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x72, 0x74, 0x69, 0x6d, - 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x63, 0x72, 0x74, 0x69, 0x6d, 0x65, 0x12, - 0x12, 0x0a, 0x04, 0x6d, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6d, - 0x69, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x74, 0x6c, 0x5f, 0x73, 0x65, 0x63, - 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x74, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x12, 0x1b, - 0x0a, 0x09, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x67, - 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x79, - 0x6d, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x0d, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0d, 0x73, 0x79, 0x6d, 0x6c, 0x69, 0x6e, 0x6b, 0x54, 0x61, 0x72, 0x67, 0x65, - 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x64, 0x35, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, - 0x6d, 0x64, 0x35, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, - 0x22, 0xc3, 0x01, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, - 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x44, 0x0a, 0x09, 0x46, 0x75, 0x6c, 0x6c, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x64, 0x69, 0x72, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x15, 0x0a, 0x06, - 0x6f, 0x5f, 0x65, 0x78, 0x63, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x6f, 0x45, - 0x78, 0x63, 0x6c, 0x12, 0x31, 0x0a, 0x15, 0x69, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x6f, - 0x74, 0x68, 0x65, 0x72, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x12, 0x69, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x4f, 0x74, 0x68, 0x65, 0x72, 0x43, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, - 0x75, 0x72, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, - 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x22, 0x2b, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x22, 0xac, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x22, 0x8f, 0x02, 0x0a, + 0x11, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x2c, 0x0a, 0x09, 0x6f, 0x6c, 0x64, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6f, 0x6c, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x2c, 0x0a, 0x09, 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6e, 0x65, 0x77, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x23, + 0x0a, 0x0d, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x68, 0x75, + 0x6e, 0x6b, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x61, 0x72, 0x65, 0x6e, + 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6e, 0x65, + 0x77, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12, 0x31, 0x0a, 0x15, 0x69, + 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x5f, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x69, 0x73, 0x46, 0x72, + 0x6f, 0x6d, 0x4f, 0x74, 0x68, 0x65, 0x72, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1e, + 0x0a, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, + 0x28, 0x05, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x22, 0xe6, + 0x02, 0x0a, 0x09, 0x46, 0x69, 0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x17, 0x0a, 0x07, + 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, + 0x69, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x12, 0x0a, + 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x69, 0x7a, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6d, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x6d, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x65, 0x5f, 0x74, 0x61, 0x67, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x65, 0x54, 0x61, 0x67, 0x12, 0x24, 0x0a, 0x0e, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x65, + 0x49, 0x64, 0x12, 0x22, 0x0a, 0x03, 0x66, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x49, + 0x64, 0x52, 0x03, 0x66, 0x69, 0x64, 0x12, 0x2f, 0x0a, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x66, 0x69, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x66, 0x69, 0x6c, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x52, 0x09, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x46, 0x69, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x69, 0x70, 0x68, 0x65, + 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x63, 0x69, 0x70, + 0x68, 0x65, 0x72, 0x4b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x6d, + 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, + 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x2a, 0x0a, 0x11, 0x69, + 0x73, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x5f, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x4d, + 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x22, 0x40, 0x0a, 0x11, 0x46, 0x69, 0x6c, 0x65, 0x43, + 0x68, 0x75, 0x6e, 0x6b, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x06, + 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, + 0x6b, 0x52, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x22, 0x58, 0x0a, 0x06, 0x46, 0x69, 0x6c, + 0x65, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, + 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x63, + 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x07, 0x52, 0x06, 0x63, 0x6f, 0x6f, + 0x6b, 0x69, 0x65, 0x22, 0x9d, 0x03, 0x0a, 0x0e, 0x46, 0x75, 0x73, 0x65, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, + 0x69, 0x7a, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6d, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x05, 0x6d, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, + 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x66, 0x69, + 0x6c, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x69, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x67, 0x69, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x72, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x63, 0x72, 0x74, 0x69, + 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6d, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x6d, 0x69, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x74, 0x6c, 0x5f, + 0x73, 0x65, 0x63, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x74, 0x74, 0x6c, 0x53, 0x65, + 0x63, 0x12, 0x1b, 0x0a, 0x09, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0b, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, + 0x0a, 0x0a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0c, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, + 0x0e, 0x73, 0x79, 0x6d, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, + 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x79, 0x6d, 0x6c, 0x69, 0x6e, 0x6b, 0x54, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x64, 0x35, 0x18, 0x0e, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x03, 0x6d, 0x64, 0x35, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, + 0x79, 0x70, 0x65, 0x22, 0xc3, 0x01, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x31, 0x0a, 0x15, 0x69, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x6f, 0x74, 0x68, 0x65, 0x72, - 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, - 0x69, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x4f, 0x74, 0x68, 0x65, 0x72, 0x43, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, - 0x65, 0x73, 0x22, 0x15, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x80, 0x01, 0x0a, 0x14, 0x41, 0x70, - 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, - 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x2b, 0x0a, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x13, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x43, - 0x68, 0x75, 0x6e, 0x6b, 0x52, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x22, 0x17, 0x0a, 0x15, - 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x98, 0x02, 0x0a, 0x12, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, - 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x24, - 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x64, 0x61, 0x74, 0x61, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x63, 0x75, 0x72, - 0x73, 0x69, 0x76, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x52, 0x65, - 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x69, 0x67, 0x6e, 0x6f, 0x72, - 0x65, 0x5f, 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x5f, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x52, - 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x31, 0x0a, - 0x15, 0x69, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x5f, 0x63, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x69, 0x73, - 0x46, 0x72, 0x6f, 0x6d, 0x4f, 0x74, 0x68, 0x65, 0x72, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, - 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, 0x08, - 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, - 0x22, 0x2b, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xba, 0x01, - 0x0a, 0x18, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x6c, - 0x64, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x6f, 0x6c, 0x64, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, - 0x19, 0x0a, 0x08, 0x6f, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6f, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x65, - 0x77, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x6e, 0x65, 0x77, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, - 0x19, 0x0a, 0x08, 0x6e, 0x65, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6e, 0x65, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x69, - 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, - 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x22, 0x1b, 0x0a, 0x19, 0x41, 0x74, - 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xec, 0x01, 0x0a, 0x13, 0x41, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x74, 0x6c, 0x5f, 0x73, - 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x74, 0x74, 0x6c, 0x53, 0x65, 0x63, - 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x43, 0x65, 0x6e, 0x74, 0x65, - 0x72, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x72, 0x61, 0x63, 0x6b, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x61, 0x63, 0x6b, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, - 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, - 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x22, 0xe2, 0x01, 0x0a, 0x14, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x17, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x06, 0x66, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x75, - 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x55, 0x72, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, - 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, - 0x12, 0x0a, 0x04, 0x61, 0x75, 0x74, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x61, - 0x75, 0x74, 0x68, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x08, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x34, 0x0a, 0x13, 0x4c, - 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, - 0x73, 0x22, 0x3d, 0x0a, 0x09, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, - 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x3b, 0x0a, 0x08, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, - 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x1d, - 0x0a, 0x0a, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x55, 0x72, 0x6c, 0x22, 0xc3, 0x01, - 0x0a, 0x14, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x55, 0x0a, 0x0d, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, - 0x66, 0x69, 0x6c, 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, 0x2e, 0x4c, 0x6f, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x0c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4d, 0x61, 0x70, 0x1a, 0x54, 0x0a, - 0x11, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x29, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x22, 0x20, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x7b, 0x0a, 0x15, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x34, - 0x0a, 0x16, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x6e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, - 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x56, 0x6f, 0x6c, - 0x75, 0x6d, 0x65, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, - 0x65, 0x63, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x45, 0x63, 0x56, 0x6f, 0x6c, 0x75, 0x6d, - 0x65, 0x73, 0x22, 0x50, 0x0a, 0x16, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x36, 0x0a, 0x0b, - 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x39, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, - 0x1a, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x84, 0x01, 0x0a, 0x11, - 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x74, 0x74, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, - 0x70, 0x65, 0x22, 0x6f, 0x0a, 0x12, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, - 0x6c, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x6f, - 0x74, 0x61, 0x6c, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x75, 0x73, 0x65, 0x64, 0x5f, - 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x75, 0x73, 0x65, 0x64, - 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x63, 0x6f, 0x75, - 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x43, 0x6f, - 0x75, 0x6e, 0x74, 0x22, 0x1e, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0xde, 0x02, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, - 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x15, 0x0a, 0x06, 0x6d, 0x61, 0x78, 0x5f, 0x6d, 0x62, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x05, 0x6d, 0x61, 0x78, 0x4d, 0x62, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x69, 0x72, 0x5f, - 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x64, - 0x69, 0x72, 0x42, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x69, 0x70, - 0x68, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x63, 0x69, 0x70, 0x68, 0x65, - 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x08, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, - 0x27, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, - 0x73, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, - 0x73, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x65, 0x74, 0x72, - 0x69, 0x63, 0x73, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x5f, 0x73, 0x65, 0x63, - 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x12, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x49, - 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x53, 0x65, 0x63, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x95, 0x01, 0x0a, 0x18, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, - 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x74, 0x68, 0x50, 0x72, 0x65, - 0x66, 0x69, 0x78, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x73, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x4e, 0x73, 0x12, 0x1c, - 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x9a, 0x01, 0x0a, - 0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69, - 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, - 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x4a, 0x0a, 0x12, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x5f, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x11, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x22, 0x61, 0x0a, 0x08, 0x4c, 0x6f, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x68, 0x61, 0x73, 0x68, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x4b, 0x65, 0x79, 0x48, 0x61, 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x65, 0x0a, 0x14, - 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x72, 0x70, 0x63, - 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x67, 0x72, 0x70, - 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x0a, 0x13, - 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, - 0xcd, 0x01, 0x0a, 0x14, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x75, 0x6e, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x12, 0x45, - 0x0a, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x27, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, - 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x73, 0x1a, 0x58, 0x0a, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, - 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x67, 0x72, 0x70, 0x63, 0x41, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, - 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, - 0x20, 0x0a, 0x0c, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x22, 0x3b, 0x0a, 0x0d, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x36, - 0x0a, 0x0c, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, - 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, - 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x25, 0x0a, 0x0d, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xeb, 0x02, - 0x0a, 0x09, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x12, 0x18, 0x0a, 0x07, 0x76, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x76, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, - 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x2e, 0x50, 0x61, - 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x1a, 0x87, 0x02, 0x0a, 0x08, 0x50, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x12, 0x27, - 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, - 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x74, 0x6c, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x64, - 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x73, 0x79, 0x6e, - 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x2e, - 0x0a, 0x13, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x77, 0x74, 0x68, 0x5f, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, 0x76, 0x6f, 0x6c, - 0x75, 0x6d, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x74, 0x68, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1b, - 0x0a, 0x09, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x08, 0x72, 0x65, 0x61, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x22, 0xba, 0x01, 0x0a, 0x0a, - 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x12, - 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x33, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, - 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x33, 0x41, 0x63, 0x63, - 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x33, 0x5f, 0x73, 0x65, 0x63, - 0x72, 0x65, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, - 0x33, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x33, - 0x5f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, - 0x33, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x33, 0x5f, 0x65, 0x6e, - 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x33, - 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x32, 0xdc, 0x0c, 0x0a, 0x0c, 0x53, 0x65, 0x61, - 0x77, 0x65, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x12, 0x67, 0x0a, 0x14, 0x4c, 0x6f, 0x6f, - 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x25, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, - 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, - 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, - 0x73, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, - 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45, - 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x4c, 0x0a, 0x0b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, - 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, - 0x0a, 0x0d, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, - 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, - 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x5e, 0x0a, 0x11, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, - 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, - 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, - 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x4f, 0x0a, 0x0c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, - 0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, - 0x65, 0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 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, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, + 0x15, 0x0a, 0x06, 0x6f, 0x5f, 0x65, 0x78, 0x63, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x05, 0x6f, 0x45, 0x78, 0x63, 0x6c, 0x12, 0x31, 0x0a, 0x15, 0x69, 0x73, 0x5f, 0x66, 0x72, 0x6f, + 0x6d, 0x5f, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x69, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x4f, 0x74, 0x68, + 0x65, 0x72, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x69, 0x67, + 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x73, + 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x22, 0x2b, 0x0a, 0x13, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xac, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, + 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x25, 0x0a, 0x05, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x31, 0x0a, 0x15, 0x69, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x6f, 0x74, + 0x68, 0x65, 0x72, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x12, 0x69, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x4f, 0x74, 0x68, 0x65, 0x72, 0x43, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, + 0x72, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, + 0x74, 0x75, 0x72, 0x65, 0x73, 0x22, 0x15, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x80, 0x01, 0x0a, + 0x14, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x79, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, + 0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x52, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x22, + 0x17, 0x0a, 0x15, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x98, 0x02, 0x0a, 0x12, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1c, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x72, 0x65, + 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, + 0x73, 0x52, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x69, 0x67, + 0x6e, 0x6f, 0x72, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x5f, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x69, 0x67, 0x6e, 0x6f, + 0x72, 0x65, 0x52, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x12, 0x31, 0x0a, 0x15, 0x69, 0x73, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x6f, 0x74, 0x68, 0x65, + 0x72, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x12, 0x69, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x4f, 0x74, 0x68, 0x65, 0x72, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, + 0x72, 0x65, 0x73, 0x22, 0x2b, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x22, 0xba, 0x01, 0x0a, 0x18, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, + 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, + 0x0d, 0x6f, 0x6c, 0x64, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6f, 0x6c, 0x64, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x79, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, + 0x0d, 0x6e, 0x65, 0x77, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6e, 0x65, 0x77, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x79, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x65, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6e, 0x65, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, + 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, + 0x05, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x22, 0x1b, 0x0a, + 0x19, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xec, 0x01, 0x0a, 0x13, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x74, + 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x74, 0x74, 0x6c, + 0x53, 0x65, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x65, 0x6e, 0x74, + 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x43, 0x65, + 0x6e, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x72, 0x61, 0x63, 0x6b, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x61, 0x63, 0x6b, 0x12, 0x1b, 0x0a, 0x09, + 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x22, 0xe2, 0x01, 0x0a, 0x14, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, + 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x1d, 0x0a, + 0x0a, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x55, 0x72, 0x6c, 0x12, 0x14, 0x0a, 0x05, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x75, 0x74, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x61, 0x75, 0x74, 0x68, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x34, + 0x0a, 0x13, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, + 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x49, 0x64, 0x73, 0x22, 0x3d, 0x0a, 0x09, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x30, 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, + 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x22, 0x3b, 0x0a, 0x08, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, + 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x75, 0x72, 0x6c, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x55, 0x72, 0x6c, + 0x22, 0xc3, 0x01, 0x0a, 0x14, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x55, 0x0a, 0x0d, 0x6c, 0x6f, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x30, 0x2e, 0x66, 0x69, 0x6c, 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, 0x55, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 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, 0x20, 0x2e, 0x66, 0x69, 0x6c, 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, 0x5b, 0x0a, 0x10, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, - 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x49, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, - 0x74, 0x69, 0x63, 0x73, 0x12, 0x1b, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 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, 0x6a, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 0x2e, 0x66, 0x69, 0x6c, - 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, - 0x74, 0x46, 0x69, 0x6c, 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, 0x60, 0x0a, - 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, + 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4d, 0x61, 0x70, + 0x1a, 0x54, 0x0a, 0x11, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4d, 0x61, 0x70, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x29, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, + 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x20, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x7b, 0x0a, 0x15, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x34, 0x0a, 0x16, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x6e, 0x6f, 0x72, + 0x6d, 0x61, 0x6c, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x14, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, + 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x6e, 0x63, 0x6c, 0x75, + 0x64, 0x65, 0x5f, 0x65, 0x63, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x45, 0x63, 0x56, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x22, 0x50, 0x0a, 0x16, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x36, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x39, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x22, 0x1a, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x84, + 0x01, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, + 0x6b, 0x54, 0x79, 0x70, 0x65, 0x22, 0x6f, 0x0a, 0x12, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, + 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x75, 0x73, + 0x65, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x75, + 0x73, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x66, 0x69, 0x6c, + 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x1e, 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, + 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xde, 0x02, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x46, 0x69, + 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x61, 0x73, 0x74, + 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x61, 0x73, 0x74, 0x65, + 0x72, 0x73, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x15, 0x0a, 0x06, 0x6d, 0x61, 0x78, 0x5f, 0x6d, 0x62, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6d, 0x61, 0x78, 0x4d, 0x62, 0x12, 0x1f, 0x0a, 0x0b, 0x64, + 0x69, 0x72, 0x5f, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x64, 0x69, 0x72, 0x42, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x12, 0x16, 0x0a, 0x06, + 0x63, 0x69, 0x70, 0x68, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x63, 0x69, + 0x70, 0x68, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, + 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, + 0x72, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x5f, 0x61, 0x64, + 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x73, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x5f, + 0x73, 0x65, 0x63, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x12, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x53, 0x65, 0x63, 0x12, 0x18, 0x0a, + 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x95, 0x01, 0x0a, 0x18, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x70, 0x72, + 0x65, 0x66, 0x69, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x74, 0x68, + 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x5f, + 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x4e, + 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, + 0x9a, 0x01, 0x0a, 0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c, 0x0a, + 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x4a, 0x0a, 0x12, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, + 0x70, 0x62, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x22, 0x61, 0x0a, 0x08, + 0x4c, 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x12, 0x2c, 0x0a, + 0x12, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x68, + 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x48, 0x61, 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, + 0x65, 0x0a, 0x14, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x67, + 0x72, 0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, + 0x67, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x31, 0x0a, 0x13, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x22, 0xcd, 0x01, 0x0a, 0x14, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, + 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, + 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x75, 0x6e, + 0x64, 0x12, 0x45, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, + 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x09, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x1a, 0x58, 0x0a, 0x08, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x61, 0x64, 0x64, + 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x67, 0x72, + 0x70, 0x63, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x75, + 0x6e, 0x74, 0x22, 0x20, 0x0a, 0x0c, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x22, 0x3b, 0x0a, 0x0d, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x22, 0x36, 0x0a, 0x0c, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x25, 0x0a, 0x0d, 0x4b, 0x76, 0x50, + 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x22, 0xeb, 0x02, 0x0a, 0x09, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x12, 0x18, + 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66, 0x69, + 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, + 0x2e, 0x50, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x87, 0x02, 0x0a, 0x08, 0x50, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, + 0x66, 0x12, 0x27, 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, + 0x65, 0x66, 0x69, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6c, 0x6f, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, + 0x74, 0x74, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x12, 0x1b, + 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, + 0x73, 0x79, 0x6e, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x73, 0x79, 0x6e, + 0x63, 0x12, 0x2e, 0x0a, 0x13, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x77, + 0x74, 0x68, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, + 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x74, 0x68, 0x43, 0x6f, 0x75, 0x6e, + 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x72, 0x65, 0x61, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x22, 0xba, + 0x01, 0x0a, 0x0a, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x12, 0x12, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x33, 0x5f, 0x61, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x33, + 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x33, 0x5f, + 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x73, 0x33, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x12, 0x1b, 0x0a, + 0x09, 0x73, 0x33, 0x5f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x73, 0x33, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x33, + 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x73, 0x33, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x22, 0xbe, 0x01, 0x0a, 0x14, + 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x12, 0x48, 0x0a, 0x08, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, + 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x73, 0x1a, 0x5c, + 0x0a, 0x0d, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, + 0x74, 0x65, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x57, 0x0a, 0x15, + 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x6f, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x75, 0x63, + 0x6b, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x75, 0x63, 0x6b, 0x65, + 0x74, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x70, 0x61, 0x74, 0x68, 0x32, 0xdc, 0x0c, 0x0a, 0x0c, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, + 0x64, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x12, 0x67, 0x0a, 0x14, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, + 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x25, + 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, + 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x4e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x1c, + 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, + 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, + 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, + 0x4c, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c, + 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, + 0x0b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0d, 0x41, + 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1e, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x4c, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c, + 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, + 0x11, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x74, + 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, + 0x62, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, + 0x0c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1d, 0x2e, + 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, + 0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1d, + 0x2e, 0x66, 0x69, 0x6c, 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, 0x1e, 0x2e, + 0x66, 0x69, 0x6c, 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, + 0x55, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, + 0x74, 0x12, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 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, 0x20, 0x2e, 0x66, 0x69, 0x6c, 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, 0x5b, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x66, 0x69, 0x6c, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, + 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x49, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x12, 0x1b, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, + 0x2e, 0x66, 0x69, 0x6c, 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, 0x6a, + 0x0a, 0x15, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, + 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x27, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, + 0x6c, 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, 0x60, 0x0a, 0x11, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, + 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x16, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x4d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, - 0x65, 0x0a, 0x16, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4c, 0x6f, 0x63, 0x61, - 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, - 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, - 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x56, 0x0a, 0x0d, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 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, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 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, 0x4f, - 0x0a, 0x0c, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x1d, - 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, - 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, - 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, - 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x3a, 0x0a, 0x05, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, - 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x17, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47, 0x65, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x4b, - 0x76, 0x50, 0x75, 0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, - 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x66, - 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x4f, 0x0a, 0x10, 0x73, 0x65, 0x61, 0x77, 0x65, - 0x65, 0x64, 0x66, 0x73, 0x2e, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x42, 0x0a, 0x46, 0x69, 0x6c, - 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2f, 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, - 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x30, 0x01, 0x12, 0x56, 0x0a, 0x0d, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x65, 0x64, 0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 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, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 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, 0x4f, 0x0a, 0x0c, 0x4c, + 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x1d, 0x2e, 0x66, 0x69, + 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, + 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66, 0x69, 0x6c, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, + 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x05, + 0x4b, 0x76, 0x47, 0x65, 0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, + 0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, + 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x4b, 0x76, 0x50, 0x75, + 0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x50, + 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x66, 0x69, 0x6c, 0x65, + 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x42, 0x4f, 0x0a, 0x10, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, + 0x73, 0x2e, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x42, 0x0a, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2f, 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, 0x66, 0x69, 0x6c, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -3945,129 +4091,134 @@ func file_filer_proto_rawDescGZIP() []byte { return file_filer_proto_rawDescData } -var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 54) +var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 57) var file_filer_proto_goTypes = []interface{}{ (*LookupDirectoryEntryRequest)(nil), // 0: filer_pb.LookupDirectoryEntryRequest (*LookupDirectoryEntryResponse)(nil), // 1: filer_pb.LookupDirectoryEntryResponse (*ListEntriesRequest)(nil), // 2: filer_pb.ListEntriesRequest (*ListEntriesResponse)(nil), // 3: filer_pb.ListEntriesResponse - (*Entry)(nil), // 4: filer_pb.Entry - (*FullEntry)(nil), // 5: filer_pb.FullEntry - (*EventNotification)(nil), // 6: filer_pb.EventNotification - (*FileChunk)(nil), // 7: filer_pb.FileChunk - (*FileChunkManifest)(nil), // 8: filer_pb.FileChunkManifest - (*FileId)(nil), // 9: filer_pb.FileId - (*FuseAttributes)(nil), // 10: filer_pb.FuseAttributes - (*CreateEntryRequest)(nil), // 11: filer_pb.CreateEntryRequest - (*CreateEntryResponse)(nil), // 12: filer_pb.CreateEntryResponse - (*UpdateEntryRequest)(nil), // 13: filer_pb.UpdateEntryRequest - (*UpdateEntryResponse)(nil), // 14: filer_pb.UpdateEntryResponse - (*AppendToEntryRequest)(nil), // 15: filer_pb.AppendToEntryRequest - (*AppendToEntryResponse)(nil), // 16: filer_pb.AppendToEntryResponse - (*DeleteEntryRequest)(nil), // 17: filer_pb.DeleteEntryRequest - (*DeleteEntryResponse)(nil), // 18: filer_pb.DeleteEntryResponse - (*AtomicRenameEntryRequest)(nil), // 19: filer_pb.AtomicRenameEntryRequest - (*AtomicRenameEntryResponse)(nil), // 20: filer_pb.AtomicRenameEntryResponse - (*AssignVolumeRequest)(nil), // 21: filer_pb.AssignVolumeRequest - (*AssignVolumeResponse)(nil), // 22: filer_pb.AssignVolumeResponse - (*LookupVolumeRequest)(nil), // 23: filer_pb.LookupVolumeRequest - (*Locations)(nil), // 24: filer_pb.Locations - (*Location)(nil), // 25: filer_pb.Location - (*LookupVolumeResponse)(nil), // 26: filer_pb.LookupVolumeResponse - (*Collection)(nil), // 27: filer_pb.Collection - (*CollectionListRequest)(nil), // 28: filer_pb.CollectionListRequest - (*CollectionListResponse)(nil), // 29: filer_pb.CollectionListResponse - (*DeleteCollectionRequest)(nil), // 30: filer_pb.DeleteCollectionRequest - (*DeleteCollectionResponse)(nil), // 31: filer_pb.DeleteCollectionResponse - (*StatisticsRequest)(nil), // 32: filer_pb.StatisticsRequest - (*StatisticsResponse)(nil), // 33: filer_pb.StatisticsResponse - (*GetFilerConfigurationRequest)(nil), // 34: filer_pb.GetFilerConfigurationRequest - (*GetFilerConfigurationResponse)(nil), // 35: filer_pb.GetFilerConfigurationResponse - (*SubscribeMetadataRequest)(nil), // 36: filer_pb.SubscribeMetadataRequest - (*SubscribeMetadataResponse)(nil), // 37: filer_pb.SubscribeMetadataResponse - (*LogEntry)(nil), // 38: filer_pb.LogEntry - (*KeepConnectedRequest)(nil), // 39: filer_pb.KeepConnectedRequest - (*KeepConnectedResponse)(nil), // 40: filer_pb.KeepConnectedResponse - (*LocateBrokerRequest)(nil), // 41: filer_pb.LocateBrokerRequest - (*LocateBrokerResponse)(nil), // 42: filer_pb.LocateBrokerResponse - (*KvGetRequest)(nil), // 43: filer_pb.KvGetRequest - (*KvGetResponse)(nil), // 44: filer_pb.KvGetResponse - (*KvPutRequest)(nil), // 45: filer_pb.KvPutRequest - (*KvPutResponse)(nil), // 46: filer_pb.KvPutResponse - (*FilerConf)(nil), // 47: filer_pb.FilerConf - (*RemoteConf)(nil), // 48: filer_pb.RemoteConf - nil, // 49: filer_pb.Entry.ExtendedEntry - (*Entry_Remote)(nil), // 50: filer_pb.Entry.Remote - nil, // 51: filer_pb.LookupVolumeResponse.LocationsMapEntry - (*LocateBrokerResponse_Resource)(nil), // 52: filer_pb.LocateBrokerResponse.Resource - (*FilerConf_PathConf)(nil), // 53: filer_pb.FilerConf.PathConf + (*RemoteEntry)(nil), // 4: filer_pb.RemoteEntry + (*Entry)(nil), // 5: filer_pb.Entry + (*FullEntry)(nil), // 6: filer_pb.FullEntry + (*EventNotification)(nil), // 7: filer_pb.EventNotification + (*FileChunk)(nil), // 8: filer_pb.FileChunk + (*FileChunkManifest)(nil), // 9: filer_pb.FileChunkManifest + (*FileId)(nil), // 10: filer_pb.FileId + (*FuseAttributes)(nil), // 11: filer_pb.FuseAttributes + (*CreateEntryRequest)(nil), // 12: filer_pb.CreateEntryRequest + (*CreateEntryResponse)(nil), // 13: filer_pb.CreateEntryResponse + (*UpdateEntryRequest)(nil), // 14: filer_pb.UpdateEntryRequest + (*UpdateEntryResponse)(nil), // 15: filer_pb.UpdateEntryResponse + (*AppendToEntryRequest)(nil), // 16: filer_pb.AppendToEntryRequest + (*AppendToEntryResponse)(nil), // 17: filer_pb.AppendToEntryResponse + (*DeleteEntryRequest)(nil), // 18: filer_pb.DeleteEntryRequest + (*DeleteEntryResponse)(nil), // 19: filer_pb.DeleteEntryResponse + (*AtomicRenameEntryRequest)(nil), // 20: filer_pb.AtomicRenameEntryRequest + (*AtomicRenameEntryResponse)(nil), // 21: filer_pb.AtomicRenameEntryResponse + (*AssignVolumeRequest)(nil), // 22: filer_pb.AssignVolumeRequest + (*AssignVolumeResponse)(nil), // 23: filer_pb.AssignVolumeResponse + (*LookupVolumeRequest)(nil), // 24: filer_pb.LookupVolumeRequest + (*Locations)(nil), // 25: filer_pb.Locations + (*Location)(nil), // 26: filer_pb.Location + (*LookupVolumeResponse)(nil), // 27: filer_pb.LookupVolumeResponse + (*Collection)(nil), // 28: filer_pb.Collection + (*CollectionListRequest)(nil), // 29: filer_pb.CollectionListRequest + (*CollectionListResponse)(nil), // 30: filer_pb.CollectionListResponse + (*DeleteCollectionRequest)(nil), // 31: filer_pb.DeleteCollectionRequest + (*DeleteCollectionResponse)(nil), // 32: filer_pb.DeleteCollectionResponse + (*StatisticsRequest)(nil), // 33: filer_pb.StatisticsRequest + (*StatisticsResponse)(nil), // 34: filer_pb.StatisticsResponse + (*GetFilerConfigurationRequest)(nil), // 35: filer_pb.GetFilerConfigurationRequest + (*GetFilerConfigurationResponse)(nil), // 36: filer_pb.GetFilerConfigurationResponse + (*SubscribeMetadataRequest)(nil), // 37: filer_pb.SubscribeMetadataRequest + (*SubscribeMetadataResponse)(nil), // 38: filer_pb.SubscribeMetadataResponse + (*LogEntry)(nil), // 39: filer_pb.LogEntry + (*KeepConnectedRequest)(nil), // 40: filer_pb.KeepConnectedRequest + (*KeepConnectedResponse)(nil), // 41: filer_pb.KeepConnectedResponse + (*LocateBrokerRequest)(nil), // 42: filer_pb.LocateBrokerRequest + (*LocateBrokerResponse)(nil), // 43: filer_pb.LocateBrokerResponse + (*KvGetRequest)(nil), // 44: filer_pb.KvGetRequest + (*KvGetResponse)(nil), // 45: filer_pb.KvGetResponse + (*KvPutRequest)(nil), // 46: filer_pb.KvPutRequest + (*KvPutResponse)(nil), // 47: filer_pb.KvPutResponse + (*FilerConf)(nil), // 48: filer_pb.FilerConf + (*RemoteConf)(nil), // 49: filer_pb.RemoteConf + (*RemoteStorageMapping)(nil), // 50: filer_pb.RemoteStorageMapping + (*RemoteStorageLocation)(nil), // 51: filer_pb.RemoteStorageLocation + nil, // 52: filer_pb.Entry.ExtendedEntry + nil, // 53: filer_pb.LookupVolumeResponse.LocationsMapEntry + (*LocateBrokerResponse_Resource)(nil), // 54: filer_pb.LocateBrokerResponse.Resource + (*FilerConf_PathConf)(nil), // 55: filer_pb.FilerConf.PathConf + nil, // 56: filer_pb.RemoteStorageMapping.MappingsEntry } var file_filer_proto_depIdxs = []int32{ - 4, // 0: filer_pb.LookupDirectoryEntryResponse.entry:type_name -> filer_pb.Entry - 4, // 1: filer_pb.ListEntriesResponse.entry:type_name -> filer_pb.Entry - 7, // 2: filer_pb.Entry.chunks:type_name -> filer_pb.FileChunk - 10, // 3: filer_pb.Entry.attributes:type_name -> filer_pb.FuseAttributes - 49, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry - 50, // 5: filer_pb.Entry.remote:type_name -> filer_pb.Entry.Remote - 4, // 6: filer_pb.FullEntry.entry:type_name -> filer_pb.Entry - 4, // 7: filer_pb.EventNotification.old_entry:type_name -> filer_pb.Entry - 4, // 8: filer_pb.EventNotification.new_entry:type_name -> filer_pb.Entry - 9, // 9: filer_pb.FileChunk.fid:type_name -> filer_pb.FileId - 9, // 10: filer_pb.FileChunk.source_fid:type_name -> filer_pb.FileId - 7, // 11: filer_pb.FileChunkManifest.chunks:type_name -> filer_pb.FileChunk - 4, // 12: filer_pb.CreateEntryRequest.entry:type_name -> filer_pb.Entry - 4, // 13: filer_pb.UpdateEntryRequest.entry:type_name -> filer_pb.Entry - 7, // 14: filer_pb.AppendToEntryRequest.chunks:type_name -> filer_pb.FileChunk - 25, // 15: filer_pb.Locations.locations:type_name -> filer_pb.Location - 51, // 16: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry - 27, // 17: filer_pb.CollectionListResponse.collections:type_name -> filer_pb.Collection - 6, // 18: filer_pb.SubscribeMetadataResponse.event_notification:type_name -> filer_pb.EventNotification - 52, // 19: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource - 53, // 20: filer_pb.FilerConf.locations:type_name -> filer_pb.FilerConf.PathConf - 24, // 21: filer_pb.LookupVolumeResponse.LocationsMapEntry.value:type_name -> filer_pb.Locations - 0, // 22: filer_pb.SeaweedFiler.LookupDirectoryEntry:input_type -> filer_pb.LookupDirectoryEntryRequest - 2, // 23: filer_pb.SeaweedFiler.ListEntries:input_type -> filer_pb.ListEntriesRequest - 11, // 24: filer_pb.SeaweedFiler.CreateEntry:input_type -> filer_pb.CreateEntryRequest - 13, // 25: filer_pb.SeaweedFiler.UpdateEntry:input_type -> filer_pb.UpdateEntryRequest - 15, // 26: filer_pb.SeaweedFiler.AppendToEntry:input_type -> filer_pb.AppendToEntryRequest - 17, // 27: filer_pb.SeaweedFiler.DeleteEntry:input_type -> filer_pb.DeleteEntryRequest - 19, // 28: filer_pb.SeaweedFiler.AtomicRenameEntry:input_type -> filer_pb.AtomicRenameEntryRequest - 21, // 29: filer_pb.SeaweedFiler.AssignVolume:input_type -> filer_pb.AssignVolumeRequest - 23, // 30: filer_pb.SeaweedFiler.LookupVolume:input_type -> filer_pb.LookupVolumeRequest - 28, // 31: filer_pb.SeaweedFiler.CollectionList:input_type -> filer_pb.CollectionListRequest - 30, // 32: filer_pb.SeaweedFiler.DeleteCollection:input_type -> filer_pb.DeleteCollectionRequest - 32, // 33: filer_pb.SeaweedFiler.Statistics:input_type -> filer_pb.StatisticsRequest - 34, // 34: filer_pb.SeaweedFiler.GetFilerConfiguration:input_type -> filer_pb.GetFilerConfigurationRequest - 36, // 35: filer_pb.SeaweedFiler.SubscribeMetadata:input_type -> filer_pb.SubscribeMetadataRequest - 36, // 36: filer_pb.SeaweedFiler.SubscribeLocalMetadata:input_type -> filer_pb.SubscribeMetadataRequest - 39, // 37: filer_pb.SeaweedFiler.KeepConnected:input_type -> filer_pb.KeepConnectedRequest - 41, // 38: filer_pb.SeaweedFiler.LocateBroker:input_type -> filer_pb.LocateBrokerRequest - 43, // 39: filer_pb.SeaweedFiler.KvGet:input_type -> filer_pb.KvGetRequest - 45, // 40: filer_pb.SeaweedFiler.KvPut:input_type -> filer_pb.KvPutRequest - 1, // 41: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse - 3, // 42: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse - 12, // 43: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse - 14, // 44: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse - 16, // 45: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse - 18, // 46: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse - 20, // 47: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse - 22, // 48: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse - 26, // 49: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse - 29, // 50: filer_pb.SeaweedFiler.CollectionList:output_type -> filer_pb.CollectionListResponse - 31, // 51: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse - 33, // 52: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse - 35, // 53: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse - 37, // 54: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse - 37, // 55: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse - 40, // 56: filer_pb.SeaweedFiler.KeepConnected:output_type -> filer_pb.KeepConnectedResponse - 42, // 57: filer_pb.SeaweedFiler.LocateBroker:output_type -> filer_pb.LocateBrokerResponse - 44, // 58: filer_pb.SeaweedFiler.KvGet:output_type -> filer_pb.KvGetResponse - 46, // 59: filer_pb.SeaweedFiler.KvPut:output_type -> filer_pb.KvPutResponse - 41, // [41:60] is the sub-list for method output_type - 22, // [22:41] is the sub-list for method input_type - 22, // [22:22] is the sub-list for extension type_name - 22, // [22:22] is the sub-list for extension extendee - 0, // [0:22] is the sub-list for field type_name + 5, // 0: filer_pb.LookupDirectoryEntryResponse.entry:type_name -> filer_pb.Entry + 5, // 1: filer_pb.ListEntriesResponse.entry:type_name -> filer_pb.Entry + 8, // 2: filer_pb.Entry.chunks:type_name -> filer_pb.FileChunk + 11, // 3: filer_pb.Entry.attributes:type_name -> filer_pb.FuseAttributes + 52, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry + 4, // 5: filer_pb.Entry.remote_entry:type_name -> filer_pb.RemoteEntry + 5, // 6: filer_pb.FullEntry.entry:type_name -> filer_pb.Entry + 5, // 7: filer_pb.EventNotification.old_entry:type_name -> filer_pb.Entry + 5, // 8: filer_pb.EventNotification.new_entry:type_name -> filer_pb.Entry + 10, // 9: filer_pb.FileChunk.fid:type_name -> filer_pb.FileId + 10, // 10: filer_pb.FileChunk.source_fid:type_name -> filer_pb.FileId + 8, // 11: filer_pb.FileChunkManifest.chunks:type_name -> filer_pb.FileChunk + 5, // 12: filer_pb.CreateEntryRequest.entry:type_name -> filer_pb.Entry + 5, // 13: filer_pb.UpdateEntryRequest.entry:type_name -> filer_pb.Entry + 8, // 14: filer_pb.AppendToEntryRequest.chunks:type_name -> filer_pb.FileChunk + 26, // 15: filer_pb.Locations.locations:type_name -> filer_pb.Location + 53, // 16: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry + 28, // 17: filer_pb.CollectionListResponse.collections:type_name -> filer_pb.Collection + 7, // 18: filer_pb.SubscribeMetadataResponse.event_notification:type_name -> filer_pb.EventNotification + 54, // 19: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource + 55, // 20: filer_pb.FilerConf.locations:type_name -> filer_pb.FilerConf.PathConf + 56, // 21: filer_pb.RemoteStorageMapping.mappings:type_name -> filer_pb.RemoteStorageMapping.MappingsEntry + 25, // 22: filer_pb.LookupVolumeResponse.LocationsMapEntry.value:type_name -> filer_pb.Locations + 51, // 23: filer_pb.RemoteStorageMapping.MappingsEntry.value:type_name -> filer_pb.RemoteStorageLocation + 0, // 24: filer_pb.SeaweedFiler.LookupDirectoryEntry:input_type -> filer_pb.LookupDirectoryEntryRequest + 2, // 25: filer_pb.SeaweedFiler.ListEntries:input_type -> filer_pb.ListEntriesRequest + 12, // 26: filer_pb.SeaweedFiler.CreateEntry:input_type -> filer_pb.CreateEntryRequest + 14, // 27: filer_pb.SeaweedFiler.UpdateEntry:input_type -> filer_pb.UpdateEntryRequest + 16, // 28: filer_pb.SeaweedFiler.AppendToEntry:input_type -> filer_pb.AppendToEntryRequest + 18, // 29: filer_pb.SeaweedFiler.DeleteEntry:input_type -> filer_pb.DeleteEntryRequest + 20, // 30: filer_pb.SeaweedFiler.AtomicRenameEntry:input_type -> filer_pb.AtomicRenameEntryRequest + 22, // 31: filer_pb.SeaweedFiler.AssignVolume:input_type -> filer_pb.AssignVolumeRequest + 24, // 32: filer_pb.SeaweedFiler.LookupVolume:input_type -> filer_pb.LookupVolumeRequest + 29, // 33: filer_pb.SeaweedFiler.CollectionList:input_type -> filer_pb.CollectionListRequest + 31, // 34: filer_pb.SeaweedFiler.DeleteCollection:input_type -> filer_pb.DeleteCollectionRequest + 33, // 35: filer_pb.SeaweedFiler.Statistics:input_type -> filer_pb.StatisticsRequest + 35, // 36: filer_pb.SeaweedFiler.GetFilerConfiguration:input_type -> filer_pb.GetFilerConfigurationRequest + 37, // 37: filer_pb.SeaweedFiler.SubscribeMetadata:input_type -> filer_pb.SubscribeMetadataRequest + 37, // 38: filer_pb.SeaweedFiler.SubscribeLocalMetadata:input_type -> filer_pb.SubscribeMetadataRequest + 40, // 39: filer_pb.SeaweedFiler.KeepConnected:input_type -> filer_pb.KeepConnectedRequest + 42, // 40: filer_pb.SeaweedFiler.LocateBroker:input_type -> filer_pb.LocateBrokerRequest + 44, // 41: filer_pb.SeaweedFiler.KvGet:input_type -> filer_pb.KvGetRequest + 46, // 42: filer_pb.SeaweedFiler.KvPut:input_type -> filer_pb.KvPutRequest + 1, // 43: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse + 3, // 44: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse + 13, // 45: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse + 15, // 46: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse + 17, // 47: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse + 19, // 48: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse + 21, // 49: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse + 23, // 50: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse + 27, // 51: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse + 30, // 52: filer_pb.SeaweedFiler.CollectionList:output_type -> filer_pb.CollectionListResponse + 32, // 53: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse + 34, // 54: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse + 36, // 55: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse + 38, // 56: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse + 38, // 57: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse + 41, // 58: filer_pb.SeaweedFiler.KeepConnected:output_type -> filer_pb.KeepConnectedResponse + 43, // 59: filer_pb.SeaweedFiler.LocateBroker:output_type -> filer_pb.LocateBrokerResponse + 45, // 60: filer_pb.SeaweedFiler.KvGet:output_type -> filer_pb.KvGetResponse + 47, // 61: filer_pb.SeaweedFiler.KvPut:output_type -> filer_pb.KvPutResponse + 43, // [43:62] is the sub-list for method output_type + 24, // [24:43] is the sub-list for method input_type + 24, // [24:24] is the sub-list for extension type_name + 24, // [24:24] is the sub-list for extension extendee + 0, // [0:24] is the sub-list for field type_name } func init() { file_filer_proto_init() } @@ -4125,7 +4276,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Entry); i { + switch v := v.(*RemoteEntry); i { case 0: return &v.state case 1: @@ -4137,7 +4288,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FullEntry); i { + switch v := v.(*Entry); i { case 0: return &v.state case 1: @@ -4149,7 +4300,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EventNotification); i { + switch v := v.(*FullEntry); i { case 0: return &v.state case 1: @@ -4161,7 +4312,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FileChunk); i { + switch v := v.(*EventNotification); i { case 0: return &v.state case 1: @@ -4173,7 +4324,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FileChunkManifest); i { + switch v := v.(*FileChunk); i { case 0: return &v.state case 1: @@ -4185,7 +4336,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FileId); i { + switch v := v.(*FileChunkManifest); i { case 0: return &v.state case 1: @@ -4197,7 +4348,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FuseAttributes); i { + switch v := v.(*FileId); i { case 0: return &v.state case 1: @@ -4209,7 +4360,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateEntryRequest); i { + switch v := v.(*FuseAttributes); i { case 0: return &v.state case 1: @@ -4221,7 +4372,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateEntryResponse); i { + switch v := v.(*CreateEntryRequest); i { case 0: return &v.state case 1: @@ -4233,7 +4384,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateEntryRequest); i { + switch v := v.(*CreateEntryResponse); i { case 0: return &v.state case 1: @@ -4245,7 +4396,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateEntryResponse); i { + switch v := v.(*UpdateEntryRequest); i { case 0: return &v.state case 1: @@ -4257,7 +4408,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AppendToEntryRequest); i { + switch v := v.(*UpdateEntryResponse); i { case 0: return &v.state case 1: @@ -4269,7 +4420,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AppendToEntryResponse); i { + switch v := v.(*AppendToEntryRequest); i { case 0: return &v.state case 1: @@ -4281,7 +4432,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteEntryRequest); i { + switch v := v.(*AppendToEntryResponse); i { case 0: return &v.state case 1: @@ -4293,7 +4444,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteEntryResponse); i { + switch v := v.(*DeleteEntryRequest); i { case 0: return &v.state case 1: @@ -4305,7 +4456,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AtomicRenameEntryRequest); i { + switch v := v.(*DeleteEntryResponse); i { case 0: return &v.state case 1: @@ -4317,7 +4468,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AtomicRenameEntryResponse); i { + switch v := v.(*AtomicRenameEntryRequest); i { case 0: return &v.state case 1: @@ -4329,7 +4480,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignVolumeRequest); i { + switch v := v.(*AtomicRenameEntryResponse); i { case 0: return &v.state case 1: @@ -4341,7 +4492,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignVolumeResponse); i { + switch v := v.(*AssignVolumeRequest); i { case 0: return &v.state case 1: @@ -4353,7 +4504,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LookupVolumeRequest); i { + switch v := v.(*AssignVolumeResponse); i { case 0: return &v.state case 1: @@ -4365,7 +4516,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Locations); i { + switch v := v.(*LookupVolumeRequest); i { case 0: return &v.state case 1: @@ -4377,7 +4528,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Location); i { + switch v := v.(*Locations); i { case 0: return &v.state case 1: @@ -4389,7 +4540,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LookupVolumeResponse); i { + switch v := v.(*Location); i { case 0: return &v.state case 1: @@ -4401,7 +4552,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Collection); i { + switch v := v.(*LookupVolumeResponse); i { case 0: return &v.state case 1: @@ -4413,7 +4564,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CollectionListRequest); i { + switch v := v.(*Collection); i { case 0: return &v.state case 1: @@ -4425,7 +4576,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CollectionListResponse); i { + switch v := v.(*CollectionListRequest); i { case 0: return &v.state case 1: @@ -4437,7 +4588,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteCollectionRequest); i { + switch v := v.(*CollectionListResponse); i { case 0: return &v.state case 1: @@ -4449,7 +4600,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteCollectionResponse); i { + switch v := v.(*DeleteCollectionRequest); i { case 0: return &v.state case 1: @@ -4461,7 +4612,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StatisticsRequest); i { + switch v := v.(*DeleteCollectionResponse); i { case 0: return &v.state case 1: @@ -4473,7 +4624,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StatisticsResponse); i { + switch v := v.(*StatisticsRequest); i { case 0: return &v.state case 1: @@ -4485,7 +4636,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetFilerConfigurationRequest); i { + switch v := v.(*StatisticsResponse); i { case 0: return &v.state case 1: @@ -4497,7 +4648,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetFilerConfigurationResponse); i { + switch v := v.(*GetFilerConfigurationRequest); i { case 0: return &v.state case 1: @@ -4509,7 +4660,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeMetadataRequest); i { + switch v := v.(*GetFilerConfigurationResponse); i { case 0: return &v.state case 1: @@ -4521,7 +4672,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeMetadataResponse); i { + switch v := v.(*SubscribeMetadataRequest); i { case 0: return &v.state case 1: @@ -4533,7 +4684,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogEntry); i { + switch v := v.(*SubscribeMetadataResponse); i { case 0: return &v.state case 1: @@ -4545,7 +4696,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KeepConnectedRequest); i { + switch v := v.(*LogEntry); i { case 0: return &v.state case 1: @@ -4557,7 +4708,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KeepConnectedResponse); i { + switch v := v.(*KeepConnectedRequest); i { case 0: return &v.state case 1: @@ -4569,7 +4720,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LocateBrokerRequest); i { + switch v := v.(*KeepConnectedResponse); i { case 0: return &v.state case 1: @@ -4581,7 +4732,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LocateBrokerResponse); i { + switch v := v.(*LocateBrokerRequest); i { case 0: return &v.state case 1: @@ -4593,7 +4744,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KvGetRequest); i { + switch v := v.(*LocateBrokerResponse); i { case 0: return &v.state case 1: @@ -4605,7 +4756,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KvGetResponse); i { + switch v := v.(*KvGetRequest); i { case 0: return &v.state case 1: @@ -4617,7 +4768,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KvPutRequest); i { + switch v := v.(*KvGetResponse); i { case 0: return &v.state case 1: @@ -4629,7 +4780,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KvPutResponse); i { + switch v := v.(*KvPutRequest); i { case 0: return &v.state case 1: @@ -4641,7 +4792,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FilerConf); i { + switch v := v.(*KvPutResponse); i { case 0: return &v.state case 1: @@ -4653,6 +4804,18 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FilerConf); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_filer_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoteConf); i { case 0: return &v.state @@ -4665,7 +4828,7 @@ func file_filer_proto_init() { } } file_filer_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Entry_Remote); i { + switch v := v.(*RemoteStorageMapping); i { case 0: return &v.state case 1: @@ -4676,7 +4839,19 @@ func file_filer_proto_init() { return nil } } - file_filer_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + file_filer_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RemoteStorageLocation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_filer_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LocateBrokerResponse_Resource); i { case 0: return &v.state @@ -4688,7 +4863,7 @@ func file_filer_proto_init() { return nil } } - file_filer_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + file_filer_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*FilerConf_PathConf); i { case 0: return &v.state @@ -4707,7 +4882,7 @@ func file_filer_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_filer_proto_rawDesc, NumEnums: 0, - NumMessages: 54, + NumMessages: 57, NumExtensions: 0, NumServices: 1, }, diff --git a/weed/pb/filer_pb/filer_pb_helper.go b/weed/pb/filer_pb/filer_pb_helper.go index b46385c8f..ae282db75 100644 --- a/weed/pb/filer_pb/filer_pb_helper.go +++ b/weed/pb/filer_pb/filer_pb_helper.go @@ -147,3 +147,7 @@ func (fp *FilerConf_PathConf) Key() interface{} { key, _ := proto.Marshal(fp) return string(key) } +func (fp *RemoteStorageLocation) Key() interface{} { + key, _ := proto.Marshal(fp) + return string(key) +} diff --git a/weed/pb/filer_pb_tail.go b/weed/pb/filer_pb_tail.go new file mode 100644 index 000000000..31fb62fb3 --- /dev/null +++ b/weed/pb/filer_pb_tail.go @@ -0,0 +1,94 @@ +package pb + +import ( + "context" + "fmt" + "github.com/chrislusf/seaweedfs/weed/glog" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "google.golang.org/grpc" + "io" + "time" +) + +type ProcessMetadataFunc func(resp *filer_pb.SubscribeMetadataResponse) error + +func FollowMetadata(filerAddress string, grpcDialOption grpc.DialOption, + clientName string, pathPrefix string, lastTsNs int64, selfSignature int32, + processEventFn ProcessMetadataFunc, fatalOnError bool) error { + + err := WithFilerClient(filerAddress, grpcDialOption, makeFunc( + clientName, pathPrefix, lastTsNs, selfSignature, processEventFn, fatalOnError)) + if err != nil { + return fmt.Errorf("subscribing filer meta change: %v", err) + } + return err +} + +func WithFilerClientFollowMetadata(filerClient filer_pb.FilerClient, + clientName string, pathPrefix string, lastTsNs int64, selfSignature int32, + processEventFn ProcessMetadataFunc, fatalOnError bool) error { + + err := filerClient.WithFilerClient(makeFunc( + clientName, pathPrefix, lastTsNs, selfSignature, processEventFn, fatalOnError)) + if err != nil { + return fmt.Errorf("subscribing filer meta change: %v", err) + } + + return nil +} + +func makeFunc(clientName string, pathPrefix string, lastTsNs int64, selfSignature int32, + processEventFn ProcessMetadataFunc, fatalOnError bool) func(client filer_pb.SeaweedFilerClient) error { + return func(client filer_pb.SeaweedFilerClient) error { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ + ClientName: clientName, + PathPrefix: pathPrefix, + SinceNs: lastTsNs, + Signature: selfSignature, + }) + if err != nil { + return fmt.Errorf("subscribe: %v", err) + } + + for { + resp, listenErr := stream.Recv() + if listenErr == io.EOF { + return nil + } + if listenErr != nil { + return listenErr + } + + if err := processEventFn(resp); err != nil { + if fatalOnError { + glog.Fatalf("process %v: %v", resp, err) + } else { + glog.Errorf("process %v: %v", resp, err) + } + } + lastTsNs = resp.TsNs + } + } +} + +func AddOffsetFunc(processEventFn ProcessMetadataFunc, offsetInterval time.Duration, offsetFunc func(counter int64, offset int64) error) ProcessMetadataFunc { + var counter int64 + var lastWriteTime time.Time + return func(resp *filer_pb.SubscribeMetadataResponse) error { + if err := processEventFn(resp); err != nil { + return err + } + counter++ + if lastWriteTime.Add(offsetInterval).Before(time.Now()) { + counter = 0 + lastWriteTime = time.Now() + if err := offsetFunc(counter, resp.TsNs); err != nil { + return err + } + } + return nil + } + +} diff --git a/weed/pb/volume_server.proto b/weed/pb/volume_server.proto index f9836c402..b1f2487ba 100644 --- a/weed/pb/volume_server.proto +++ b/weed/pb/volume_server.proto @@ -93,6 +93,10 @@ service VolumeServer { rpc VolumeServerLeave (VolumeServerLeaveRequest) returns (VolumeServerLeaveResponse) { } + // remote storage + rpc FetchAndWriteNeedle (FetchAndWriteNeedleRequest) returns (FetchAndWriteNeedleResponse) { + } + // query rpc Query (QueryRequest) returns (stream QueriedStripe) { } @@ -426,6 +430,7 @@ message VolumeInfo { string replication = 3; } +// tiered storage message VolumeTierMoveDatToRemoteRequest { uint32 volume_id = 1; string collection = 2; @@ -460,6 +465,25 @@ message VolumeServerLeaveRequest { message VolumeServerLeaveResponse { } +// remote storage +message FetchAndWriteNeedleRequest { + uint32 volume_id = 1; + uint64 needle_id = 2; + int64 offset = 3; + int64 size = 4; + // remote conf + string remote_type = 5; + string remote_name = 6; + string s3_access_key = 8; + string s3_secret_key = 9; + string s3_region = 10; + string s3_endpoint = 11; + string remote_bucket = 12; + string remote_key = 13; +} +message FetchAndWriteNeedleResponse { +} + // select on volume servers message QueryRequest { repeated string selections = 1; diff --git a/weed/pb/volume_server_pb/volume_server.pb.go b/weed/pb/volume_server_pb/volume_server.pb.go index c642142ba..b360349e4 100644 --- a/weed/pb/volume_server_pb/volume_server.pb.go +++ b/weed/pb/volume_server_pb/volume_server.pb.go @@ -3901,6 +3901,7 @@ func (x *VolumeInfo) GetReplication() string { return "" } +// tiered storage type VolumeTierMoveDatToRemoteRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4314,6 +4315,181 @@ func (*VolumeServerLeaveResponse) Descriptor() ([]byte, []int) { return file_volume_server_proto_rawDescGZIP(), []int{77} } +// remote storage +type FetchAndWriteNeedleRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"` + NeedleId uint64 `protobuf:"varint,2,opt,name=needle_id,json=needleId,proto3" json:"needle_id,omitempty"` + Offset int64 `protobuf:"varint,3,opt,name=offset,proto3" json:"offset,omitempty"` + Size int64 `protobuf:"varint,4,opt,name=size,proto3" json:"size,omitempty"` + // remote conf + RemoteType string `protobuf:"bytes,5,opt,name=remote_type,json=remoteType,proto3" json:"remote_type,omitempty"` + RemoteName string `protobuf:"bytes,6,opt,name=remote_name,json=remoteName,proto3" json:"remote_name,omitempty"` + S3AccessKey string `protobuf:"bytes,8,opt,name=s3_access_key,json=s3AccessKey,proto3" json:"s3_access_key,omitempty"` + S3SecretKey string `protobuf:"bytes,9,opt,name=s3_secret_key,json=s3SecretKey,proto3" json:"s3_secret_key,omitempty"` + S3Region string `protobuf:"bytes,10,opt,name=s3_region,json=s3Region,proto3" json:"s3_region,omitempty"` + S3Endpoint string `protobuf:"bytes,11,opt,name=s3_endpoint,json=s3Endpoint,proto3" json:"s3_endpoint,omitempty"` + RemoteBucket string `protobuf:"bytes,12,opt,name=remote_bucket,json=remoteBucket,proto3" json:"remote_bucket,omitempty"` + RemoteKey string `protobuf:"bytes,13,opt,name=remote_key,json=remoteKey,proto3" json:"remote_key,omitempty"` +} + +func (x *FetchAndWriteNeedleRequest) Reset() { + *x = FetchAndWriteNeedleRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_volume_server_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FetchAndWriteNeedleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FetchAndWriteNeedleRequest) ProtoMessage() {} + +func (x *FetchAndWriteNeedleRequest) ProtoReflect() protoreflect.Message { + mi := &file_volume_server_proto_msgTypes[78] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FetchAndWriteNeedleRequest.ProtoReflect.Descriptor instead. +func (*FetchAndWriteNeedleRequest) Descriptor() ([]byte, []int) { + return file_volume_server_proto_rawDescGZIP(), []int{78} +} + +func (x *FetchAndWriteNeedleRequest) GetVolumeId() uint32 { + if x != nil { + return x.VolumeId + } + return 0 +} + +func (x *FetchAndWriteNeedleRequest) GetNeedleId() uint64 { + if x != nil { + return x.NeedleId + } + return 0 +} + +func (x *FetchAndWriteNeedleRequest) GetOffset() int64 { + if x != nil { + return x.Offset + } + return 0 +} + +func (x *FetchAndWriteNeedleRequest) GetSize() int64 { + if x != nil { + return x.Size + } + return 0 +} + +func (x *FetchAndWriteNeedleRequest) GetRemoteType() string { + if x != nil { + return x.RemoteType + } + return "" +} + +func (x *FetchAndWriteNeedleRequest) GetRemoteName() string { + if x != nil { + return x.RemoteName + } + return "" +} + +func (x *FetchAndWriteNeedleRequest) GetS3AccessKey() string { + if x != nil { + return x.S3AccessKey + } + return "" +} + +func (x *FetchAndWriteNeedleRequest) GetS3SecretKey() string { + if x != nil { + return x.S3SecretKey + } + return "" +} + +func (x *FetchAndWriteNeedleRequest) GetS3Region() string { + if x != nil { + return x.S3Region + } + return "" +} + +func (x *FetchAndWriteNeedleRequest) GetS3Endpoint() string { + if x != nil { + return x.S3Endpoint + } + return "" +} + +func (x *FetchAndWriteNeedleRequest) GetRemoteBucket() string { + if x != nil { + return x.RemoteBucket + } + return "" +} + +func (x *FetchAndWriteNeedleRequest) GetRemoteKey() string { + if x != nil { + return x.RemoteKey + } + return "" +} + +type FetchAndWriteNeedleResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *FetchAndWriteNeedleResponse) Reset() { + *x = FetchAndWriteNeedleResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_volume_server_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FetchAndWriteNeedleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FetchAndWriteNeedleResponse) ProtoMessage() {} + +func (x *FetchAndWriteNeedleResponse) ProtoReflect() protoreflect.Message { + mi := &file_volume_server_proto_msgTypes[79] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FetchAndWriteNeedleResponse.ProtoReflect.Descriptor instead. +func (*FetchAndWriteNeedleResponse) Descriptor() ([]byte, []int) { + return file_volume_server_proto_rawDescGZIP(), []int{79} +} + // select on volume servers type QueryRequest struct { state protoimpl.MessageState @@ -4330,7 +4506,7 @@ type QueryRequest struct { func (x *QueryRequest) Reset() { *x = QueryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[78] + mi := &file_volume_server_proto_msgTypes[80] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4343,7 +4519,7 @@ func (x *QueryRequest) String() string { func (*QueryRequest) ProtoMessage() {} func (x *QueryRequest) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[78] + mi := &file_volume_server_proto_msgTypes[80] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4356,7 +4532,7 @@ func (x *QueryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRequest.ProtoReflect.Descriptor instead. func (*QueryRequest) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78} + return file_volume_server_proto_rawDescGZIP(), []int{80} } func (x *QueryRequest) GetSelections() []string { @@ -4405,7 +4581,7 @@ type QueriedStripe struct { func (x *QueriedStripe) Reset() { *x = QueriedStripe{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[79] + mi := &file_volume_server_proto_msgTypes[81] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4418,7 +4594,7 @@ func (x *QueriedStripe) String() string { func (*QueriedStripe) ProtoMessage() {} func (x *QueriedStripe) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[79] + mi := &file_volume_server_proto_msgTypes[81] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4431,7 +4607,7 @@ func (x *QueriedStripe) ProtoReflect() protoreflect.Message { // Deprecated: Use QueriedStripe.ProtoReflect.Descriptor instead. func (*QueriedStripe) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{79} + return file_volume_server_proto_rawDescGZIP(), []int{81} } func (x *QueriedStripe) GetRecords() []byte { @@ -4453,7 +4629,7 @@ type VolumeNeedleStatusRequest struct { func (x *VolumeNeedleStatusRequest) Reset() { *x = VolumeNeedleStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[80] + mi := &file_volume_server_proto_msgTypes[82] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4466,7 +4642,7 @@ func (x *VolumeNeedleStatusRequest) String() string { func (*VolumeNeedleStatusRequest) ProtoMessage() {} func (x *VolumeNeedleStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[80] + mi := &file_volume_server_proto_msgTypes[82] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4479,7 +4655,7 @@ func (x *VolumeNeedleStatusRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeNeedleStatusRequest.ProtoReflect.Descriptor instead. func (*VolumeNeedleStatusRequest) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{80} + return file_volume_server_proto_rawDescGZIP(), []int{82} } func (x *VolumeNeedleStatusRequest) GetVolumeId() uint32 { @@ -4512,7 +4688,7 @@ type VolumeNeedleStatusResponse struct { func (x *VolumeNeedleStatusResponse) Reset() { *x = VolumeNeedleStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[81] + mi := &file_volume_server_proto_msgTypes[83] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4525,7 +4701,7 @@ func (x *VolumeNeedleStatusResponse) String() string { func (*VolumeNeedleStatusResponse) ProtoMessage() {} func (x *VolumeNeedleStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[81] + mi := &file_volume_server_proto_msgTypes[83] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4538,7 +4714,7 @@ func (x *VolumeNeedleStatusResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VolumeNeedleStatusResponse.ProtoReflect.Descriptor instead. func (*VolumeNeedleStatusResponse) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{81} + return file_volume_server_proto_rawDescGZIP(), []int{83} } func (x *VolumeNeedleStatusResponse) GetNeedleId() uint64 { @@ -4596,7 +4772,7 @@ type QueryRequest_Filter struct { func (x *QueryRequest_Filter) Reset() { *x = QueryRequest_Filter{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[82] + mi := &file_volume_server_proto_msgTypes[84] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4609,7 +4785,7 @@ func (x *QueryRequest_Filter) String() string { func (*QueryRequest_Filter) ProtoMessage() {} func (x *QueryRequest_Filter) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[82] + mi := &file_volume_server_proto_msgTypes[84] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4622,7 +4798,7 @@ func (x *QueryRequest_Filter) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRequest_Filter.ProtoReflect.Descriptor instead. func (*QueryRequest_Filter) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 0} + return file_volume_server_proto_rawDescGZIP(), []int{80, 0} } func (x *QueryRequest_Filter) GetField() string { @@ -4661,7 +4837,7 @@ type QueryRequest_InputSerialization struct { func (x *QueryRequest_InputSerialization) Reset() { *x = QueryRequest_InputSerialization{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[83] + mi := &file_volume_server_proto_msgTypes[85] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4674,7 +4850,7 @@ func (x *QueryRequest_InputSerialization) String() string { func (*QueryRequest_InputSerialization) ProtoMessage() {} func (x *QueryRequest_InputSerialization) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[83] + mi := &file_volume_server_proto_msgTypes[85] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4687,7 +4863,7 @@ func (x *QueryRequest_InputSerialization) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRequest_InputSerialization.ProtoReflect.Descriptor instead. func (*QueryRequest_InputSerialization) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 1} + return file_volume_server_proto_rawDescGZIP(), []int{80, 1} } func (x *QueryRequest_InputSerialization) GetCompressionType() string { @@ -4730,7 +4906,7 @@ type QueryRequest_OutputSerialization struct { func (x *QueryRequest_OutputSerialization) Reset() { *x = QueryRequest_OutputSerialization{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[84] + mi := &file_volume_server_proto_msgTypes[86] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4743,7 +4919,7 @@ func (x *QueryRequest_OutputSerialization) String() string { func (*QueryRequest_OutputSerialization) ProtoMessage() {} func (x *QueryRequest_OutputSerialization) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[84] + mi := &file_volume_server_proto_msgTypes[86] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4756,7 +4932,7 @@ func (x *QueryRequest_OutputSerialization) ProtoReflect() protoreflect.Message { // Deprecated: Use QueryRequest_OutputSerialization.ProtoReflect.Descriptor instead. func (*QueryRequest_OutputSerialization) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 2} + return file_volume_server_proto_rawDescGZIP(), []int{80, 2} } func (x *QueryRequest_OutputSerialization) GetCsvOutput() *QueryRequest_OutputSerialization_CSVOutput { @@ -4791,7 +4967,7 @@ type QueryRequest_InputSerialization_CSVInput struct { func (x *QueryRequest_InputSerialization_CSVInput) Reset() { *x = QueryRequest_InputSerialization_CSVInput{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[85] + mi := &file_volume_server_proto_msgTypes[87] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4804,7 +4980,7 @@ func (x *QueryRequest_InputSerialization_CSVInput) String() string { func (*QueryRequest_InputSerialization_CSVInput) ProtoMessage() {} func (x *QueryRequest_InputSerialization_CSVInput) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[85] + mi := &file_volume_server_proto_msgTypes[87] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4817,7 +4993,7 @@ func (x *QueryRequest_InputSerialization_CSVInput) ProtoReflect() protoreflect.M // Deprecated: Use QueryRequest_InputSerialization_CSVInput.ProtoReflect.Descriptor instead. func (*QueryRequest_InputSerialization_CSVInput) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 0} + return file_volume_server_proto_rawDescGZIP(), []int{80, 1, 0} } func (x *QueryRequest_InputSerialization_CSVInput) GetFileHeaderInfo() string { @@ -4880,7 +5056,7 @@ type QueryRequest_InputSerialization_JSONInput struct { func (x *QueryRequest_InputSerialization_JSONInput) Reset() { *x = QueryRequest_InputSerialization_JSONInput{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[86] + mi := &file_volume_server_proto_msgTypes[88] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4893,7 +5069,7 @@ func (x *QueryRequest_InputSerialization_JSONInput) String() string { func (*QueryRequest_InputSerialization_JSONInput) ProtoMessage() {} func (x *QueryRequest_InputSerialization_JSONInput) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[86] + mi := &file_volume_server_proto_msgTypes[88] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4906,7 +5082,7 @@ func (x *QueryRequest_InputSerialization_JSONInput) ProtoReflect() protoreflect. // Deprecated: Use QueryRequest_InputSerialization_JSONInput.ProtoReflect.Descriptor instead. func (*QueryRequest_InputSerialization_JSONInput) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 1} + return file_volume_server_proto_rawDescGZIP(), []int{80, 1, 1} } func (x *QueryRequest_InputSerialization_JSONInput) GetType() string { @@ -4925,7 +5101,7 @@ type QueryRequest_InputSerialization_ParquetInput struct { func (x *QueryRequest_InputSerialization_ParquetInput) Reset() { *x = QueryRequest_InputSerialization_ParquetInput{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[87] + mi := &file_volume_server_proto_msgTypes[89] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4938,7 +5114,7 @@ func (x *QueryRequest_InputSerialization_ParquetInput) String() string { func (*QueryRequest_InputSerialization_ParquetInput) ProtoMessage() {} func (x *QueryRequest_InputSerialization_ParquetInput) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[87] + mi := &file_volume_server_proto_msgTypes[89] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4951,7 +5127,7 @@ func (x *QueryRequest_InputSerialization_ParquetInput) ProtoReflect() protorefle // Deprecated: Use QueryRequest_InputSerialization_ParquetInput.ProtoReflect.Descriptor instead. func (*QueryRequest_InputSerialization_ParquetInput) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 2} + return file_volume_server_proto_rawDescGZIP(), []int{80, 1, 2} } type QueryRequest_OutputSerialization_CSVOutput struct { @@ -4969,7 +5145,7 @@ type QueryRequest_OutputSerialization_CSVOutput struct { func (x *QueryRequest_OutputSerialization_CSVOutput) Reset() { *x = QueryRequest_OutputSerialization_CSVOutput{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[88] + mi := &file_volume_server_proto_msgTypes[90] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4982,7 +5158,7 @@ func (x *QueryRequest_OutputSerialization_CSVOutput) String() string { func (*QueryRequest_OutputSerialization_CSVOutput) ProtoMessage() {} func (x *QueryRequest_OutputSerialization_CSVOutput) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[88] + mi := &file_volume_server_proto_msgTypes[90] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4995,7 +5171,7 @@ func (x *QueryRequest_OutputSerialization_CSVOutput) ProtoReflect() protoreflect // Deprecated: Use QueryRequest_OutputSerialization_CSVOutput.ProtoReflect.Descriptor instead. func (*QueryRequest_OutputSerialization_CSVOutput) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 2, 0} + return file_volume_server_proto_rawDescGZIP(), []int{80, 2, 0} } func (x *QueryRequest_OutputSerialization_CSVOutput) GetQuoteFields() string { @@ -5044,7 +5220,7 @@ type QueryRequest_OutputSerialization_JSONOutput struct { func (x *QueryRequest_OutputSerialization_JSONOutput) Reset() { *x = QueryRequest_OutputSerialization_JSONOutput{} if protoimpl.UnsafeEnabled { - mi := &file_volume_server_proto_msgTypes[89] + mi := &file_volume_server_proto_msgTypes[91] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5057,7 +5233,7 @@ func (x *QueryRequest_OutputSerialization_JSONOutput) String() string { func (*QueryRequest_OutputSerialization_JSONOutput) ProtoMessage() {} func (x *QueryRequest_OutputSerialization_JSONOutput) ProtoReflect() protoreflect.Message { - mi := &file_volume_server_proto_msgTypes[89] + mi := &file_volume_server_proto_msgTypes[91] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5070,7 +5246,7 @@ func (x *QueryRequest_OutputSerialization_JSONOutput) ProtoReflect() protoreflec // Deprecated: Use QueryRequest_OutputSerialization_JSONOutput.ProtoReflect.Descriptor instead. func (*QueryRequest_OutputSerialization_JSONOutput) Descriptor() ([]byte, []int) { - return file_volume_server_proto_rawDescGZIP(), []int{78, 2, 1} + return file_volume_server_proto_rawDescGZIP(), []int{80, 2, 1} } func (x *QueryRequest_OutputSerialization_JSONOutput) GetRecordDelimiter() string { @@ -5539,6 +5715,33 @@ var file_volume_server_proto_rawDesc = []byte{ 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x1b, 0x0a, 0x19, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8e, 0x03, 0x0a, 0x1a, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, + 0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, + 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x08, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, + 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6d, + 0x6f, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, + 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x73, + 0x33, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x73, 0x33, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12, + 0x22, 0x0a, 0x0d, 0x73, 0x33, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x33, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, + 0x4b, 0x65, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x33, 0x5f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x33, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, + 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x33, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, + 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x33, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x62, 0x75, 0x63, 0x6b, + 0x65, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, + 0x42, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, + 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x6d, 0x6f, + 0x74, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x1d, 0x0a, 0x1b, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, + 0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xf8, 0x0c, 0x0a, 0x0c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, @@ -5662,7 +5865,7 @@ var file_volume_server_proto_rawDesc = []byte{ 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x72, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x63, 0x72, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, - 0x74, 0x6c, 0x32, 0xa9, 0x21, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, + 0x74, 0x6c, 0x32, 0x9f, 0x22, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x0b, 0x42, 0x61, 0x74, 0x63, 0x68, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x24, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x44, 0x65, 0x6c, 0x65, 0x74, @@ -5916,24 +6119,31 @@ var file_volume_server_proto_rawDesc = []byte{ 0x65, 0x72, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4c, 0x65, - 0x61, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, - 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x1e, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, - 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, - 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x69, 0x65, - 0x64, 0x53, 0x74, 0x72, 0x69, 0x70, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x71, 0x0a, 0x12, 0x56, - 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x12, 0x2b, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, - 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, - 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, - 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, - 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x39, - 0x5a, 0x37, 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, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, - 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x61, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x74, 0x0a, + 0x13, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, 0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, + 0x65, 0x64, 0x6c, 0x65, 0x12, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, 0x64, + 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, + 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, 0x64, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x1e, 0x2e, 0x76, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, + 0x51, 0x75, 0x65, 0x72, 0x69, 0x65, 0x64, 0x53, 0x74, 0x72, 0x69, 0x70, 0x65, 0x22, 0x00, 0x30, + 0x01, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, + 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2b, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, + 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, + 0x65, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x42, 0x39, 0x5a, 0x37, 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, 0x76, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -5948,7 +6158,7 @@ func file_volume_server_proto_rawDescGZIP() []byte { return file_volume_server_proto_rawDescData } -var file_volume_server_proto_msgTypes = make([]protoimpl.MessageInfo, 90) +var file_volume_server_proto_msgTypes = make([]protoimpl.MessageInfo, 92) var file_volume_server_proto_goTypes = []interface{}{ (*BatchDeleteRequest)(nil), // 0: volume_server_pb.BatchDeleteRequest (*BatchDeleteResponse)(nil), // 1: volume_server_pb.BatchDeleteResponse @@ -6028,32 +6238,34 @@ var file_volume_server_proto_goTypes = []interface{}{ (*VolumeServerStatusResponse)(nil), // 75: volume_server_pb.VolumeServerStatusResponse (*VolumeServerLeaveRequest)(nil), // 76: volume_server_pb.VolumeServerLeaveRequest (*VolumeServerLeaveResponse)(nil), // 77: volume_server_pb.VolumeServerLeaveResponse - (*QueryRequest)(nil), // 78: volume_server_pb.QueryRequest - (*QueriedStripe)(nil), // 79: volume_server_pb.QueriedStripe - (*VolumeNeedleStatusRequest)(nil), // 80: volume_server_pb.VolumeNeedleStatusRequest - (*VolumeNeedleStatusResponse)(nil), // 81: volume_server_pb.VolumeNeedleStatusResponse - (*QueryRequest_Filter)(nil), // 82: volume_server_pb.QueryRequest.Filter - (*QueryRequest_InputSerialization)(nil), // 83: volume_server_pb.QueryRequest.InputSerialization - (*QueryRequest_OutputSerialization)(nil), // 84: volume_server_pb.QueryRequest.OutputSerialization - (*QueryRequest_InputSerialization_CSVInput)(nil), // 85: volume_server_pb.QueryRequest.InputSerialization.CSVInput - (*QueryRequest_InputSerialization_JSONInput)(nil), // 86: volume_server_pb.QueryRequest.InputSerialization.JSONInput - (*QueryRequest_InputSerialization_ParquetInput)(nil), // 87: volume_server_pb.QueryRequest.InputSerialization.ParquetInput - (*QueryRequest_OutputSerialization_CSVOutput)(nil), // 88: volume_server_pb.QueryRequest.OutputSerialization.CSVOutput - (*QueryRequest_OutputSerialization_JSONOutput)(nil), // 89: volume_server_pb.QueryRequest.OutputSerialization.JSONOutput + (*FetchAndWriteNeedleRequest)(nil), // 78: volume_server_pb.FetchAndWriteNeedleRequest + (*FetchAndWriteNeedleResponse)(nil), // 79: volume_server_pb.FetchAndWriteNeedleResponse + (*QueryRequest)(nil), // 80: volume_server_pb.QueryRequest + (*QueriedStripe)(nil), // 81: volume_server_pb.QueriedStripe + (*VolumeNeedleStatusRequest)(nil), // 82: volume_server_pb.VolumeNeedleStatusRequest + (*VolumeNeedleStatusResponse)(nil), // 83: volume_server_pb.VolumeNeedleStatusResponse + (*QueryRequest_Filter)(nil), // 84: volume_server_pb.QueryRequest.Filter + (*QueryRequest_InputSerialization)(nil), // 85: volume_server_pb.QueryRequest.InputSerialization + (*QueryRequest_OutputSerialization)(nil), // 86: volume_server_pb.QueryRequest.OutputSerialization + (*QueryRequest_InputSerialization_CSVInput)(nil), // 87: volume_server_pb.QueryRequest.InputSerialization.CSVInput + (*QueryRequest_InputSerialization_JSONInput)(nil), // 88: volume_server_pb.QueryRequest.InputSerialization.JSONInput + (*QueryRequest_InputSerialization_ParquetInput)(nil), // 89: volume_server_pb.QueryRequest.InputSerialization.ParquetInput + (*QueryRequest_OutputSerialization_CSVOutput)(nil), // 90: volume_server_pb.QueryRequest.OutputSerialization.CSVOutput + (*QueryRequest_OutputSerialization_JSONOutput)(nil), // 91: volume_server_pb.QueryRequest.OutputSerialization.JSONOutput } var file_volume_server_proto_depIdxs = []int32{ 2, // 0: volume_server_pb.BatchDeleteResponse.results:type_name -> volume_server_pb.DeleteResult 68, // 1: volume_server_pb.VolumeInfo.files:type_name -> volume_server_pb.RemoteFile 66, // 2: volume_server_pb.VolumeServerStatusResponse.disk_statuses:type_name -> volume_server_pb.DiskStatus 67, // 3: volume_server_pb.VolumeServerStatusResponse.memory_status:type_name -> volume_server_pb.MemStatus - 82, // 4: volume_server_pb.QueryRequest.filter:type_name -> volume_server_pb.QueryRequest.Filter - 83, // 5: volume_server_pb.QueryRequest.input_serialization:type_name -> volume_server_pb.QueryRequest.InputSerialization - 84, // 6: volume_server_pb.QueryRequest.output_serialization:type_name -> volume_server_pb.QueryRequest.OutputSerialization - 85, // 7: volume_server_pb.QueryRequest.InputSerialization.csv_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.CSVInput - 86, // 8: volume_server_pb.QueryRequest.InputSerialization.json_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.JSONInput - 87, // 9: volume_server_pb.QueryRequest.InputSerialization.parquet_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.ParquetInput - 88, // 10: volume_server_pb.QueryRequest.OutputSerialization.csv_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.CSVOutput - 89, // 11: volume_server_pb.QueryRequest.OutputSerialization.json_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.JSONOutput + 84, // 4: volume_server_pb.QueryRequest.filter:type_name -> volume_server_pb.QueryRequest.Filter + 85, // 5: volume_server_pb.QueryRequest.input_serialization:type_name -> volume_server_pb.QueryRequest.InputSerialization + 86, // 6: volume_server_pb.QueryRequest.output_serialization:type_name -> volume_server_pb.QueryRequest.OutputSerialization + 87, // 7: volume_server_pb.QueryRequest.InputSerialization.csv_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.CSVInput + 88, // 8: volume_server_pb.QueryRequest.InputSerialization.json_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.JSONInput + 89, // 9: volume_server_pb.QueryRequest.InputSerialization.parquet_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.ParquetInput + 90, // 10: volume_server_pb.QueryRequest.OutputSerialization.csv_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.CSVOutput + 91, // 11: volume_server_pb.QueryRequest.OutputSerialization.json_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.JSONOutput 0, // 12: volume_server_pb.VolumeServer.BatchDelete:input_type -> volume_server_pb.BatchDeleteRequest 4, // 13: volume_server_pb.VolumeServer.VacuumVolumeCheck:input_type -> volume_server_pb.VacuumVolumeCheckRequest 6, // 14: volume_server_pb.VolumeServer.VacuumVolumeCompact:input_type -> volume_server_pb.VacuumVolumeCompactRequest @@ -6090,48 +6302,50 @@ var file_volume_server_proto_depIdxs = []int32{ 72, // 45: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:input_type -> volume_server_pb.VolumeTierMoveDatFromRemoteRequest 74, // 46: volume_server_pb.VolumeServer.VolumeServerStatus:input_type -> volume_server_pb.VolumeServerStatusRequest 76, // 47: volume_server_pb.VolumeServer.VolumeServerLeave:input_type -> volume_server_pb.VolumeServerLeaveRequest - 78, // 48: volume_server_pb.VolumeServer.Query:input_type -> volume_server_pb.QueryRequest - 80, // 49: volume_server_pb.VolumeServer.VolumeNeedleStatus:input_type -> volume_server_pb.VolumeNeedleStatusRequest - 1, // 50: volume_server_pb.VolumeServer.BatchDelete:output_type -> volume_server_pb.BatchDeleteResponse - 5, // 51: volume_server_pb.VolumeServer.VacuumVolumeCheck:output_type -> volume_server_pb.VacuumVolumeCheckResponse - 7, // 52: volume_server_pb.VolumeServer.VacuumVolumeCompact:output_type -> volume_server_pb.VacuumVolumeCompactResponse - 9, // 53: volume_server_pb.VolumeServer.VacuumVolumeCommit:output_type -> volume_server_pb.VacuumVolumeCommitResponse - 11, // 54: volume_server_pb.VolumeServer.VacuumVolumeCleanup:output_type -> volume_server_pb.VacuumVolumeCleanupResponse - 13, // 55: volume_server_pb.VolumeServer.DeleteCollection:output_type -> volume_server_pb.DeleteCollectionResponse - 15, // 56: volume_server_pb.VolumeServer.AllocateVolume:output_type -> volume_server_pb.AllocateVolumeResponse - 17, // 57: volume_server_pb.VolumeServer.VolumeSyncStatus:output_type -> volume_server_pb.VolumeSyncStatusResponse - 19, // 58: volume_server_pb.VolumeServer.VolumeIncrementalCopy:output_type -> volume_server_pb.VolumeIncrementalCopyResponse - 21, // 59: volume_server_pb.VolumeServer.VolumeMount:output_type -> volume_server_pb.VolumeMountResponse - 23, // 60: volume_server_pb.VolumeServer.VolumeUnmount:output_type -> volume_server_pb.VolumeUnmountResponse - 25, // 61: volume_server_pb.VolumeServer.VolumeDelete:output_type -> volume_server_pb.VolumeDeleteResponse - 27, // 62: volume_server_pb.VolumeServer.VolumeMarkReadonly:output_type -> volume_server_pb.VolumeMarkReadonlyResponse - 29, // 63: volume_server_pb.VolumeServer.VolumeMarkWritable:output_type -> volume_server_pb.VolumeMarkWritableResponse - 31, // 64: volume_server_pb.VolumeServer.VolumeConfigure:output_type -> volume_server_pb.VolumeConfigureResponse - 33, // 65: volume_server_pb.VolumeServer.VolumeStatus:output_type -> volume_server_pb.VolumeStatusResponse - 35, // 66: volume_server_pb.VolumeServer.VolumeCopy:output_type -> volume_server_pb.VolumeCopyResponse - 65, // 67: volume_server_pb.VolumeServer.ReadVolumeFileStatus:output_type -> volume_server_pb.ReadVolumeFileStatusResponse - 37, // 68: volume_server_pb.VolumeServer.CopyFile:output_type -> volume_server_pb.CopyFileResponse - 39, // 69: volume_server_pb.VolumeServer.ReadNeedleBlob:output_type -> volume_server_pb.ReadNeedleBlobResponse - 41, // 70: volume_server_pb.VolumeServer.WriteNeedleBlob:output_type -> volume_server_pb.WriteNeedleBlobResponse - 43, // 71: volume_server_pb.VolumeServer.VolumeTailSender:output_type -> volume_server_pb.VolumeTailSenderResponse - 45, // 72: volume_server_pb.VolumeServer.VolumeTailReceiver:output_type -> volume_server_pb.VolumeTailReceiverResponse - 47, // 73: volume_server_pb.VolumeServer.VolumeEcShardsGenerate:output_type -> volume_server_pb.VolumeEcShardsGenerateResponse - 49, // 74: volume_server_pb.VolumeServer.VolumeEcShardsRebuild:output_type -> volume_server_pb.VolumeEcShardsRebuildResponse - 51, // 75: volume_server_pb.VolumeServer.VolumeEcShardsCopy:output_type -> volume_server_pb.VolumeEcShardsCopyResponse - 53, // 76: volume_server_pb.VolumeServer.VolumeEcShardsDelete:output_type -> volume_server_pb.VolumeEcShardsDeleteResponse - 55, // 77: volume_server_pb.VolumeServer.VolumeEcShardsMount:output_type -> volume_server_pb.VolumeEcShardsMountResponse - 57, // 78: volume_server_pb.VolumeServer.VolumeEcShardsUnmount:output_type -> volume_server_pb.VolumeEcShardsUnmountResponse - 59, // 79: volume_server_pb.VolumeServer.VolumeEcShardRead:output_type -> volume_server_pb.VolumeEcShardReadResponse - 61, // 80: volume_server_pb.VolumeServer.VolumeEcBlobDelete:output_type -> volume_server_pb.VolumeEcBlobDeleteResponse - 63, // 81: volume_server_pb.VolumeServer.VolumeEcShardsToVolume:output_type -> volume_server_pb.VolumeEcShardsToVolumeResponse - 71, // 82: volume_server_pb.VolumeServer.VolumeTierMoveDatToRemote:output_type -> volume_server_pb.VolumeTierMoveDatToRemoteResponse - 73, // 83: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:output_type -> volume_server_pb.VolumeTierMoveDatFromRemoteResponse - 75, // 84: volume_server_pb.VolumeServer.VolumeServerStatus:output_type -> volume_server_pb.VolumeServerStatusResponse - 77, // 85: volume_server_pb.VolumeServer.VolumeServerLeave:output_type -> volume_server_pb.VolumeServerLeaveResponse - 79, // 86: volume_server_pb.VolumeServer.Query:output_type -> volume_server_pb.QueriedStripe - 81, // 87: volume_server_pb.VolumeServer.VolumeNeedleStatus:output_type -> volume_server_pb.VolumeNeedleStatusResponse - 50, // [50:88] is the sub-list for method output_type - 12, // [12:50] is the sub-list for method input_type + 78, // 48: volume_server_pb.VolumeServer.FetchAndWriteNeedle:input_type -> volume_server_pb.FetchAndWriteNeedleRequest + 80, // 49: volume_server_pb.VolumeServer.Query:input_type -> volume_server_pb.QueryRequest + 82, // 50: volume_server_pb.VolumeServer.VolumeNeedleStatus:input_type -> volume_server_pb.VolumeNeedleStatusRequest + 1, // 51: volume_server_pb.VolumeServer.BatchDelete:output_type -> volume_server_pb.BatchDeleteResponse + 5, // 52: volume_server_pb.VolumeServer.VacuumVolumeCheck:output_type -> volume_server_pb.VacuumVolumeCheckResponse + 7, // 53: volume_server_pb.VolumeServer.VacuumVolumeCompact:output_type -> volume_server_pb.VacuumVolumeCompactResponse + 9, // 54: volume_server_pb.VolumeServer.VacuumVolumeCommit:output_type -> volume_server_pb.VacuumVolumeCommitResponse + 11, // 55: volume_server_pb.VolumeServer.VacuumVolumeCleanup:output_type -> volume_server_pb.VacuumVolumeCleanupResponse + 13, // 56: volume_server_pb.VolumeServer.DeleteCollection:output_type -> volume_server_pb.DeleteCollectionResponse + 15, // 57: volume_server_pb.VolumeServer.AllocateVolume:output_type -> volume_server_pb.AllocateVolumeResponse + 17, // 58: volume_server_pb.VolumeServer.VolumeSyncStatus:output_type -> volume_server_pb.VolumeSyncStatusResponse + 19, // 59: volume_server_pb.VolumeServer.VolumeIncrementalCopy:output_type -> volume_server_pb.VolumeIncrementalCopyResponse + 21, // 60: volume_server_pb.VolumeServer.VolumeMount:output_type -> volume_server_pb.VolumeMountResponse + 23, // 61: volume_server_pb.VolumeServer.VolumeUnmount:output_type -> volume_server_pb.VolumeUnmountResponse + 25, // 62: volume_server_pb.VolumeServer.VolumeDelete:output_type -> volume_server_pb.VolumeDeleteResponse + 27, // 63: volume_server_pb.VolumeServer.VolumeMarkReadonly:output_type -> volume_server_pb.VolumeMarkReadonlyResponse + 29, // 64: volume_server_pb.VolumeServer.VolumeMarkWritable:output_type -> volume_server_pb.VolumeMarkWritableResponse + 31, // 65: volume_server_pb.VolumeServer.VolumeConfigure:output_type -> volume_server_pb.VolumeConfigureResponse + 33, // 66: volume_server_pb.VolumeServer.VolumeStatus:output_type -> volume_server_pb.VolumeStatusResponse + 35, // 67: volume_server_pb.VolumeServer.VolumeCopy:output_type -> volume_server_pb.VolumeCopyResponse + 65, // 68: volume_server_pb.VolumeServer.ReadVolumeFileStatus:output_type -> volume_server_pb.ReadVolumeFileStatusResponse + 37, // 69: volume_server_pb.VolumeServer.CopyFile:output_type -> volume_server_pb.CopyFileResponse + 39, // 70: volume_server_pb.VolumeServer.ReadNeedleBlob:output_type -> volume_server_pb.ReadNeedleBlobResponse + 41, // 71: volume_server_pb.VolumeServer.WriteNeedleBlob:output_type -> volume_server_pb.WriteNeedleBlobResponse + 43, // 72: volume_server_pb.VolumeServer.VolumeTailSender:output_type -> volume_server_pb.VolumeTailSenderResponse + 45, // 73: volume_server_pb.VolumeServer.VolumeTailReceiver:output_type -> volume_server_pb.VolumeTailReceiverResponse + 47, // 74: volume_server_pb.VolumeServer.VolumeEcShardsGenerate:output_type -> volume_server_pb.VolumeEcShardsGenerateResponse + 49, // 75: volume_server_pb.VolumeServer.VolumeEcShardsRebuild:output_type -> volume_server_pb.VolumeEcShardsRebuildResponse + 51, // 76: volume_server_pb.VolumeServer.VolumeEcShardsCopy:output_type -> volume_server_pb.VolumeEcShardsCopyResponse + 53, // 77: volume_server_pb.VolumeServer.VolumeEcShardsDelete:output_type -> volume_server_pb.VolumeEcShardsDeleteResponse + 55, // 78: volume_server_pb.VolumeServer.VolumeEcShardsMount:output_type -> volume_server_pb.VolumeEcShardsMountResponse + 57, // 79: volume_server_pb.VolumeServer.VolumeEcShardsUnmount:output_type -> volume_server_pb.VolumeEcShardsUnmountResponse + 59, // 80: volume_server_pb.VolumeServer.VolumeEcShardRead:output_type -> volume_server_pb.VolumeEcShardReadResponse + 61, // 81: volume_server_pb.VolumeServer.VolumeEcBlobDelete:output_type -> volume_server_pb.VolumeEcBlobDeleteResponse + 63, // 82: volume_server_pb.VolumeServer.VolumeEcShardsToVolume:output_type -> volume_server_pb.VolumeEcShardsToVolumeResponse + 71, // 83: volume_server_pb.VolumeServer.VolumeTierMoveDatToRemote:output_type -> volume_server_pb.VolumeTierMoveDatToRemoteResponse + 73, // 84: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:output_type -> volume_server_pb.VolumeTierMoveDatFromRemoteResponse + 75, // 85: volume_server_pb.VolumeServer.VolumeServerStatus:output_type -> volume_server_pb.VolumeServerStatusResponse + 77, // 86: volume_server_pb.VolumeServer.VolumeServerLeave:output_type -> volume_server_pb.VolumeServerLeaveResponse + 79, // 87: volume_server_pb.VolumeServer.FetchAndWriteNeedle:output_type -> volume_server_pb.FetchAndWriteNeedleResponse + 81, // 88: volume_server_pb.VolumeServer.Query:output_type -> volume_server_pb.QueriedStripe + 83, // 89: volume_server_pb.VolumeServer.VolumeNeedleStatus:output_type -> volume_server_pb.VolumeNeedleStatusResponse + 51, // [51:90] is the sub-list for method output_type + 12, // [12:51] is the sub-list for method input_type 12, // [12:12] is the sub-list for extension type_name 12, // [12:12] is the sub-list for extension extendee 0, // [0:12] is the sub-list for field type_name @@ -7080,7 +7294,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest); i { + switch v := v.(*FetchAndWriteNeedleRequest); i { case 0: return &v.state case 1: @@ -7092,7 +7306,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueriedStripe); i { + switch v := v.(*FetchAndWriteNeedleResponse); i { case 0: return &v.state case 1: @@ -7104,7 +7318,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VolumeNeedleStatusRequest); i { + switch v := v.(*QueryRequest); i { case 0: return &v.state case 1: @@ -7116,7 +7330,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VolumeNeedleStatusResponse); i { + switch v := v.(*QueriedStripe); i { case 0: return &v.state case 1: @@ -7128,7 +7342,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest_Filter); i { + switch v := v.(*VolumeNeedleStatusRequest); i { case 0: return &v.state case 1: @@ -7140,7 +7354,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest_InputSerialization); i { + switch v := v.(*VolumeNeedleStatusResponse); i { case 0: return &v.state case 1: @@ -7152,7 +7366,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest_OutputSerialization); i { + switch v := v.(*QueryRequest_Filter); i { case 0: return &v.state case 1: @@ -7164,7 +7378,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest_InputSerialization_CSVInput); i { + switch v := v.(*QueryRequest_InputSerialization); i { case 0: return &v.state case 1: @@ -7176,7 +7390,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest_InputSerialization_JSONInput); i { + switch v := v.(*QueryRequest_OutputSerialization); i { case 0: return &v.state case 1: @@ -7188,7 +7402,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest_InputSerialization_ParquetInput); i { + switch v := v.(*QueryRequest_InputSerialization_CSVInput); i { case 0: return &v.state case 1: @@ -7200,7 +7414,7 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QueryRequest_OutputSerialization_CSVOutput); i { + switch v := v.(*QueryRequest_InputSerialization_JSONInput); i { case 0: return &v.state case 1: @@ -7212,6 +7426,30 @@ func file_volume_server_proto_init() { } } file_volume_server_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryRequest_InputSerialization_ParquetInput); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_volume_server_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryRequest_OutputSerialization_CSVOutput); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_volume_server_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*QueryRequest_OutputSerialization_JSONOutput); i { case 0: return &v.state @@ -7230,7 +7468,7 @@ func file_volume_server_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_volume_server_proto_rawDesc, NumEnums: 0, - NumMessages: 90, + NumMessages: 92, NumExtensions: 0, NumServices: 1, }, @@ -7296,6 +7534,8 @@ type VolumeServerClient interface { 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) // query Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error) VolumeNeedleStatus(ctx context.Context, in *VolumeNeedleStatusRequest, opts ...grpc.CallOption) (*VolumeNeedleStatusResponse, error) @@ -7771,6 +8011,15 @@ func (c *volumeServerClient) VolumeServerLeave(ctx context.Context, in *VolumeSe 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[6], "/volume_server_pb.VolumeServer/Query", opts...) if err != nil { @@ -7854,6 +8103,8 @@ type VolumeServerServer interface { 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) // query Query(*QueryRequest, VolumeServer_QueryServer) error VolumeNeedleStatus(context.Context, *VolumeNeedleStatusRequest) (*VolumeNeedleStatusResponse, error) @@ -7971,6 +8222,9 @@ func (*UnimplementedVolumeServerServer) VolumeServerStatus(context.Context, *Vol 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") } @@ -8648,6 +8902,24 @@ func _VolumeServer_VolumeServerLeave_Handler(srv interface{}, ctx context.Contex 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 { @@ -8811,6 +9083,10 @@ var _VolumeServer_serviceDesc = grpc.ServiceDesc{ MethodName: "VolumeServerLeave", Handler: _VolumeServer_VolumeServerLeave_Handler, }, + { + MethodName: "FetchAndWriteNeedle", + Handler: _VolumeServer_FetchAndWriteNeedle_Handler, + }, { MethodName: "VolumeNeedleStatus", Handler: _VolumeServer_VolumeNeedleStatus_Handler, diff --git a/weed/remote_storage/remote_storage.go b/weed/remote_storage/remote_storage.go new file mode 100644 index 000000000..c94260ac0 --- /dev/null +++ b/weed/remote_storage/remote_storage.go @@ -0,0 +1,75 @@ +package remote_storage + +import ( + "fmt" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "io" + "strings" + "sync" +) + +func ParseLocation(remote string) (loc *filer_pb.RemoteStorageLocation) { + loc = &filer_pb.RemoteStorageLocation{} + if strings.HasSuffix(string(remote), "/") { + remote = remote[:len(remote)-1] + } + parts := strings.SplitN(string(remote), "/", 3) + if len(parts) >= 1 { + loc.Name = parts[0] + } + if len(parts) >= 2 { + loc.Bucket = parts[1] + } + loc.Path = string(remote[len(loc.Name)+1+len(loc.Bucket):]) + if loc.Path == "" { + loc.Path = "/" + } + return +} + +type VisitFunc func(dir string, name string, isDirectory bool, remoteEntry *filer_pb.RemoteEntry) error + +type RemoteStorageClient interface { + Traverse(loc *filer_pb.RemoteStorageLocation, visitFn VisitFunc) error + ReadFile(loc *filer_pb.RemoteStorageLocation, offset int64, size int64) (data []byte, err error) + WriteFile(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry, reader io.Reader) (remoteEntry *filer_pb.RemoteEntry, err error) + UpdateFileMetadata(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry) (err error) + DeleteFile(loc *filer_pb.RemoteStorageLocation) (err error) +} + +type RemoteStorageClientMaker interface { + Make(remoteConf *filer_pb.RemoteConf) (RemoteStorageClient, error) +} + +var ( + RemoteStorageClientMakers = make(map[string]RemoteStorageClientMaker) + remoteStorageClients = make(map[string]RemoteStorageClient) + remoteStorageClientsLock sync.Mutex +) + +func makeRemoteStorageClient(remoteConf *filer_pb.RemoteConf) (RemoteStorageClient, error) { + maker, found := RemoteStorageClientMakers[remoteConf.Type] + if !found { + return nil, fmt.Errorf("remote storage type %s not found", remoteConf.Type) + } + return maker.Make(remoteConf) +} + +func GetRemoteStorage(remoteConf *filer_pb.RemoteConf) (RemoteStorageClient, error) { + remoteStorageClientsLock.Lock() + defer remoteStorageClientsLock.Unlock() + + existingRemoteStorageClient, found := remoteStorageClients[remoteConf.Name] + if found { + return existingRemoteStorageClient, nil + } + + newRemoteStorageClient, err := makeRemoteStorageClient(remoteConf) + if err != nil { + return nil, fmt.Errorf("make remote storage client %s: %v", remoteConf.Name, err) + } + + remoteStorageClients[remoteConf.Name] = newRemoteStorageClient + + return newRemoteStorageClient, nil +} diff --git a/weed/remote_storage/s3/s3_storage_client.go b/weed/remote_storage/s3/s3_storage_client.go new file mode 100644 index 000000000..5be64406e --- /dev/null +++ b/weed/remote_storage/s3/s3_storage_client.go @@ -0,0 +1,215 @@ +package s3 + +import ( + "fmt" + "github.com/aws/aws-sdk-go/aws" + "github.com/aws/aws-sdk-go/aws/credentials" + "github.com/aws/aws-sdk-go/aws/session" + "github.com/aws/aws-sdk-go/service/s3" + "github.com/aws/aws-sdk-go/service/s3/s3iface" + "github.com/aws/aws-sdk-go/service/s3/s3manager" + "github.com/chrislusf/seaweedfs/weed/filer" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "github.com/chrislusf/seaweedfs/weed/remote_storage" + "github.com/chrislusf/seaweedfs/weed/util" + "io" +) + +func init() { + remote_storage.RemoteStorageClientMakers["s3"] = new(s3RemoteStorageMaker) +} + +type s3RemoteStorageMaker struct{} + +func (s s3RemoteStorageMaker) Make(conf *filer_pb.RemoteConf) (remote_storage.RemoteStorageClient, error) { + client := &s3RemoteStorageClient{ + conf: conf, + } + config := &aws.Config{ + Region: aws.String(conf.S3Region), + Endpoint: aws.String(conf.S3Endpoint), + S3ForcePathStyle: aws.Bool(true), + } + if conf.S3AccessKey != "" && conf.S3SecretKey != "" { + config.Credentials = credentials.NewStaticCredentials(conf.S3AccessKey, conf.S3SecretKey, "") + } + + sess, err := session.NewSession(config) + if err != nil { + return nil, fmt.Errorf("create aws session: %v", err) + } + client.conn = s3.New(sess) + return client, nil +} + +type s3RemoteStorageClient struct { + conf *filer_pb.RemoteConf + conn s3iface.S3API +} + +var _ = remote_storage.RemoteStorageClient(&s3RemoteStorageClient{}) + +func (s *s3RemoteStorageClient) Traverse(remote *filer_pb.RemoteStorageLocation, visitFn remote_storage.VisitFunc) (err error) { + + pathKey := remote.Path[1:] + + listInput := &s3.ListObjectsV2Input{ + Bucket: aws.String(remote.Bucket), + ContinuationToken: nil, + Delimiter: nil, // not aws.String("/"), iterate through all entries + EncodingType: nil, + ExpectedBucketOwner: nil, + FetchOwner: nil, + MaxKeys: nil, // aws.Int64(1000), + Prefix: aws.String(pathKey), + RequestPayer: nil, + StartAfter: nil, + } + isLastPage := false + for !isLastPage && err == nil { + listErr := s.conn.ListObjectsV2Pages(listInput, func(page *s3.ListObjectsV2Output, lastPage bool) bool { + for _, content := range page.Contents { + key := *content.Key + if len(pathKey) == 0 { + key = "/" + key + } else { + key = key[len(pathKey):] + } + dir, name := util.FullPath(key).DirAndName() + if err := visitFn(dir, name, false, &filer_pb.RemoteEntry{ + LastModifiedAt: (*content.LastModified).Unix(), + Size: *content.Size, + ETag: *content.ETag, + StorageName: s.conf.Name, + }); err != nil { + return false + } + } + listInput.ContinuationToken = page.NextContinuationToken + isLastPage = lastPage + return true + }) + if listErr != nil { + err = fmt.Errorf("list %v: %v", remote, listErr) + } + } + return +} +func (s *s3RemoteStorageClient) ReadFile(loc *filer_pb.RemoteStorageLocation, offset int64, size int64) (data []byte, err error) { + downloader := s3manager.NewDownloaderWithClient(s.conn, func(u *s3manager.Downloader) { + u.PartSize = int64(4 * 1024 * 1024) + u.Concurrency = 1 + }) + + dataSlice := make([]byte, int(size)) + writerAt := aws.NewWriteAtBuffer(dataSlice) + + _, err = downloader.Download(writerAt, &s3.GetObjectInput{ + Bucket: aws.String(loc.Bucket), + Key: aws.String(loc.Path[1:]), + Range: aws.String(fmt.Sprintf("bytes=%d-%d", offset, offset+size-1)), + }) + if err != nil { + return nil, fmt.Errorf("failed to download file %s%s: %v", loc.Bucket, loc.Path, err) + } + + return writerAt.Bytes(), nil +} + +func (s *s3RemoteStorageClient) WriteFile(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry, reader io.Reader) (remoteEntry *filer_pb.RemoteEntry, err error) { + + fileSize := int64(filer.FileSize(entry)) + + partSize := int64(8 * 1024 * 1024) // The minimum/default allowed part size is 5MB + for partSize*1000 < fileSize { + partSize *= 4 + } + + // Create an uploader with the session and custom options + uploader := s3manager.NewUploaderWithClient(s.conn, func(u *s3manager.Uploader) { + u.PartSize = partSize + u.Concurrency = 5 + }) + + // process tagging + tags := "" + for k, v := range entry.Extended { + if len(tags) > 0 { + tags = tags + "&" + } + tags = tags + k + "=" + string(v) + } + + // Upload the file to S3. + _, err = uploader.Upload(&s3manager.UploadInput{ + Bucket: aws.String(loc.Bucket), + Key: aws.String(loc.Path[1:]), + Body: reader, + ACL: aws.String("private"), + ServerSideEncryption: aws.String("AES256"), + StorageClass: aws.String("STANDARD_IA"), + Tagging: aws.String(tags), + }) + + //in case it fails to upload + if err != nil { + return nil, fmt.Errorf("upload to s3 %s/%s%s: %v", loc.Name, loc.Bucket, loc.Path, err) + } + + // read back the remote entry + return s.readFileRemoteEntry(loc) + +} + +func toTagging(attributes map[string][]byte) *s3.Tagging { + tagging := &s3.Tagging{} + for k, v := range attributes { + tagging.TagSet = append(tagging.TagSet, &s3.Tag{ + Key: aws.String(k), + Value: aws.String(string(v)), + }) + } + return tagging +} + +func (s *s3RemoteStorageClient) readFileRemoteEntry(loc *filer_pb.RemoteStorageLocation) (*filer_pb.RemoteEntry, error) { + resp, err := s.conn.HeadObject(&s3.HeadObjectInput{ + Bucket: aws.String(loc.Bucket), + Key: aws.String(loc.Path[1:]), + }) + if err != nil { + return nil, err + } + + return &filer_pb.RemoteEntry{ + LastModifiedAt: resp.LastModified.Unix(), + Size: *resp.ContentLength, + ETag: *resp.ETag, + StorageName: s.conf.Name, + }, nil + +} + +func (s *s3RemoteStorageClient) UpdateFileMetadata(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry) (err error) { + tagging := toTagging(entry.Extended) + if len(tagging.TagSet) > 0 { + _, err = s.conn.PutObjectTagging(&s3.PutObjectTaggingInput{ + Bucket: aws.String(loc.Bucket), + Key: aws.String(loc.Path[1:]), + Tagging: toTagging(entry.Extended), + }) + } else { + _, err = s.conn.DeleteObjectTagging(&s3.DeleteObjectTaggingInput{ + Bucket: aws.String(loc.Bucket), + Key: aws.String(loc.Path[1:]), + }) + } + return +} +func (s *s3RemoteStorageClient) DeleteFile(loc *filer_pb.RemoteStorageLocation) (err error) { + _, err = s.conn.DeleteObject(&s3.DeleteObjectInput{ + Bucket: aws.String(loc.Bucket), + Key: aws.String(loc.Path[1:]), + }) + return +} diff --git a/weed/replication/sink/filersink/filer_sink.go b/weed/replication/sink/filersink/filer_sink.go index 2b526cc52..d42ca63a8 100644 --- a/weed/replication/sink/filersink/filer_sink.go +++ b/weed/replication/sink/filersink/filer_sink.go @@ -134,7 +134,7 @@ func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry, signatures [ Attributes: entry.Attributes, Chunks: replicatedChunks, Content: entry.Content, - Remote: entry.Remote, + RemoteEntry: entry.RemoteEntry, }, IsFromOtherCluster: true, Signatures: signatures, diff --git a/weed/s3api/auth_credentials_subscribe.go b/weed/s3api/auth_credentials_subscribe.go index ea4b69550..05cce632a 100644 --- a/weed/s3api/auth_credentials_subscribe.go +++ b/weed/s3api/auth_credentials_subscribe.go @@ -1,13 +1,11 @@ package s3api import ( - "context" - "fmt" "github.com/chrislusf/seaweedfs/weed/filer" "github.com/chrislusf/seaweedfs/weed/glog" + "github.com/chrislusf/seaweedfs/weed/pb" "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" - "io" - "time" + "github.com/chrislusf/seaweedfs/weed/util" ) func (s3a *S3ApiServer) subscribeMetaEvents(clientName string, prefix string, lastTsNs int64) error { @@ -34,37 +32,8 @@ func (s3a *S3ApiServer) subscribeMetaEvents(clientName string, prefix string, la return nil } - for { - err := s3a.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ - ClientName: clientName, - PathPrefix: prefix, - SinceNs: lastTsNs, - }) - if err != nil { - return fmt.Errorf("subscribe: %v", err) - } + return util.Retry("followIamChanges", func() error { + return pb.WithFilerClientFollowMetadata(s3a, clientName, prefix, lastTsNs, 0, processEventFn, true) + }) - for { - resp, listenErr := stream.Recv() - if listenErr == io.EOF { - return nil - } - if listenErr != nil { - return listenErr - } - - if err := processEventFn(resp); err != nil { - glog.Fatalf("process %v: %v", resp, err) - } - lastTsNs = resp.TsNs - } - }) - if err != nil { - glog.Errorf("subscribing filer meta change: %v", err) - } - time.Sleep(time.Second) - } } diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go index d7afaa65a..534bc4840 100644 --- a/weed/server/filer_server.go +++ b/weed/server/filer_server.go @@ -149,6 +149,8 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption) fs.filer.LoadFilerConf() + fs.filer.LoadRemoteStorageConfAndMapping() + grace.OnInterrupt(func() { fs.filer.Shutdown() }) diff --git a/weed/server/filer_server_handlers_read.go b/weed/server/filer_server_handlers_read.go index 957e08855..86454f8c5 100644 --- a/weed/server/filer_server_handlers_read.go +++ b/weed/server/filer_server_handlers_read.go @@ -101,7 +101,7 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request) //Seaweed custom header are not visible to Vue or javascript seaweedHeaders := []string{} - for header, _ := range w.Header() { + for header := range w.Header() { if strings.HasPrefix(header, "Seaweed-") { seaweedHeaders = append(seaweedHeaders, header) } @@ -163,9 +163,18 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request) } return err } - err = filer.StreamContent(fs.filer.MasterClient, writer, entry.Chunks, offset, size) - if err != nil { - glog.Errorf("failed to stream content %s: %v", r.URL, err) + if entry.IsInRemoteOnly() { + var data []byte + data, err = fs.filer.ReadRemote(entry, offset, size) + if err != nil { + glog.Errorf("failed to read remote %s: %v", r.URL, err) + } + _, err = w.Write(data) + } else { + err = filer.StreamContent(fs.filer.MasterClient, writer, entry.Chunks, offset, size) + if err != nil { + glog.Errorf("failed to stream content %s: %v", r.URL, err) + } } return err }) diff --git a/weed/server/master_server.go b/weed/server/master_server.go index 9d222a342..a23ad0698 100644 --- a/weed/server/master_server.go +++ b/weed/server/master_server.go @@ -228,6 +228,7 @@ func (ms *MasterServer) startAdminScripts() { shellOptions.Masters = &masterAddress shellOptions.FilerHost, shellOptions.FilerPort, err = util.ParseHostPort(filerHostPort) + shellOptions.FilerAddress = filerHostPort shellOptions.Directory = "/" if err != nil { glog.V(0).Infof("failed to parse master.filer.default = %s : %v\n", filerHostPort, err) diff --git a/weed/server/volume_grpc_remote.go b/weed/server/volume_grpc_remote.go new file mode 100644 index 000000000..0b6cd465b --- /dev/null +++ b/weed/server/volume_grpc_remote.go @@ -0,0 +1,49 @@ +package weed_server + +import ( + "context" + "fmt" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb" + "github.com/chrislusf/seaweedfs/weed/remote_storage" + "github.com/chrislusf/seaweedfs/weed/storage/needle" + "github.com/chrislusf/seaweedfs/weed/storage/types" +) + +func (vs *VolumeServer) FetchAndWriteNeedle(ctx context.Context, req *volume_server_pb.FetchAndWriteNeedleRequest) (resp *volume_server_pb.FetchAndWriteNeedleResponse, err error) { + resp = &volume_server_pb.FetchAndWriteNeedleResponse{} + v := vs.store.GetVolume(needle.VolumeId(req.VolumeId)) + if v == nil { + return nil, fmt.Errorf("not found volume id %d", req.VolumeId) + } + + remoteConf := &filer_pb.RemoteConf{ + Type: req.RemoteType, + Name: req.RemoteName, + S3AccessKey: req.S3AccessKey, + S3SecretKey: req.S3SecretKey, + S3Region: req.S3Region, + S3Endpoint: req.S3Endpoint, + } + + client, getClientErr := remote_storage.GetRemoteStorage(remoteConf) + if getClientErr != nil { + return nil, fmt.Errorf("get remote client: %v", getClientErr) + } + + remoteStorageLocation := &filer_pb.RemoteStorageLocation{ + Name: req.RemoteName, + Bucket: req.RemoteBucket, + Path: req.RemoteKey, + } + data, ReadRemoteErr := client.ReadFile(remoteStorageLocation, req.Offset, req.Size) + if ReadRemoteErr != nil { + return nil, fmt.Errorf("read from remote %+v: %v", remoteStorageLocation, ReadRemoteErr) + } + + if err = v.WriteNeedleBlob(types.NeedleId(req.NeedleId), data, types.Size(req.Size)); err != nil { + return nil, fmt.Errorf("write blob needle %d size %d: %v", req.NeedleId, req.Size, err) + } + + return resp, nil +} diff --git a/weed/shell/command_remote_configure.go b/weed/shell/command_remote_configure.go index 567143ce1..7a9ad1f65 100644 --- a/weed/shell/command_remote_configure.go +++ b/weed/shell/command_remote_configure.go @@ -7,9 +7,11 @@ import ( "github.com/chrislusf/seaweedfs/weed/filer" "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" "github.com/chrislusf/seaweedfs/weed/util" + "github.com/golang/protobuf/jsonpb" "github.com/golang/protobuf/proto" "io" "regexp" + "strings" ) func init() { @@ -84,6 +86,9 @@ func (c *commandRemoteConfigure) listExistingRemoteStorages(commandEnv *CommandE fmt.Fprintf(writer, "skipping %s\n", entry.Name) return nil } + if !strings.HasSuffix(entry.Name, filer.REMOTE_STORAGE_CONF_SUFFIX) { + return nil + } conf := &filer_pb.RemoteConf{} if err := proto.Unmarshal(entry.Content, conf); err != nil { @@ -92,9 +97,15 @@ func (c *commandRemoteConfigure) listExistingRemoteStorages(commandEnv *CommandE conf.S3SecretKey = "" - fmt.Fprintf(writer, "%+v\n", conf) + m := jsonpb.Marshaler{ + EmitDefaults: false, + Indent: " ", + } - return nil + err := m.Marshal(writer, conf) + fmt.Fprintln(writer) + + return err }) } @@ -105,7 +116,7 @@ func (c *commandRemoteConfigure) deleteRemoteStorage(commandEnv *CommandEnv, wri request := &filer_pb.DeleteEntryRequest{ Directory: filer.DirectoryEtcRemote, - Name: storageName, + Name: storageName + filer.REMOTE_STORAGE_CONF_SUFFIX, IgnoreRecursiveError: false, IsDeleteData: true, IsRecursive: true, @@ -132,7 +143,7 @@ func (c *commandRemoteConfigure) saveRemoteStorage(commandEnv *CommandEnv, write } if err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { - return filer.SaveInsideFiler(client, filer.DirectoryEtcRemote, conf.Name, data) + return filer.SaveInsideFiler(client, filer.DirectoryEtcRemote, conf.Name+filer.REMOTE_STORAGE_CONF_SUFFIX, data) }); err != nil && err != filer_pb.ErrNotFound { return err } diff --git a/weed/shell/command_remote_mount.go b/weed/shell/command_remote_mount.go new file mode 100644 index 000000000..37b235a55 --- /dev/null +++ b/weed/shell/command_remote_mount.go @@ -0,0 +1,232 @@ +package shell + +import ( + "context" + "flag" + "fmt" + "github.com/chrislusf/seaweedfs/weed/filer" + "github.com/chrislusf/seaweedfs/weed/pb/filer_pb" + "github.com/chrislusf/seaweedfs/weed/remote_storage" + "github.com/chrislusf/seaweedfs/weed/util" + "github.com/golang/protobuf/jsonpb" + "io" +) + +func init() { + Commands = append(Commands, &commandRemoteMount{}) +} + +type commandRemoteMount struct { +} + +func (c *commandRemoteMount) Name() string { + return "remote.mount" +} + +func (c *commandRemoteMount) Help() string { + return `mount remote storage and pull its metadata + + # assume a remote storage is configured to name "s3_1" + remote.configure -name=s3_1 -type=s3 -access_key=xxx -secret_key=yyy + + # mount and pull one bucket + remote.mount -dir=xxx -remote=s3_1/bucket + # mount and pull one directory in the bucket + remote.mount -dir=xxx -remote=s3_1/bucket/dir1 + +` +} + +func (c *commandRemoteMount) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) { + + remoteMountCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError) + + dir := remoteMountCommand.String("dir", "", "a directory in filer") + nonEmpty := remoteMountCommand.Bool("nonempty", false, "allows the mounting over a non-empty directory") + remote := remoteMountCommand.String("remote", "", "a directory in remote storage, ex. //path/to/dir") + + if err = remoteMountCommand.Parse(args); err != nil { + return nil + } + + if *dir == "" { + return c.listExistingRemoteStorageMounts(commandEnv, writer) + } + + remoteStorageLocation := remote_storage.ParseLocation(*remote) + + // find configuration for remote storage + // remotePath is //path/to/dir + remoteConf, err := c.findRemoteStorageConfiguration(commandEnv, writer, remoteStorageLocation) + if err != nil { + return fmt.Errorf("find configuration for %s: %v", *remote, err) + } + + // pull metadata from remote + if err = c.pullMetadata(commandEnv, writer, *dir, *nonEmpty, remoteConf, remoteStorageLocation); err != nil { + return fmt.Errorf("pull metadata: %v", err) + } + + // store a mount configuration in filer + if err = c.saveMountMapping(commandEnv, writer, *dir, remoteStorageLocation); err != nil { + return fmt.Errorf("save mount mapping: %v", err) + } + + return nil +} + +func (c *commandRemoteMount) listExistingRemoteStorageMounts(commandEnv *CommandEnv, writer io.Writer) (err error) { + + // read current mapping + mappings, readErr := filer.ReadMountMappings(commandEnv.option.GrpcDialOption, commandEnv.option.FilerAddress) + if readErr != nil { + return readErr + } + + m := jsonpb.Marshaler{ + EmitDefaults: false, + Indent: " ", + } + + err = m.Marshal(writer, mappings) + fmt.Fprintln(writer) + + return + +} + +func (c *commandRemoteMount) findRemoteStorageConfiguration(commandEnv *CommandEnv, writer io.Writer, remote *filer_pb.RemoteStorageLocation) (conf *filer_pb.RemoteConf, err error) { + + return filer.ReadRemoteStorageConf(commandEnv.option.GrpcDialOption, commandEnv.option.FilerAddress, remote.Name) + +} + +func (c *commandRemoteMount) pullMetadata(commandEnv *CommandEnv, writer io.Writer, dir string, nonEmpty bool, remoteConf *filer_pb.RemoteConf, remote *filer_pb.RemoteStorageLocation) error { + + // find existing directory, and ensure the directory is empty + err := commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { + parent, name := util.FullPath(dir).DirAndName() + _, lookupErr := client.LookupDirectoryEntry(context.Background(), &filer_pb.LookupDirectoryEntryRequest{ + Directory: parent, + Name: name, + }) + if lookupErr != nil { + return fmt.Errorf("lookup %s: %v", dir, lookupErr) + } + + mountToDirIsEmpty := true + listErr := filer_pb.SeaweedList(client, dir, "", func(entry *filer_pb.Entry, isLast bool) error { + mountToDirIsEmpty = false + return nil + }, "", false, 1) + + if listErr != nil { + return fmt.Errorf("list %s: %v", dir, listErr) + } + + if !mountToDirIsEmpty { + if !nonEmpty { + return fmt.Errorf("dir %s is not empty", dir) + } + } + + return nil + }) + if err != nil { + return err + } + + // visit remote storage + remoteStorage, err := remote_storage.GetRemoteStorage(remoteConf) + if err != nil { + return err + } + + err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { + ctx := context.Background() + err = remoteStorage.Traverse(remote, func(remoteDir, name string, isDirectory bool, remoteEntry *filer_pb.RemoteEntry) error { + localDir := dir + remoteDir + println(util.NewFullPath(localDir, name)) + + lookupResponse, lookupErr := filer_pb.LookupEntry(client, &filer_pb.LookupDirectoryEntryRequest{ + Directory: localDir, + Name: name, + }) + var existingEntry *filer_pb.Entry + if lookupErr != nil { + if lookupErr != filer_pb.ErrNotFound { + return lookupErr + } + } else { + existingEntry = lookupResponse.Entry + } + + if existingEntry == nil { + _, createErr := client.CreateEntry(ctx, &filer_pb.CreateEntryRequest{ + Directory: localDir, + Entry: &filer_pb.Entry{ + Name: name, + IsDirectory: isDirectory, + Attributes: &filer_pb.FuseAttributes{ + FileSize: uint64(remoteEntry.Size), + Mtime: remoteEntry.LastModifiedAt, + FileMode: uint32(0644), + }, + RemoteEntry: remoteEntry, + }, + }) + return createErr + } else { + if existingEntry.RemoteEntry == nil || existingEntry.RemoteEntry.ETag != remoteEntry.ETag { + existingEntry.RemoteEntry = remoteEntry + existingEntry.Attributes.FileSize = uint64(remoteEntry.Size) + existingEntry.Attributes.Mtime = remoteEntry.LastModifiedAt + _, updateErr := client.UpdateEntry(ctx, &filer_pb.UpdateEntryRequest{ + Directory: localDir, + Entry: existingEntry, + }) + return updateErr + } + } + return nil + }) + return err + }) + + if err != nil { + return err + } + + return nil +} + +func (c *commandRemoteMount) saveMountMapping(commandEnv *CommandEnv, writer io.Writer, dir string, remoteStorageLocation *filer_pb.RemoteStorageLocation) (err error) { + + // read current mapping + var oldContent, newContent []byte + err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { + oldContent, err = filer.ReadInsideFiler(client, filer.DirectoryEtcRemote, filer.REMOTE_STORAGE_MOUNT_FILE) + return err + }) + if err != nil { + if err != filer_pb.ErrNotFound { + return fmt.Errorf("read existing mapping: %v", err) + } + } + + // add new mapping + newContent, err = filer.AddRemoteStorageMapping(oldContent, dir, remoteStorageLocation) + if err != nil { + return fmt.Errorf("add mapping %s~%s: %v", dir, remoteStorageLocation, err) + } + + // save back + err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error { + return filer.SaveInsideFiler(client, filer.DirectoryEtcRemote, filer.REMOTE_STORAGE_MOUNT_FILE, newContent) + }) + if err != nil { + return fmt.Errorf("save mapping: %v", err) + } + + return nil +} diff --git a/weed/shell/command_volume_balance_test.go b/weed/shell/command_volume_balance_test.go index b77811f51..d6624bbcc 100644 --- a/weed/shell/command_volume_balance_test.go +++ b/weed/shell/command_volume_balance_test.go @@ -1,6 +1,8 @@ package shell import ( + "github.com/chrislusf/seaweedfs/weed/storage/types" + "github.com/stretchr/testify/assert" "testing" "github.com/chrislusf/seaweedfs/weed/pb/master_pb" @@ -181,3 +183,14 @@ func TestBalance(t *testing.T) { } } + +func TestVolumeSelection(t *testing.T) { + topologyInfo := parseOutput(topoData) + + vids, err := collectVolumeIdsForTierChange(nil, topologyInfo, 1000, types.ToDiskType("hdd"), "", 20.0, 0); + if err != nil { + t.Errorf("collectVolumeIdsForTierChange: %v", err) + } + assert.Equal(t, 378, len(vids)) + +} diff --git a/weed/shell/command_volume_list_test.go b/weed/shell/command_volume_list_test.go index 72c76f242..379cf4943 100644 --- a/weed/shell/command_volume_list_test.go +++ b/weed/shell/command_volume_list_test.go @@ -68,7 +68,7 @@ func parseOutput(output string) *master_pb.TopologyInfo { maxVolumeCount, _ := strconv.Atoi(maxVolumeCountStr) disk = &master_pb.DiskInfo{ Type: diskType, - MaxVolumeCount: uint64(maxVolumeCount), + MaxVolumeCount: int64(maxVolumeCount), } dn.DiskInfos[types.ToDiskType(diskType).String()] = disk } else { diff --git a/weed/shell/commands.go b/weed/shell/commands.go index 697c68ae3..5497e89cc 100644 --- a/weed/shell/commands.go +++ b/weed/shell/commands.go @@ -20,9 +20,10 @@ type ShellOptions struct { Masters *string GrpcDialOption grpc.DialOption // shell transient context - FilerHost string - FilerPort int64 - Directory string + FilerHost string + FilerPort int64 + FilerAddress string + Directory string } type CommandEnv struct { diff --git a/weed/topology/store_replicate.go b/weed/topology/store_replicate.go index ea0a8c968..6d68bb26f 100644 --- a/weed/topology/store_replicate.go +++ b/weed/topology/store_replicate.go @@ -50,7 +50,7 @@ func ReplicatedWrite(masterFn operation.GetMasterFn, s *storage.Store, volumeId } if len(remoteLocations) > 0 { //send to other replica locations - if err = distributedOperation(remoteLocations, s, func(location operation.Location) error { + if err = DistributedOperation(remoteLocations, func(location operation.Location) error { u := url.URL{ Scheme: "http", Host: location.Url, @@ -115,7 +115,7 @@ func ReplicatedDelete(masterFn operation.GetMasterFn, store *storage.Store, } if len(remoteLocations) > 0 { //send to other replica locations - if err = distributedOperation(remoteLocations, store, func(location operation.Location) error { + if err = DistributedOperation(remoteLocations, func(location operation.Location) error { return util.Delete("http://"+location.Url+r.URL.Path+"?type=replicate", string(jwt)) }); err != nil { size = 0 @@ -144,7 +144,7 @@ type RemoteResult struct { Error error } -func distributedOperation(locations []operation.Location, store *storage.Store, op func(location operation.Location) error) error { +func DistributedOperation(locations []operation.Location, op func(location operation.Location) error) error { length := len(locations) results := make(chan RemoteResult) for _, location := range locations {