aboutsummaryrefslogtreecommitdiff
path: root/weed
diff options
context:
space:
mode:
Diffstat (limited to 'weed')
-rw-r--r--weed/Makefile4
-rw-r--r--weed/command/benchmark.go27
-rw-r--r--weed/command/command.go2
-rw-r--r--weed/command/filer.go37
-rw-r--r--weed/command/filer_backup.go157
-rw-r--r--weed/command/filer_cat.go2
-rw-r--r--weed/command/filer_copy.go2
-rw-r--r--weed/command/filer_meta_backup.go268
-rw-r--r--weed/command/filer_meta_tail.go8
-rw-r--r--weed/command/filer_replication.go29
-rw-r--r--weed/command/filer_sync.go203
-rw-r--r--weed/command/master.go7
-rw-r--r--weed/command/mount.go4
-rw-r--r--weed/command/mount_std.go24
-rw-r--r--weed/command/msg_broker.go2
-rw-r--r--weed/command/s3.go2
-rw-r--r--weed/command/scaffold.go37
-rw-r--r--weed/command/server.go36
-rw-r--r--weed/command/upload.go2
-rw-r--r--weed/command/volume.go86
-rw-r--r--weed/command/webdav.go2
-rw-r--r--weed/filer/abstract_sql/abstract_sql_store.go41
-rw-r--r--weed/filer/filechunk_manifest.go11
-rw-r--r--weed/filer/filer.go8
-rw-r--r--weed/filer/filer_delete_entry.go6
-rw-r--r--weed/filer/filer_on_meta_event.go25
-rw-r--r--weed/filer/filerstore.go5
-rw-r--r--weed/filer/filerstore_wrapper.go23
-rw-r--r--weed/filer/leveldb/leveldb_store.go2
-rw-r--r--weed/filer/leveldb2/leveldb2_store.go1
-rw-r--r--weed/filer/leveldb3/leveldb3_store.go1
-rw-r--r--weed/filer/meta_aggregator.go1
-rw-r--r--weed/filer/mysql/mysql_sql_gen.go42
-rw-r--r--weed/filer/mysql/mysql_store.go11
-rw-r--r--weed/filer/mysql2/mysql2_store.go8
-rw-r--r--weed/filer/postgres/postgres_sql_gen.go41
-rw-r--r--weed/filer/postgres/postgres_store.go8
-rw-r--r--weed/filer/postgres2/postgres2_store.go8
-rw-r--r--weed/filer/read_write.go4
-rw-r--r--weed/filer/rocksdb/rocksdb_store.go2
-rw-r--r--weed/filer/stream.go34
-rw-r--r--weed/filesys/dir.go43
-rw-r--r--weed/filesys/dir_link.go37
-rw-r--r--weed/filesys/dir_rename.go4
-rw-r--r--weed/filesys/dir_test.go34
-rw-r--r--weed/filesys/dirty_page.go9
-rw-r--r--weed/filesys/file.go87
-rw-r--r--weed/filesys/filehandle.go96
-rw-r--r--weed/filesys/fscache.go5
-rw-r--r--weed/filesys/wfs.go1
-rw-r--r--weed/operation/buffer_pool.go24
-rw-r--r--weed/operation/upload_content.go7
-rw-r--r--weed/pb/filer_pb/filer.pb.go2
-rw-r--r--weed/pb/filer_pb/filer_client.go34
-rw-r--r--weed/pb/grpc_client_server.go65
-rw-r--r--weed/pb/iam_pb/iam.pb.go2
-rw-r--r--weed/pb/master_pb/master.pb.go2
-rw-r--r--weed/pb/messaging_pb/messaging.pb.go2
-rw-r--r--weed/pb/volume_info.go29
-rw-r--r--weed/pb/volume_server.proto24
-rw-r--r--weed/pb/volume_server_pb/volume_server.pb.go1960
-rw-r--r--weed/replication/repl_util/replication_util.go (renamed from weed/replication/repl_util/replication_utli.go)2
-rw-r--r--weed/replication/replicator.go2
-rw-r--r--weed/replication/sink/azuresink/azure_sink.go14
-rw-r--r--weed/replication/sink/b2sink/b2_sink.go14
-rw-r--r--weed/replication/sink/filersink/filer_sink.go6
-rw-r--r--weed/replication/sink/gcssink/gcs_sink.go14
-rw-r--r--weed/replication/sink/localsink/local_sink.go8
-rw-r--r--weed/replication/sink/replication_sink.go1
-rw-r--r--weed/replication/sink/s3sink/s3_sink.go29
-rw-r--r--weed/replication/sink/s3sink/s3_write.go13
-rw-r--r--weed/replication/source/filer_source.go8
-rw-r--r--weed/s3api/filer_util.go10
-rw-r--r--weed/s3api/s3api_bucket_handlers.go2
-rw-r--r--weed/s3api/s3api_object_copy_handlers.go27
-rw-r--r--weed/s3api/s3api_object_handlers.go28
-rw-r--r--weed/s3api/s3api_object_handlers_postpolicy.go2
-rw-r--r--weed/s3api/s3api_objects_list_handlers.go1
-rw-r--r--weed/security/tls.go66
-rw-r--r--weed/sequence/snowflake_sequencer.go46
-rw-r--r--weed/server/common.go8
-rw-r--r--weed/server/filer_grpc_server_rename.go25
-rw-r--r--weed/server/filer_server.go45
-rw-r--r--weed/server/filer_server_handlers.go39
-rw-r--r--weed/server/filer_server_handlers_read.go13
-rw-r--r--weed/server/filer_server_handlers_write.go4
-rw-r--r--weed/server/filer_server_handlers_write_autochunk.go127
-rw-r--r--weed/server/filer_server_handlers_write_upload.go175
-rw-r--r--weed/server/master_grpc_server.go6
-rw-r--r--weed/server/master_grpc_server_volume.go7
-rw-r--r--weed/server/master_server.go7
-rw-r--r--weed/server/volume_grpc_read_write.go38
-rw-r--r--weed/server/volume_server.go8
-rw-r--r--weed/server/volume_server_handlers.go30
-rw-r--r--weed/server/volume_server_handlers_read.go7
-rw-r--r--weed/server/volume_server_tcp_handlers_write.go137
-rw-r--r--weed/shell/command_ec_encode.go2
-rw-r--r--weed/shell/command_fs_cat.go2
-rw-r--r--weed/shell/command_fs_mv.go4
-rw-r--r--weed/shell/command_s3_clean_uploads.go92
-rw-r--r--weed/shell/command_volume_balance.go5
-rw-r--r--weed/shell/command_volume_balance_test.go12
-rw-r--r--weed/shell/command_volume_check_disk.go257
-rw-r--r--weed/shell/command_volume_list_test.go893
-rw-r--r--weed/shell/command_volume_server_evacuate_test.go17
-rw-r--r--weed/shell/shell_liner.go11
-rw-r--r--weed/storage/backend/backend.go6
-rw-r--r--weed/storage/backend/disk_file.go2
-rw-r--r--weed/storage/backend/s3_backend/s3_sessions.go5
-rw-r--r--weed/storage/erasure_coding/ec_volume.go2
-rw-r--r--weed/storage/needle/crc.go23
-rw-r--r--weed/storage/needle/needle_read_write.go36
-rw-r--r--weed/storage/needle_map/memdb.go9
-rw-r--r--weed/storage/needle_map_leveldb.go6
-rw-r--r--weed/storage/needle_map_sorted_file.go8
-rw-r--r--weed/storage/store.go21
-rw-r--r--weed/storage/super_block/replica_placement.go3
-rw-r--r--weed/storage/volume.go16
-rw-r--r--weed/storage/volume_loading.go10
-rw-r--r--weed/storage/volume_read.go131
-rw-r--r--weed/storage/volume_stream_write.go104
-rw-r--r--weed/storage/volume_tier.go13
-rw-r--r--weed/storage/volume_vacuum.go2
-rw-r--r--weed/storage/volume_write.go (renamed from weed/storage/volume_read_write.go)185
-rw-r--r--weed/topology/data_node_ec.go17
-rw-r--r--weed/topology/volume_layout.go2
-rw-r--r--weed/topology/volume_location_list.go2
-rw-r--r--weed/util/chunk_cache/chunk_cache_in_memory.go4
-rw-r--r--weed/util/constants.go2
-rw-r--r--weed/util/fasthttp_util.go115
-rw-r--r--weed/util/fla9/fla9.go5
-rw-r--r--weed/util/http_util.go2
-rw-r--r--weed/util/limiter.go74
-rw-r--r--weed/wdclient/net2/base_connection_pool.go159
-rw-r--r--weed/wdclient/net2/connection_pool.go97
-rw-r--r--weed/wdclient/net2/doc.go6
-rw-r--r--weed/wdclient/net2/ip.go177
-rw-r--r--weed/wdclient/net2/managed_connection.go185
-rw-r--r--weed/wdclient/net2/port.go19
-rw-r--r--weed/wdclient/resource_pool/doc.go5
-rw-r--r--weed/wdclient/resource_pool/managed_handle.go97
-rw-r--r--weed/wdclient/resource_pool/multi_resource_pool.go200
-rw-r--r--weed/wdclient/resource_pool/resource_pool.go96
-rw-r--r--weed/wdclient/resource_pool/semaphore.go154
-rw-r--r--weed/wdclient/resource_pool/simple_resource_pool.go343
-rw-r--r--weed/wdclient/volume_tcp_client.go97
146 files changed, 6618 insertions, 1776 deletions
diff --git a/weed/Makefile b/weed/Makefile
index fd0843c22..8f1257d09 100644
--- a/weed/Makefile
+++ b/weed/Makefile
@@ -33,3 +33,7 @@ debug_webdav:
debug_s3:
go build -gcflags="all=-N -l"
dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec weed -- -v=4 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
diff --git a/weed/command/benchmark.go b/weed/command/benchmark.go
index c1bc80c42..4fedb55f1 100644
--- a/weed/command/benchmark.go
+++ b/weed/command/benchmark.go
@@ -41,6 +41,7 @@ type BenchmarkOptions struct {
grpcDialOption grpc.DialOption
masterClient *wdclient.MasterClient
fsync *bool
+ useTcp *bool
}
var (
@@ -67,6 +68,7 @@ func init() {
b.cpuprofile = cmdBenchmark.Flag.String("cpuprofile", "", "cpu profile output file")
b.maxCpu = cmdBenchmark.Flag.Int("maxCpu", 0, "maximum number of CPUs. 0 means all available CPUs")
b.fsync = cmdBenchmark.Flag.Bool("fsync", false, "flush data to disk after write")
+ b.useTcp = cmdBenchmark.Flag.Bool("useTcp", false, "send data via tcp")
sharedBytes = make([]byte, 1024)
}
@@ -223,6 +225,8 @@ func writeFiles(idChan chan int, fileIdLineChan chan string, s *stat) {
random := rand.New(rand.NewSource(time.Now().UnixNano()))
+ volumeTcpClient := wdclient.NewVolumeTcpClient()
+
for id := range idChan {
start := time.Now()
fileSize := int64(*b.fileSize + random.Intn(64))
@@ -243,7 +247,15 @@ func writeFiles(idChan chan int, fileIdLineChan chan string, s *stat) {
if !isSecure && assignResult.Auth != "" {
isSecure = true
}
- if _, err := fp.Upload(0, b.masterClient.GetMaster, false, assignResult.Auth, b.grpcDialOption); err == nil {
+ if *b.useTcp {
+ if uploadByTcp(volumeTcpClient, fp) {
+ fileIdLineChan <- fp.Fid
+ s.completed++
+ s.transferred += fileSize
+ } else {
+ s.failed++
+ }
+ } else if _, err := fp.Upload(0, b.masterClient.GetMaster, false, assignResult.Auth, b.grpcDialOption); err == nil {
if random.Intn(100) < *b.deletePercentage {
s.total++
delayedDeleteChan <- &delayedFile{time.Now().Add(time.Second), fp}
@@ -293,7 +305,7 @@ func readFiles(fileIdLineChan chan string, s *stat) {
}
var bytes []byte
for _, url := range urls {
- bytes, _, err = util.FastGet(url)
+ bytes, _, err = util.Get(url)
if err == nil {
break
}
@@ -329,6 +341,17 @@ func writeFileIds(fileName string, fileIdLineChan chan string, finishChan chan b
}
}
+func uploadByTcp(volumeTcpClient *wdclient.VolumeTcpClient, fp *operation.FilePart) bool {
+
+ err := volumeTcpClient.PutFileChunk(fp.Server, fp.Fid, uint32(fp.FileSize), fp.Reader)
+ if err != nil {
+ glog.Errorf("upload chunk err: %v", err)
+ return false
+ }
+
+ return true
+}
+
func readFileIds(fileName string, fileIdLineChan chan string) {
file, err := os.Open(fileName) // For read access.
if err != nil {
diff --git a/weed/command/command.go b/weed/command/command.go
index fea1dd9d3..ce754702f 100644
--- a/weed/command/command.go
+++ b/weed/command/command.go
@@ -15,7 +15,9 @@ var Commands = []*Command{
cmdDownload,
cmdExport,
cmdFiler,
+ cmdFilerBackup,
cmdFilerCat,
+ cmdFilerMetaBackup,
cmdFilerMetaTail,
cmdFilerReplicate,
cmdFilerSynchronize,
diff --git a/weed/command/filer.go b/weed/command/filer.go
index b256e385f..08385e62c 100644
--- a/weed/command/filer.go
+++ b/weed/command/filer.go
@@ -49,6 +49,7 @@ type FilerOptions struct {
metricsHttpPort *int
saveToFilerLimit *int
defaultLevelDbDirectory *string
+ concurrentUploadLimitMB *int
}
func init() {
@@ -56,12 +57,12 @@ func init() {
f.masters = cmdFiler.Flag.String("master", "localhost:9333", "comma-separated master servers")
f.collection = cmdFiler.Flag.String("collection", "", "all data will be stored in this default collection")
f.ip = cmdFiler.Flag.String("ip", util.DetectedHostAddress(), "filer server http listen ip address")
- f.bindIp = cmdFiler.Flag.String("ip.bind", "0.0.0.0", "ip address to bind to")
+ f.bindIp = cmdFiler.Flag.String("ip.bind", "", "ip address to bind to")
f.port = cmdFiler.Flag.Int("port", 8888, "filer server http listen port")
f.publicPort = cmdFiler.Flag.Int("port.readonly", 0, "readonly port opened to public")
f.defaultReplicaPlacement = cmdFiler.Flag.String("defaultReplicaPlacement", "", "default replication type. If not specified, use master setting.")
f.disableDirListing = cmdFiler.Flag.Bool("disableDirListing", false, "turn off directory listing")
- f.maxMB = cmdFiler.Flag.Int("maxMB", 32, "split files larger than the limit")
+ f.maxMB = cmdFiler.Flag.Int("maxMB", 4, "split files larger than the limit")
f.dirListingLimit = cmdFiler.Flag.Int("dirListLimit", 100000, "limit sub dir listing size")
f.dataCenter = cmdFiler.Flag.String("dataCenter", "", "prefer to read and write to volumes in this data center")
f.rack = cmdFiler.Flag.String("rack", "", "prefer to write to volumes in this rack")
@@ -71,6 +72,7 @@ func init() {
f.metricsHttpPort = cmdFiler.Flag.Int("metricsPort", 0, "Prometheus metrics listen port")
f.saveToFilerLimit = cmdFiler.Flag.Int("saveToFilerLimit", 0, "files smaller than this limit will be saved in filer store")
f.defaultLevelDbDirectory = cmdFiler.Flag.String("defaultStoreDir", ".", "if filer.toml is empty, use an embedded filer store in the directory")
+ f.concurrentUploadLimitMB = cmdFiler.Flag.Int("concurrentUploadLimitMB", 128, "limit total concurrent upload size")
// start s3 on filer
filerStartS3 = cmdFiler.Flag.Bool("s3", false, "whether to start S3 gateway")
@@ -176,21 +178,22 @@ func (fo *FilerOptions) startFiler() {
}
fs, nfs_err := weed_server.NewFilerServer(defaultMux, publicVolumeMux, &weed_server.FilerOption{
- Masters: strings.Split(*fo.masters, ","),
- Collection: *fo.collection,
- DefaultReplication: *fo.defaultReplicaPlacement,
- DisableDirListing: *fo.disableDirListing,
- MaxMB: *fo.maxMB,
- DirListingLimit: *fo.dirListingLimit,
- DataCenter: *fo.dataCenter,
- Rack: *fo.rack,
- DefaultLevelDbDir: defaultLevelDbDirectory,
- DisableHttp: *fo.disableHttp,
- Host: *fo.ip,
- Port: uint32(*fo.port),
- Cipher: *fo.cipher,
- SaveToFilerLimit: *fo.saveToFilerLimit,
- Filers: peers,
+ Masters: strings.Split(*fo.masters, ","),
+ Collection: *fo.collection,
+ DefaultReplication: *fo.defaultReplicaPlacement,
+ DisableDirListing: *fo.disableDirListing,
+ MaxMB: *fo.maxMB,
+ DirListingLimit: *fo.dirListingLimit,
+ DataCenter: *fo.dataCenter,
+ Rack: *fo.rack,
+ DefaultLevelDbDir: defaultLevelDbDirectory,
+ DisableHttp: *fo.disableHttp,
+ Host: *fo.ip,
+ Port: uint32(*fo.port),
+ Cipher: *fo.cipher,
+ SaveToFilerLimit: int64(*fo.saveToFilerLimit),
+ Filers: peers,
+ ConcurrentUploadLimit: int64(*fo.concurrentUploadLimitMB) * 1024 * 1024,
})
if nfs_err != nil {
glog.Fatalf("Filer startup error: %v", nfs_err)
diff --git a/weed/command/filer_backup.go b/weed/command/filer_backup.go
new file mode 100644
index 000000000..888b46fe7
--- /dev/null
+++ b/weed/command/filer_backup.go
@@ -0,0 +1,157 @@
+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"
+)
+
+type FilerBackupOptions struct {
+ isActivePassive *bool
+ filer *string
+ path *string
+ debug *bool
+ proxyByFiler *bool
+ timeAgo *time.Duration
+}
+
+var (
+ filerBackupOptions FilerBackupOptions
+)
+
+func init() {
+ cmdFilerBackup.Run = runFilerBackup // break init cycle
+ filerBackupOptions.filer = cmdFilerBackup.Flag.String("filer", "localhost:8888", "filer of one SeaweedFS cluster")
+ filerBackupOptions.path = cmdFilerBackup.Flag.String("filerPath", "/", "directory to sync on filer")
+ filerBackupOptions.proxyByFiler = cmdFilerBackup.Flag.Bool("filerProxy", false, "read and write file chunks by filer instead of volume servers")
+ filerBackupOptions.debug = cmdFilerBackup.Flag.Bool("debug", false, "debug mode to print out received files")
+ filerBackupOptions.timeAgo = cmdFilerBackup.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 cmdFilerBackup = &Command{
+ UsageLine: "filer.backup -filer=<filerHost>:<filerPort> ",
+ Short: "resume-able continuously replicate files from a SeaweedFS cluster to another location defined in replication.toml",
+ Long: `resume-able continuously replicate files from a SeaweedFS cluster to another location defined in replication.toml
+
+ filer.backup listens on filer notifications. If any file is updated, it will fetch the updated content,
+ and write to the destination. This is to replace filer.replicate command since additional message queue is not needed.
+
+ If restarted and "-timeAgo" is not set, the synchronization will resume from the previous checkpoints, persisted every minute.
+ A fresh sync will start from the earliest metadata logs. To reset the checkpoints, just set "-timeAgo" to a high value.
+
+`,
+}
+
+func runFilerBackup(cmd *Command, args []string) bool {
+
+ grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client")
+
+ util.LoadConfiguration("security", false)
+ util.LoadConfiguration("replication", true)
+
+ for {
+ err := doFilerBackup(grpcDialOption, &filerBackupOptions)
+ if err != nil {
+ glog.Errorf("backup from %s: %v", *filerBackupOptions.filer, err)
+ time.Sleep(1747 * time.Millisecond)
+ }
+ }
+
+ return true
+}
+
+const (
+ BackupKeyPrefix = "backup."
+)
+
+func doFilerBackup(grpcDialOption grpc.DialOption, backupOption *FilerBackupOptions) error {
+
+ // find data sink
+ config := util.GetViper()
+ dataSink := findSink(config)
+ if dataSink == nil {
+ return fmt.Errorf("no data sink configured in replication.toml")
+ }
+
+ sourceFiler := *backupOption.filer
+ sourcePath := *backupOption.path
+ timeAgo := *backupOption.timeAgo
+ targetPath := dataSink.GetSinkToDirectory()
+ debug := *backupOption.debug
+
+ // get start time for the data sink
+ startFrom := time.Unix(0, 0)
+ sinkId := util.HashStringToLong(dataSink.GetName() + dataSink.GetSinkToDirectory())
+ if timeAgo.Milliseconds() == 0 {
+ lastOffsetTsNs, err := getOffset(grpcDialOption, sourceFiler, BackupKeyPrefix, int32(sinkId))
+ if err != nil {
+ glog.V(0).Infof("starting from %v", startFrom)
+ } else {
+ startFrom = time.Unix(0, lastOffsetTsNs)
+ glog.V(0).Infof("resuming from %v", startFrom)
+ }
+ } else {
+ startFrom = time.Now().Add(-timeAgo)
+ glog.V(0).Infof("start time is set to %v", startFrom)
+ }
+
+ // create filer sink
+ filerSource := &source.FilerSource{}
+ filerSource.DoInitialize(sourceFiler, pb.ServerToGrpcAddress(sourceFiler), sourcePath, *backupOption.proxyByFiler)
+ dataSink.SetSourceFiler(filerSource)
+
+ 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)
+ }
+ }
+
+ }
+
+ })
+
+}
diff --git a/weed/command/filer_cat.go b/weed/command/filer_cat.go
index a46098b04..c4281feba 100644
--- a/weed/command/filer_cat.go
+++ b/weed/command/filer_cat.go
@@ -110,7 +110,7 @@ func runFilerCat(cmd *Command, args []string) bool {
filerCat.filerClient = client
- return filer.StreamContent(&filerCat, writer, respLookupEntry.Entry.Chunks, 0, math.MaxInt64)
+ return filer.StreamContent(&filerCat, writer, respLookupEntry.Entry.Chunks, 0, math.MaxInt64, false)
})
diff --git a/weed/command/filer_copy.go b/weed/command/filer_copy.go
index 42f5ec4c3..a36bb8cea 100644
--- a/weed/command/filer_copy.go
+++ b/weed/command/filer_copy.go
@@ -56,7 +56,7 @@ func init() {
copy.collection = cmdCopy.Flag.String("collection", "", "optional collection name")
copy.ttl = cmdCopy.Flag.String("ttl", "", "time to live, e.g.: 1m, 1h, 1d, 1M, 1y")
copy.diskType = cmdCopy.Flag.String("disk", "", "[hdd|ssd|<tag>] hard drive or solid state drive or any tag")
- copy.maxMB = cmdCopy.Flag.Int("maxMB", 32, "split files larger than the limit")
+ copy.maxMB = cmdCopy.Flag.Int("maxMB", 4, "split files larger than the limit")
copy.concurrenctFiles = cmdCopy.Flag.Int("c", 8, "concurrent file copy goroutines")
copy.concurrenctChunks = cmdCopy.Flag.Int("concurrentChunks", 8, "concurrent chunk copy goroutines for each file")
}
diff --git a/weed/command/filer_meta_backup.go b/weed/command/filer_meta_backup.go
new file mode 100644
index 000000000..ba0b44659
--- /dev/null
+++ b/weed/command/filer_meta_backup.go
@@ -0,0 +1,268 @@
+package command
+
+import (
+ "context"
+ "fmt"
+ "github.com/chrislusf/seaweedfs/weed/filer"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/spf13/viper"
+ "google.golang.org/grpc"
+ "io"
+ "reflect"
+ "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"
+)
+
+var (
+ metaBackup FilerMetaBackupOptions
+)
+
+type FilerMetaBackupOptions struct {
+ grpcDialOption grpc.DialOption
+ filerAddress *string
+ filerDirectory *string
+ restart *bool
+ backupFilerConfig *string
+
+ store filer.FilerStore
+}
+
+func init() {
+ cmdFilerMetaBackup.Run = runFilerMetaBackup // break init cycle
+ metaBackup.filerAddress = cmdFilerMetaBackup.Flag.String("filer", "localhost:8888", "filer hostname:port")
+ metaBackup.filerDirectory = cmdFilerMetaBackup.Flag.String("filerDir", "/", "a folder on the filer")
+ metaBackup.restart = cmdFilerMetaBackup.Flag.Bool("restart", false, "copy the full metadata before async incremental backup")
+ metaBackup.backupFilerConfig = cmdFilerMetaBackup.Flag.String("config", "", "path to filer.toml specifying backup filer store")
+}
+
+var cmdFilerMetaBackup = &Command{
+ UsageLine: "filer.meta.backup [-filer=localhost:8888] [-filerDir=/] [-restart] -config=/path/to/backup_filer.toml",
+ Short: "continuously backup filer meta data changes to anther filer store specified in a backup_filer.toml",
+ Long: `continuously backup filer meta data changes.
+The backup writes to another filer store specified in a backup_filer.toml.
+
+ weed filer.meta.backup -config=/path/to/backup_filer.toml -filer="localhost:8888"
+ weed filer.meta.backup -config=/path/to/backup_filer.toml -filer="localhost:8888" -restart
+
+ `,
+}
+
+func runFilerMetaBackup(cmd *Command, args []string) bool {
+
+ metaBackup.grpcDialOption = security.LoadClientTLS(util.GetViper(), "grpc.client")
+
+ // load backup_filer.toml
+ v := viper.New()
+ v.SetConfigFile(*metaBackup.backupFilerConfig)
+
+ if err := v.ReadInConfig(); err != nil { // Handle errors reading the config file
+ glog.Fatalf("Failed to load %s file.\nPlease use this command to generate the a %s.toml file\n"+
+ " weed scaffold -config=%s -output=.\n\n\n",
+ *metaBackup.backupFilerConfig, "backup_filer", "filer")
+ }
+
+ if err := metaBackup.initStore(v); err != nil {
+ glog.V(0).Infof("init backup filer store: %v", err)
+ return true
+ }
+
+ missingPreviousBackup := false
+ _, err := metaBackup.getOffset()
+ if err != nil {
+ missingPreviousBackup = true
+ }
+
+ if *metaBackup.restart || missingPreviousBackup {
+ glog.V(0).Infof("traversing metadata tree...")
+ startTime := time.Now()
+ if err := metaBackup.traverseMetadata(); err != nil {
+ glog.Errorf("traverse meta data: %v", err)
+ return true
+ }
+ glog.V(0).Infof("metadata copied up to %v", startTime)
+ if err := metaBackup.setOffset(startTime); err != nil {
+ startTime = time.Now()
+ }
+ }
+
+ for {
+ err := metaBackup.streamMetadataBackup()
+ if err != nil {
+ glog.Errorf("filer meta backup from %s: %v", *metaBackup.filerAddress, err)
+ time.Sleep(1747 * time.Millisecond)
+ }
+ }
+
+ return true
+}
+
+func (metaBackup *FilerMetaBackupOptions) initStore(v *viper.Viper) error {
+ // load configuration for default filer store
+ hasDefaultStoreConfigured := false
+ for _, store := range filer.Stores {
+ if v.GetBool(store.GetName() + ".enabled") {
+ store = reflect.New(reflect.ValueOf(store).Elem().Type()).Interface().(filer.FilerStore)
+ if err := store.Initialize(v, store.GetName()+"."); err != nil {
+ glog.Fatalf("failed to initialize store for %s: %+v", store.GetName(), err)
+ }
+ glog.V(0).Infof("configured filer store to %s", store.GetName())
+ hasDefaultStoreConfigured = true
+ metaBackup.store = filer.NewFilerStoreWrapper(store)
+ break
+ }
+ }
+ if !hasDefaultStoreConfigured {
+ return fmt.Errorf("no filer store enabled in %s", v.ConfigFileUsed())
+ }
+
+ return nil
+}
+
+func (metaBackup *FilerMetaBackupOptions) traverseMetadata() (err error) {
+ var saveErr error
+
+ traverseErr := filer_pb.TraverseBfs(metaBackup, util.FullPath(*metaBackup.filerDirectory), func(parentPath util.FullPath, entry *filer_pb.Entry) {
+
+ println("+", parentPath.Child(entry.Name))
+ if err := metaBackup.store.InsertEntry(context.Background(), filer.FromPbEntry(string(parentPath), entry)); err != nil {
+ saveErr = fmt.Errorf("insert entry error: %v\n", err)
+ return
+ }
+
+ })
+
+ if traverseErr != nil {
+ return fmt.Errorf("traverse: %v", traverseErr)
+ }
+ return saveErr
+}
+
+var (
+ MetaBackupKey = []byte("metaBackup")
+)
+
+func (metaBackup *FilerMetaBackupOptions) streamMetadataBackup() error {
+
+ startTime, err := metaBackup.getOffset()
+ if err != nil {
+ startTime = time.Now()
+ }
+ glog.V(0).Infof("streaming from %v", startTime)
+
+ store := metaBackup.store
+
+ eachEntryFunc := func(resp *filer_pb.SubscribeMetadataResponse) error {
+
+ ctx := context.Background()
+ message := resp.EventNotification
+
+ if message.OldEntry == nil && message.NewEntry == nil {
+ return nil
+ }
+ if message.OldEntry == nil && message.NewEntry != nil {
+ println("+", util.FullPath(message.NewParentPath).Child(message.NewEntry.Name))
+ entry := filer.FromPbEntry(message.NewParentPath, message.NewEntry)
+ return store.InsertEntry(ctx, entry)
+ }
+ if message.OldEntry != nil && message.NewEntry == nil {
+ println("-", util.FullPath(resp.Directory).Child(message.OldEntry.Name))
+ return store.DeleteEntry(ctx, util.FullPath(resp.Directory).Child(message.OldEntry.Name))
+ }
+ if message.OldEntry != nil && message.NewEntry != nil {
+ if resp.Directory == message.NewParentPath && message.OldEntry.Name == message.NewEntry.Name {
+ println("~", util.FullPath(message.NewParentPath).Child(message.NewEntry.Name))
+ entry := filer.FromPbEntry(message.NewParentPath, message.NewEntry)
+ return store.UpdateEntry(ctx, entry)
+ }
+ println("-", util.FullPath(resp.Directory).Child(message.OldEntry.Name))
+ if err := store.DeleteEntry(ctx, util.FullPath(resp.Directory).Child(message.OldEntry.Name)); err != nil {
+ return err
+ }
+ println("+", util.FullPath(message.NewParentPath).Child(message.NewEntry.Name))
+ return store.InsertEntry(ctx, filer.FromPbEntry(message.NewParentPath, message.NewEntry))
+ }
+
+ 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
+ }
+ }
+
+ }
+
+ })
+ return tailErr
+}
+
+func (metaBackup *FilerMetaBackupOptions) getOffset() (lastWriteTime time.Time, err error) {
+ value, err := metaBackup.store.KvGet(context.Background(), MetaBackupKey)
+ if err != nil {
+ return
+ }
+ tsNs := util.BytesToUint64(value)
+
+ return time.Unix(0, int64(tsNs)), nil
+}
+
+func (metaBackup *FilerMetaBackupOptions) setOffset(lastWriteTime time.Time) error {
+ valueBuf := make([]byte, 8)
+ util.Uint64toBytes(valueBuf, uint64(lastWriteTime.UnixNano()))
+
+ if err := metaBackup.store.KvPut(context.Background(), MetaBackupKey, valueBuf); err != nil {
+ return err
+ }
+ return nil
+}
+
+var _ = filer_pb.FilerClient(&FilerMetaBackupOptions{})
+
+func (metaBackup *FilerMetaBackupOptions) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
+
+ return pb.WithFilerClient(*metaBackup.filerAddress, metaBackup.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
+ return fn(client)
+ })
+
+}
+
+func (metaBackup *FilerMetaBackupOptions) AdjustedUrl(location *filer_pb.Location) string {
+ return location.Url
+}
diff --git a/weed/command/filer_meta_tail.go b/weed/command/filer_meta_tail.go
index f055b19a8..8451ffd78 100644
--- a/weed/command/filer_meta_tail.go
+++ b/weed/command/filer_meta_tail.go
@@ -23,9 +23,9 @@ func init() {
}
var cmdFilerMetaTail = &Command{
- UsageLine: "filer.meta.tail [-filer=localhost:8888] [-target=/]",
- Short: "see recent changes on a filer",
- Long: `See recent changes on a filer.
+ UsageLine: "filer.meta.tail [-filer=localhost:8888] [-pathPrefix=/]",
+ Short: "see continuous changes on a filer",
+ Long: `See continuous changes on a filer.
weed filer.meta.tail -timeAgo=30h | grep truncate
weed filer.meta.tail -timeAgo=30h | jq .
@@ -36,7 +36,7 @@ var cmdFilerMetaTail = &Command{
var (
tailFiler = cmdFilerMetaTail.Flag.String("filer", "localhost:8888", "filer hostname:port")
- tailTarget = cmdFilerMetaTail.Flag.String("pathPrefix", "/", "path to a folder or file, or common prefix for the folders or files on filer")
+ tailTarget = cmdFilerMetaTail.Flag.String("pathPrefix", "/", "path to a folder or common prefix for the folders or files on filer")
tailStart = cmdFilerMetaTail.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\"")
tailPattern = cmdFilerMetaTail.Flag.String("pattern", "", "full path or just filename pattern, ex: \"/home/?opher\", \"*.pdf\", see https://golang.org/pkg/path/filepath/#Match ")
esServers = cmdFilerMetaTail.Flag.String("es", "", "comma-separated elastic servers http://<host:port>")
diff --git a/weed/command/filer_replication.go b/weed/command/filer_replication.go
index e8c06b208..885c95540 100644
--- a/weed/command/filer_replication.go
+++ b/weed/command/filer_replication.go
@@ -74,18 +74,7 @@ func runFilerReplicate(cmd *Command, args []string) bool {
}
}
- var dataSink sink.ReplicationSink
- for _, sk := range sink.Sinks {
- if config.GetBool("sink." + sk.GetName() + ".enabled") {
- if err := sk.Initialize(config, "sink."+sk.GetName()+"."); err != nil {
- glog.Fatalf("Failed to initialize sink for %s: %+v",
- sk.GetName(), err)
- }
- glog.V(0).Infof("Configure sink to %s", sk.GetName())
- dataSink = sk
- break
- }
- }
+ dataSink := findSink(config)
if dataSink == nil {
println("no data sink configured in replication.toml:")
@@ -135,6 +124,22 @@ func runFilerReplicate(cmd *Command, args []string) bool {
}
+func findSink(config *util.ViperProxy) sink.ReplicationSink {
+ var dataSink sink.ReplicationSink
+ for _, sk := range sink.Sinks {
+ if config.GetBool("sink." + sk.GetName() + ".enabled") {
+ if err := sk.Initialize(config, "sink."+sk.GetName()+"."); err != nil {
+ glog.Fatalf("Failed to initialize sink for %s: %+v",
+ sk.GetName(), err)
+ }
+ glog.V(0).Infof("Configure sink to %s", sk.GetName())
+ dataSink = sk
+ break
+ }
+ }
+ return dataSink
+}
+
func validateOneEnabledInput(config *util.ViperProxy) {
enabledInput := ""
for _, input := range sub.NotificationInputs {
diff --git a/weed/command/filer_sync.go b/weed/command/filer_sync.go
index f8beb6fda..0f34e5701 100644
--- a/weed/command/filer_sync.go
+++ b/weed/command/filer_sync.go
@@ -8,6 +8,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/replication"
+ "github.com/chrislusf/seaweedfs/weed/replication/sink"
"github.com/chrislusf/seaweedfs/weed/replication/sink/filersink"
"github.com/chrislusf/seaweedfs/weed/replication/source"
"github.com/chrislusf/seaweedfs/weed/security"
@@ -137,7 +138,7 @@ func doSubscribeFilerMetaChanges(grpcDialOption grpc.DialOption, sourceFiler, so
// if first time, start from now
// if has previously synced, resume from that point of time
- sourceFilerOffsetTsNs, err := readSyncOffset(grpcDialOption, targetFiler, sourceFilerSignature)
+ sourceFilerOffsetTsNs, err := getOffset(grpcDialOption, targetFiler, SyncKeyPrefix, sourceFilerSignature)
if err != nil {
return err
}
@@ -151,93 +152,17 @@ func doSubscribeFilerMetaChanges(grpcDialOption grpc.DialOption, sourceFiler, so
filerSink.DoInitialize(targetFiler, pb.ServerToGrpcAddress(targetFiler), targetPath, replicationStr, collection, ttlSec, diskType, grpcDialOption, sinkWriteChunkByFiler)
filerSink.SetSourceFiler(filerSource)
+ persistEventFn := genProcessFunction(sourcePath, targetPath, filerSink, debug)
+
processEventFn := func(resp *filer_pb.SubscribeMetadataResponse) error {
message := resp.EventNotification
-
- var sourceOldKey, sourceNewKey util.FullPath
- if message.OldEntry != nil {
- sourceOldKey = util.FullPath(resp.Directory).Child(message.OldEntry.Name)
- }
- if message.NewEntry != nil {
- sourceNewKey = util.FullPath(message.NewParentPath).Child(message.NewEntry.Name)
- }
-
for _, sig := range message.Signatures {
if sig == targetFilerSignature && targetFilerSignature != 0 {
fmt.Printf("%s skipping %s change to %v\n", targetFiler, sourceFiler, message)
return nil
}
}
- if debug {
- fmt.Printf("%s check %s change %s,%s sig %v, target sig: %v\n", targetFiler, sourceFiler, sourceOldKey, sourceNewKey, message.Signatures, targetFilerSignature)
- }
-
- if !strings.HasPrefix(resp.Directory, sourcePath) {
- return nil
- }
-
- // handle deletions
- if message.OldEntry != nil && message.NewEntry == nil {
- if !strings.HasPrefix(string(sourceOldKey), sourcePath) {
- return nil
- }
- key := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
- return filerSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
- }
-
- // handle new entries
- if message.OldEntry == nil && message.NewEntry != nil {
- if !strings.HasPrefix(string(sourceNewKey), sourcePath) {
- return nil
- }
- key := util.Join(targetPath, string(sourceNewKey)[len(sourcePath):])
- return filerSink.CreateEntry(key, message.NewEntry, message.Signatures)
- }
-
- // this is something special?
- if message.OldEntry == nil && message.NewEntry == nil {
- return nil
- }
-
- // handle updates
- if strings.HasPrefix(string(sourceOldKey), sourcePath) {
- // old key is in the watched directory
- if strings.HasPrefix(string(sourceNewKey), sourcePath) {
- // new key is also in the watched directory
- oldKey := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
- message.NewParentPath = util.Join(targetPath, message.NewParentPath[len(sourcePath):])
- foundExisting, err := filerSink.UpdateEntry(string(oldKey), message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks, message.Signatures)
- if foundExisting {
- return err
- }
-
- // not able to find old entry
- if err = filerSink.DeleteEntry(string(oldKey), message.OldEntry.IsDirectory, false, message.Signatures); err != nil {
- return fmt.Errorf("delete old entry %v: %v", oldKey, err)
- }
-
- // create the new entry
- newKey := util.Join(targetPath, string(sourceNewKey)[len(sourcePath):])
- return filerSink.CreateEntry(newKey, message.NewEntry, message.Signatures)
-
- } else {
- // new key is outside of the watched directory
- key := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
- return filerSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
- }
- } else {
- // old key is outside of the watched directory
- if strings.HasPrefix(string(sourceNewKey), sourcePath) {
- // new key is in the watched directory
- key := util.Join(targetPath, string(sourceNewKey)[len(sourcePath):])
- return filerSink.CreateEntry(key, message.NewEntry, message.Signatures)
- } else {
- // new key is also outside of the watched directory
- // skip
- }
- }
-
- return nil
+ return persistEventFn(resp)
}
return pb.WithFilerClient(sourceFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
@@ -275,7 +200,7 @@ func doSubscribeFilerMetaChanges(grpcDialOption grpc.DialOption, sourceFiler, so
glog.V(0).Infof("sync %s => %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 := writeSyncOffset(grpcDialOption, targetFiler, sourceFilerSignature, resp.TsNs); err != nil {
+ if err := setOffset(grpcDialOption, targetFiler, SyncKeyPrefix, sourceFilerSignature, resp.TsNs); err != nil {
return err
}
}
@@ -290,11 +215,11 @@ const (
SyncKeyPrefix = "sync."
)
-func readSyncOffset(grpcDialOption grpc.DialOption, filer string, filerSignature int32) (lastOffsetTsNs int64, readErr error) {
+func getOffset(grpcDialOption grpc.DialOption, filer string, signaturePrefix string, signature int32) (lastOffsetTsNs int64, readErr error) {
readErr = pb.WithFilerClient(filer, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
- syncKey := []byte(SyncKeyPrefix + "____")
- util.Uint32toBytes(syncKey[len(SyncKeyPrefix):len(SyncKeyPrefix)+4], uint32(filerSignature))
+ syncKey := []byte(signaturePrefix + "____")
+ util.Uint32toBytes(syncKey[len(signaturePrefix):len(signaturePrefix)+4], uint32(signature))
resp, err := client.KvGet(context.Background(), &filer_pb.KvGetRequest{Key: syncKey})
if err != nil {
@@ -317,11 +242,11 @@ func readSyncOffset(grpcDialOption grpc.DialOption, filer string, filerSignature
}
-func writeSyncOffset(grpcDialOption grpc.DialOption, filer string, filerSignature int32, offsetTsNs int64) error {
+func setOffset(grpcDialOption grpc.DialOption, filer string, signaturePrefix string, signature int32, offsetTsNs int64) error {
return pb.WithFilerClient(filer, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
- syncKey := []byte(SyncKeyPrefix + "____")
- util.Uint32toBytes(syncKey[len(SyncKeyPrefix):len(SyncKeyPrefix)+4], uint32(filerSignature))
+ syncKey := []byte(signaturePrefix + "____")
+ util.Uint32toBytes(syncKey[len(signaturePrefix):len(signaturePrefix)+4], uint32(signature))
valueBuf := make([]byte, 8)
util.Uint64toBytes(valueBuf, uint64(offsetTsNs))
@@ -343,3 +268,107 @@ func writeSyncOffset(grpcDialOption grpc.DialOption, filer string, filerSignatur
})
}
+
+func genProcessFunction(sourcePath string, targetPath string, dataSink sink.ReplicationSink, debug bool) func(resp *filer_pb.SubscribeMetadataResponse) error {
+ // process function
+ processEventFn := func(resp *filer_pb.SubscribeMetadataResponse) error {
+ message := resp.EventNotification
+
+ var sourceOldKey, sourceNewKey util.FullPath
+ if message.OldEntry != nil {
+ sourceOldKey = util.FullPath(resp.Directory).Child(message.OldEntry.Name)
+ }
+ if message.NewEntry != nil {
+ sourceNewKey = util.FullPath(message.NewParentPath).Child(message.NewEntry.Name)
+ }
+
+ if debug {
+ glog.V(0).Infof("received %v", resp)
+ }
+
+ if !strings.HasPrefix(resp.Directory, sourcePath) {
+ return nil
+ }
+
+ // handle deletions
+ if message.OldEntry != nil && message.NewEntry == nil {
+ if !strings.HasPrefix(string(sourceOldKey), sourcePath) {
+ return nil
+ }
+ key := buildKey(dataSink, message, targetPath, sourceOldKey, sourcePath)
+ return dataSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
+ }
+
+ // handle new entries
+ if message.OldEntry == nil && message.NewEntry != nil {
+ if !strings.HasPrefix(string(sourceNewKey), sourcePath) {
+ return nil
+ }
+ key := buildKey(dataSink, message, targetPath, sourceNewKey, sourcePath)
+ return dataSink.CreateEntry(key, message.NewEntry, message.Signatures)
+ }
+
+ // this is something special?
+ if message.OldEntry == nil && message.NewEntry == nil {
+ return nil
+ }
+
+ // handle updates
+ if strings.HasPrefix(string(sourceOldKey), sourcePath) {
+ // old key is in the watched directory
+ if strings.HasPrefix(string(sourceNewKey), sourcePath) {
+ // new key is also in the watched directory
+ if !dataSink.IsIncremental() {
+ oldKey := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
+ message.NewParentPath = util.Join(targetPath, message.NewParentPath[len(sourcePath):])
+ foundExisting, err := dataSink.UpdateEntry(string(oldKey), message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks, message.Signatures)
+ if foundExisting {
+ return err
+ }
+
+ // not able to find old entry
+ if err = dataSink.DeleteEntry(string(oldKey), message.OldEntry.IsDirectory, false, message.Signatures); err != nil {
+ return fmt.Errorf("delete old entry %v: %v", oldKey, err)
+ }
+ }
+ // create the new entry
+ newKey := buildKey(dataSink, message, targetPath, sourceNewKey, sourcePath)
+ return dataSink.CreateEntry(newKey, message.NewEntry, message.Signatures)
+
+ } else {
+ // new key is outside of the watched directory
+ if !dataSink.IsIncremental() {
+ key := buildKey(dataSink, message, targetPath, sourceOldKey, sourcePath)
+ return dataSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
+ }
+ }
+ } else {
+ // old key is outside of the watched directory
+ if strings.HasPrefix(string(sourceNewKey), sourcePath) {
+ // new key is in the watched directory
+ key := buildKey(dataSink, message, targetPath, sourceNewKey, sourcePath)
+ return dataSink.CreateEntry(key, message.NewEntry, message.Signatures)
+ } else {
+ // new key is also outside of the watched directory
+ // skip
+ }
+ }
+
+ return nil
+ }
+ return processEventFn
+}
+
+func buildKey(dataSink sink.ReplicationSink, message *filer_pb.EventNotification, targetPath string, sourceKey util.FullPath, sourcePath string) string {
+ if !dataSink.IsIncremental() {
+ return util.Join(targetPath, string(sourceKey)[len(sourcePath):])
+ }
+ var mTime int64
+ if message.NewEntry != nil {
+ mTime = message.NewEntry.Attributes.Mtime
+ } else if message.OldEntry != nil {
+ mTime = message.OldEntry.Attributes.Mtime
+ }
+ dateKey := time.Unix(mTime, 0).Format("2006-01-02")
+ return util.Join(targetPath, dateKey, string(sourceKey)[len(sourcePath):])
+}
diff --git a/weed/command/master.go b/weed/command/master.go
index d569919cd..0f5e2156d 100644
--- a/weed/command/master.go
+++ b/weed/command/master.go
@@ -6,7 +6,6 @@ import (
"google.golang.org/grpc/reflection"
"net/http"
"os"
- "runtime"
"sort"
"strconv"
"strings"
@@ -48,8 +47,8 @@ type MasterOptions struct {
func init() {
cmdMaster.Run = runMaster // break init cycle
m.port = cmdMaster.Flag.Int("port", 9333, "http listen port")
- m.ip = cmdMaster.Flag.String("ip", util.DetectedHostAddress(), "master <ip>|<server> address")
- m.ipBind = cmdMaster.Flag.String("ip.bind", "0.0.0.0", "ip address to bind to")
+ m.ip = cmdMaster.Flag.String("ip", util.DetectedHostAddress(), "master <ip>|<server> address, also used as identifier")
+ m.ipBind = cmdMaster.Flag.String("ip.bind", "", "ip address to bind to")
m.metaFolder = cmdMaster.Flag.String("mdir", os.TempDir(), "data directory to store meta data")
m.peers = cmdMaster.Flag.String("peers", "", "all master nodes in comma separated ip:port list, example: 127.0.0.1:9093,127.0.0.1:9094,127.0.0.1:9095")
m.volumeSizeLimitMB = cmdMaster.Flag.Uint("volumeSizeLimitMB", 30*1000, "Master stops directing writes to oversized volumes.")
@@ -86,7 +85,6 @@ func runMaster(cmd *Command, args []string) bool {
util.LoadConfiguration("security", false)
util.LoadConfiguration("master", false)
- runtime.GOMAXPROCS(runtime.NumCPU())
grace.SetupProfiling(*masterCpuProfile, *masterMemProfile)
parent, _ := util.FullPath(*m.metaFolder).DirAndName()
@@ -138,7 +136,6 @@ func startMaster(masterOption MasterOptions, masterWhiteList []string) {
if err != nil {
glog.Fatalf("master failed to listen on grpc port %d: %v", grpcPort, err)
}
- // Create your protocol servers.
grpcS := pb.NewGrpcServer(security.LoadServerTLS(util.GetViper(), "grpc.master"))
master_pb.RegisterSeaweedServer(grpcS, ms)
protobuf.RegisterRaftServer(grpcS, raftServer)
diff --git a/weed/command/mount.go b/weed/command/mount.go
index ea439af7c..5811f0b99 100644
--- a/weed/command/mount.go
+++ b/weed/command/mount.go
@@ -25,6 +25,7 @@ type MountOptions struct {
volumeServerAccess *string
uidMap *string
gidMap *string
+ readOnly *bool
}
var (
@@ -45,7 +46,7 @@ func init() {
mountOptions.diskType = cmdMount.Flag.String("disk", "", "[hdd|ssd|<tag>] hard drive or solid state drive or any tag")
mountOptions.ttlSec = cmdMount.Flag.Int("ttl", 0, "file ttl in seconds")
mountOptions.chunkSizeLimitMB = cmdMount.Flag.Int("chunkSizeLimitMB", 2, "local write buffer size, also chunk large files")
- mountOptions.concurrentWriters = cmdMount.Flag.Int("concurrentWriters", 128, "limit concurrent goroutine writers if not 0")
+ mountOptions.concurrentWriters = cmdMount.Flag.Int("concurrentWriters", 32, "limit concurrent goroutine writers if not 0")
mountOptions.cacheDir = cmdMount.Flag.String("cacheDir", os.TempDir(), "local cache directory for file chunks and meta data")
mountOptions.cacheSizeMB = cmdMount.Flag.Int64("cacheCapacityMB", 1000, "local file chunk cache capacity in MB (0 will disable cache)")
mountOptions.dataCenter = cmdMount.Flag.String("dataCenter", "", "prefer to write to the data center")
@@ -55,6 +56,7 @@ func init() {
mountOptions.volumeServerAccess = cmdMount.Flag.String("volumeServerAccess", "direct", "access volume servers by [direct|publicUrl|filerProxy]")
mountOptions.uidMap = cmdMount.Flag.String("map.uid", "", "map local uid to uid on filer, comma-separated <local_uid>:<filer_uid>")
mountOptions.gidMap = cmdMount.Flag.String("map.gid", "", "map local gid to gid on filer, comma-separated <local_gid>:<filer_gid>")
+ mountOptions.readOnly = cmdMount.Flag.Bool("readOnly", false, "read only")
mountCpuProfile = cmdMount.Flag.String("cpuprofile", "", "cpu profile output file")
mountMemProfile = cmdMount.Flag.String("memprofile", "", "memory profile output file")
diff --git a/weed/command/mount_std.go b/weed/command/mount_std.go
index a6d562d40..8e5b7a483 100644
--- a/weed/command/mount_std.go
+++ b/weed/command/mount_std.go
@@ -53,7 +53,7 @@ func RunMount(option *MountOptions, umask os.FileMode) bool {
filer := *option.filer
// parse filer grpc address
- filerGrpcAddress, err := pb.ParseFilerGrpcAddress(filer)
+ filerGrpcAddress, err := pb.ParseServerToGrpcAddress(filer)
if err != nil {
glog.V(0).Infof("ParseFilerGrpcAddress: %v", err)
return true
@@ -63,16 +63,23 @@ func RunMount(option *MountOptions, umask os.FileMode) bool {
// try to connect to filer, filerBucketsPath may be useful later
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client")
var cipher bool
- err = pb.WithGrpcFilerClient(filerGrpcAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
- resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
+ for i := 0; i < 10; i++ {
+ err = pb.WithGrpcFilerClient(filerGrpcAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
+ resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
+ if err != nil {
+ return fmt.Errorf("get filer grpc address %s configuration: %v", filerGrpcAddress, err)
+ }
+ cipher = resp.Cipher
+ return nil
+ })
if err != nil {
- return fmt.Errorf("get filer grpc address %s configuration: %v", filerGrpcAddress, err)
+ glog.V(0).Infof("failed to talk to filer %s: %v", filerGrpcAddress, err)
+ glog.V(0).Infof("wait for %d seconds ...", i+1)
+ time.Sleep(time.Duration(i+1) * time.Second)
}
- cipher = resp.Cipher
- return nil
- })
+ }
if err != nil {
- glog.Infof("failed to talk to filer %s: %v", filerGrpcAddress, err)
+ glog.Errorf("failed to talk to filer %s: %v", filerGrpcAddress, err)
return true
}
@@ -196,6 +203,7 @@ func RunMount(option *MountOptions, umask os.FileMode) bool {
VolumeServerAccess: *mountOptions.volumeServerAccess,
Cipher: cipher,
UidGidMapper: uidGidMapper,
+ ReadOnly: *option.readOnly,
})
// mount
diff --git a/weed/command/msg_broker.go b/weed/command/msg_broker.go
index b4b5855ff..db0b4148d 100644
--- a/weed/command/msg_broker.go
+++ b/weed/command/msg_broker.go
@@ -63,7 +63,7 @@ func (msgBrokerOpt *MessageBrokerOptions) startQueueServer() bool {
grace.SetupProfiling(*messageBrokerStandaloneOptions.cpuprofile, *messageBrokerStandaloneOptions.memprofile)
- filerGrpcAddress, err := pb.ParseFilerGrpcAddress(*msgBrokerOpt.filer)
+ filerGrpcAddress, err := pb.ParseServerToGrpcAddress(*msgBrokerOpt.filer)
if err != nil {
glog.Fatal(err)
return false
diff --git a/weed/command/s3.go b/weed/command/s3.go
index d8e3e306b..c8292a7d5 100644
--- a/weed/command/s3.go
+++ b/weed/command/s3.go
@@ -137,7 +137,7 @@ func runS3(cmd *Command, args []string) bool {
func (s3opt *S3Options) startS3Server() bool {
- filerGrpcAddress, err := pb.ParseFilerGrpcAddress(*s3opt.filer)
+ filerGrpcAddress, err := pb.ParseServerToGrpcAddress(*s3opt.filer)
if err != nil {
glog.Fatal(err)
return false
diff --git a/weed/command/scaffold.go b/weed/command/scaffold.go
index 993391a42..4794da18e 100644
--- a/weed/command/scaffold.go
+++ b/weed/command/scaffold.go
@@ -103,9 +103,9 @@ dir = "./filerrdb" # directory to store rocksdb files
[mysql] # or memsql, tidb
# CREATE TABLE IF NOT EXISTS filemeta (
-# dirhash BIGINT COMMENT 'first 64 bits of MD5 hash value of directory field',
-# name VARCHAR(1000) COMMENT 'directory or file name',
-# directory TEXT COMMENT 'full path to parent directory',
+# dirhash BIGINT COMMENT 'first 64 bits of MD5 hash value of directory field',
+# name VARCHAR(1000) BINARY COMMENT 'directory or file name',
+# directory TEXT COMMENT 'full path to parent directory',
# meta LONGBLOB,
# PRIMARY KEY (dirhash, name)
# ) DEFAULT CHARSET=utf8;
@@ -120,13 +120,16 @@ connection_max_idle = 2
connection_max_open = 100
connection_max_lifetime_seconds = 0
interpolateParams = false
+# if insert/upsert failing, you can disable upsert or update query syntax to match your RDBMS syntax:
+enableUpsert = true
+upsertQuery = """INSERT INTO ` + "`%s`" + ` (dirhash,name,directory,meta) VALUES(?,?,?,?) ON DUPLICATE KEY UPDATE meta = VALUES(meta)"""
[mysql2] # or memsql, tidb
enabled = false
createTable = """
CREATE TABLE IF NOT EXISTS ` + "`%s`" + ` (
dirhash BIGINT,
- name VARCHAR(1000),
+ name VARCHAR(1000) BINARY,
directory TEXT,
meta LONGBLOB,
PRIMARY KEY (dirhash, name)
@@ -141,6 +144,9 @@ connection_max_idle = 2
connection_max_open = 100
connection_max_lifetime_seconds = 0
interpolateParams = false
+# if insert/upsert failing, you can disable upsert or update query syntax to match your RDBMS syntax:
+enableUpsert = true
+upsertQuery = """INSERT INTO ` + "`%s`" + ` (dirhash,name,directory,meta) VALUES(?,?,?,?) ON DUPLICATE KEY UPDATE meta = VALUES(meta)"""
[postgres] # or cockroachdb, YugabyteDB
# CREATE TABLE IF NOT EXISTS filemeta (
@@ -161,6 +167,9 @@ sslmode = "disable"
connection_max_idle = 100
connection_max_open = 100
connection_max_lifetime_seconds = 0
+# if insert/upsert failing, you can disable upsert or update query syntax to match your RDBMS syntax:
+enableUpsert = true
+upsertQuery = """INSERT INTO "%[1]s" (dirhash,name,directory,meta) VALUES($1,$2,$3,$4) ON CONFLICT (dirhash,name) DO UPDATE SET meta = EXCLUDED.meta WHERE "%[1]s".meta != EXCLUDED.meta"""
[postgres2]
enabled = false
@@ -183,6 +192,9 @@ sslmode = "disable"
connection_max_idle = 100
connection_max_open = 100
connection_max_lifetime_seconds = 0
+# if insert/upsert failing, you can disable upsert or update query syntax to match your RDBMS syntax:
+enableUpsert = true
+upsertQuery = """INSERT INTO "%[1]s" (dirhash,name,directory,meta) VALUES($1,$2,$3,$4) ON CONFLICT (dirhash,name) DO UPDATE SET meta = EXCLUDED.meta WHERE "%[1]s".meta != EXCLUDED.meta"""
[cassandra]
# CREATE TABLE filemeta (
@@ -356,6 +368,9 @@ directory = "/buckets"
[sink.local]
enabled = false
directory = "/data"
+# all replicated files are under modified time as yyyy-mm-dd directories
+# so each date directory contains all new and updated files.
+is_incremental = false
[sink.local_incremental]
# all replicated files are under modified time as yyyy-mm-dd directories
@@ -373,6 +388,7 @@ directory = "/backup"
replication = ""
collection = ""
ttlSec = 0
+is_incremental = false
[sink.s3]
# read credentials doc at https://docs.aws.amazon.com/sdk-for-go/v1/developer-guide/sessions.html
@@ -384,6 +400,7 @@ region = "us-east-2"
bucket = "your_bucket_name" # an existing bucket
directory = "/" # destination directory
endpoint = ""
+is_incremental = false
[sink.google_cloud_storage]
# read credentials doc at https://cloud.google.com/docs/authentication/getting-started
@@ -391,6 +408,7 @@ enabled = false
google_application_credentials = "/path/to/x.json" # path to json credential file
bucket = "your_bucket_seaweedfs" # an existing bucket
directory = "/" # destination directory
+is_incremental = false
[sink.azure]
# experimental, let me know if it works
@@ -399,6 +417,7 @@ account_name = ""
account_key = ""
container = "mycontainer" # an existing container
directory = "/" # destination directory
+is_incremental = false
[sink.backblaze]
enabled = false
@@ -406,6 +425,7 @@ b2_account_id = ""
b2_master_application_key = ""
bucket = "mybucket" # an existing bucket
directory = "/" # destination directory
+is_incremental = false
`
@@ -432,22 +452,28 @@ expires_after_seconds = 10 # seconds
# the host name is not checked, so the PERM files can be shared.
[grpc]
ca = ""
+# Set wildcard domain for enable TLS authentication by common names
+allowed_wildcard_domain = "" # .mycompany.com
[grpc.volume]
cert = ""
key = ""
+allowed_commonNames = "" # comma-separated SSL certificate common names
[grpc.master]
cert = ""
key = ""
+allowed_commonNames = "" # comma-separated SSL certificate common names
[grpc.filer]
cert = ""
key = ""
+allowed_commonNames = "" # comma-separated SSL certificate common names
[grpc.msg_broker]
cert = ""
key = ""
+allowed_commonNames = "" # comma-separated SSL certificate common names
# use this for any place needs a grpc client
# i.e., "weed backup|benchmark|filer.copy|filer.replicate|mount|s3|upload"
@@ -455,7 +481,6 @@ key = ""
cert = ""
key = ""
-
# volume server https options
# Note: work in progress!
# this does not work with other clients, e.g., "weed filer|mount" etc, yet.
@@ -493,7 +518,7 @@ default = "localhost:8888" # used by maintenance scripts if the scripts needs
[master.sequencer]
-type = "raft" # Choose [raft|etcd] type for storing the file id sequence
+type = "raft" # Choose [raft|etcd|snowflake] type for storing the file id sequence
# when sequencer.type = etcd, set listen client urls of etcd cluster that store file id sequence
# example : http://127.0.0.1:2379,http://127.0.0.1:2389
sequencer_etcd_urls = "http://127.0.0.1:2379"
diff --git a/weed/command/server.go b/weed/command/server.go
index 611578953..6eb3bf97c 100644
--- a/weed/command/server.go
+++ b/weed/command/server.go
@@ -2,9 +2,8 @@ package command
import (
"fmt"
+ "github.com/chrislusf/seaweedfs/weed/util/grace"
"os"
- "runtime"
- "runtime/pprof"
"strings"
"time"
@@ -16,6 +15,7 @@ import (
type ServerOptions struct {
cpuprofile *string
+ memprofile *string
v VolumeServerOptions
}
@@ -49,8 +49,8 @@ var cmdServer = &Command{
}
var (
- serverIp = cmdServer.Flag.String("ip", util.DetectedHostAddress(), "ip or server name")
- serverBindIp = cmdServer.Flag.String("ip.bind", "0.0.0.0", "ip address to bind to")
+ serverIp = cmdServer.Flag.String("ip", util.DetectedHostAddress(), "ip or server name, also used as identifier")
+ serverBindIp = cmdServer.Flag.String("ip.bind", "", "ip address to bind to")
serverTimeout = cmdServer.Flag.Int("idleTimeout", 30, "connection idle seconds")
serverDataCenter = cmdServer.Flag.String("dataCenter", "", "current volume server's data center name")
serverRack = cmdServer.Flag.String("rack", "", "current volume server's rack name")
@@ -76,6 +76,7 @@ var (
func init() {
serverOptions.cpuprofile = cmdServer.Flag.String("cpuprofile", "", "cpu profile output file")
+ serverOptions.memprofile = cmdServer.Flag.String("memprofile", "", "memory profile output file")
masterOptions.port = cmdServer.Flag.Int("master.port", 9333, "master server http listen port")
masterOptions.metaFolder = cmdServer.Flag.String("master.dir", "", "data directory to store meta data, default to same as -dir specified")
@@ -93,11 +94,12 @@ func init() {
filerOptions.publicPort = cmdServer.Flag.Int("filer.port.public", 0, "filer server public http listen port")
filerOptions.defaultReplicaPlacement = cmdServer.Flag.String("filer.defaultReplicaPlacement", "", "default replication type. If not specified, use master setting.")
filerOptions.disableDirListing = cmdServer.Flag.Bool("filer.disableDirListing", false, "turn off directory listing")
- filerOptions.maxMB = cmdServer.Flag.Int("filer.maxMB", 32, "split files larger than the limit")
+ filerOptions.maxMB = cmdServer.Flag.Int("filer.maxMB", 4, "split files larger than the limit")
filerOptions.dirListingLimit = cmdServer.Flag.Int("filer.dirListLimit", 1000, "limit sub dir listing size")
filerOptions.cipher = cmdServer.Flag.Bool("filer.encryptVolumeData", false, "encrypt data on volume servers")
filerOptions.peers = cmdServer.Flag.String("filer.peers", "", "all filers sharing the same filer store in comma separated ip:port list")
filerOptions.saveToFilerLimit = cmdServer.Flag.Int("filer.saveToFilerLimit", 0, "Small files smaller than this limit can be cached in filer store.")
+ filerOptions.concurrentUploadLimitMB = cmdServer.Flag.Int("filer.concurrentUploadLimitMB", 64, "limit total concurrent upload size")
serverOptions.v.port = cmdServer.Flag.Int("volume.port", 8080, "volume server http listen port")
serverOptions.v.publicPort = cmdServer.Flag.Int("volume.port.public", 0, "volume server public port")
@@ -107,10 +109,12 @@ func init() {
serverOptions.v.readRedirect = cmdServer.Flag.Bool("volume.read.redirect", true, "Redirect moved or non-local volumes.")
serverOptions.v.compactionMBPerSecond = cmdServer.Flag.Int("volume.compactionMBps", 0, "limit compaction speed in mega bytes per second")
serverOptions.v.fileSizeLimitMB = cmdServer.Flag.Int("volume.fileSizeLimitMB", 256, "limit file size to avoid out of memory")
+ serverOptions.v.concurrentUploadLimitMB = cmdServer.Flag.Int("volume.concurrentUploadLimitMB", 64, "limit total concurrent upload size")
serverOptions.v.publicUrl = cmdServer.Flag.String("volume.publicUrl", "", "publicly accessible address")
serverOptions.v.preStopSeconds = cmdServer.Flag.Int("volume.preStopSeconds", 10, "number of seconds between stop send heartbeats and stop volume server")
serverOptions.v.pprof = cmdServer.Flag.Bool("volume.pprof", false, "enable pprof http handlers. precludes --memprofile and --cpuprofile")
serverOptions.v.idxFolder = cmdServer.Flag.String("volume.dir.idx", "", "directory to store .idx files")
+ serverOptions.v.enableTcp = cmdServer.Flag.Bool("volume.tcp", false, "<exprimental> enable tcp port")
s3Options.port = cmdServer.Flag.Int("s3.port", 8333, "s3 server http listen port")
s3Options.domainName = cmdServer.Flag.String("s3.domainName", "", "suffix of the host name in comma separated list, {bucket}.{domainName}")
@@ -137,14 +141,7 @@ func runServer(cmd *Command, args []string) bool {
util.LoadConfiguration("security", false)
util.LoadConfiguration("master", false)
- if *serverOptions.cpuprofile != "" {
- f, err := os.Create(*serverOptions.cpuprofile)
- if err != nil {
- glog.Fatal(err)
- }
- pprof.StartCPUProfile(f)
- defer pprof.StopCPUProfile()
- }
+ grace.SetupProfiling(*serverOptions.cpuprofile, *serverOptions.memprofile)
if *isStartingS3 {
*isStartingFiler = true
@@ -156,19 +153,21 @@ func runServer(cmd *Command, args []string) bool {
*isStartingFiler = true
}
- _, peerList := checkPeers(*serverIp, *masterOptions.port, *masterOptions.peers)
- peers := strings.Join(peerList, ",")
- masterOptions.peers = &peers
+ if *isStartingMasterServer {
+ _, peerList := checkPeers(*serverIp, *masterOptions.port, *masterOptions.peers)
+ peers := strings.Join(peerList, ",")
+ masterOptions.peers = &peers
+ }
// ip address
masterOptions.ip = serverIp
masterOptions.ipBind = serverBindIp
- filerOptions.masters = &peers
+ filerOptions.masters = masterOptions.peers
filerOptions.ip = serverIp
filerOptions.bindIp = serverBindIp
serverOptions.v.ip = serverIp
serverOptions.v.bindIp = serverBindIp
- serverOptions.v.masters = &peers
+ serverOptions.v.masters = masterOptions.peers
serverOptions.v.idleConnectionTimeout = serverTimeout
serverOptions.v.dataCenter = serverDataCenter
serverOptions.v.rack = serverRack
@@ -189,7 +188,6 @@ func runServer(cmd *Command, args []string) bool {
webdavOptions.filer = &filerAddress
msgBrokerOptions.filer = &filerAddress
- runtime.GOMAXPROCS(runtime.NumCPU())
go stats_collect.StartMetricsServer(*serverMetricsHttpPort)
folders := strings.Split(*volumeDataFolders, ",")
diff --git a/weed/command/upload.go b/weed/command/upload.go
index 67fde2185..0f9361b40 100644
--- a/weed/command/upload.go
+++ b/weed/command/upload.go
@@ -43,7 +43,7 @@ func init() {
upload.dataCenter = cmdUpload.Flag.String("dataCenter", "", "optional data center name")
upload.diskType = cmdUpload.Flag.String("disk", "", "[hdd|ssd|<tag>] hard drive or solid state drive or any tag")
upload.ttl = cmdUpload.Flag.String("ttl", "", "time to live, e.g.: 1m, 1h, 1d, 1M, 1y")
- upload.maxMB = cmdUpload.Flag.Int("maxMB", 32, "split files larger than the limit")
+ upload.maxMB = cmdUpload.Flag.Int("maxMB", 4, "split files larger than the limit")
upload.usePublicUrl = cmdUpload.Flag.Bool("usePublicUrl", false, "upload to public url from volume server")
}
diff --git a/weed/command/volume.go b/weed/command/volume.go
index 659c93d96..9df500178 100644
--- a/weed/command/volume.go
+++ b/weed/command/volume.go
@@ -6,7 +6,6 @@ import (
"net/http"
httppprof "net/http/pprof"
"os"
- "runtime"
"runtime/pprof"
"strconv"
"strings"
@@ -36,41 +35,43 @@ var (
)
type VolumeServerOptions struct {
- port *int
- publicPort *int
- folders []string
- folderMaxLimits []int
- idxFolder *string
- ip *string
- publicUrl *string
- bindIp *string
- masters *string
- idleConnectionTimeout *int
- dataCenter *string
- rack *string
- whiteList []string
- indexType *string
- diskType *string
- fixJpgOrientation *bool
- readRedirect *bool
- cpuProfile *string
- memProfile *string
- compactionMBPerSecond *int
- fileSizeLimitMB *int
- minFreeSpacePercents []float32
- pprof *bool
- preStopSeconds *int
- metricsHttpPort *int
+ port *int
+ publicPort *int
+ folders []string
+ folderMaxLimits []int
+ idxFolder *string
+ ip *string
+ publicUrl *string
+ bindIp *string
+ masters *string
+ idleConnectionTimeout *int
+ dataCenter *string
+ rack *string
+ whiteList []string
+ indexType *string
+ diskType *string
+ fixJpgOrientation *bool
+ readRedirect *bool
+ cpuProfile *string
+ memProfile *string
+ compactionMBPerSecond *int
+ fileSizeLimitMB *int
+ concurrentUploadLimitMB *int
+ minFreeSpacePercents []float32
+ pprof *bool
+ preStopSeconds *int
+ metricsHttpPort *int
// pulseSeconds *int
+ enableTcp *bool
}
func init() {
cmdVolume.Run = runVolume // break init cycle
v.port = cmdVolume.Flag.Int("port", 8080, "http listen port")
v.publicPort = cmdVolume.Flag.Int("port.public", 0, "port opened to public")
- v.ip = cmdVolume.Flag.String("ip", util.DetectedHostAddress(), "ip or server name")
+ v.ip = cmdVolume.Flag.String("ip", util.DetectedHostAddress(), "ip or server name, also used as identifier")
v.publicUrl = cmdVolume.Flag.String("publicUrl", "", "Publicly accessible address")
- v.bindIp = cmdVolume.Flag.String("ip.bind", "0.0.0.0", "ip address to bind to")
+ v.bindIp = cmdVolume.Flag.String("ip.bind", "", "ip address to bind to")
v.masters = cmdVolume.Flag.String("mserver", "localhost:9333", "comma-separated master servers")
v.preStopSeconds = cmdVolume.Flag.Int("preStopSeconds", 10, "number of seconds between stop send heartbeats and stop volume server")
// v.pulseSeconds = cmdVolume.Flag.Int("pulseSeconds", 5, "number of seconds between heartbeats, must be smaller than or equal to the master's setting")
@@ -85,9 +86,11 @@ func init() {
v.memProfile = cmdVolume.Flag.String("memprofile", "", "memory profile output file")
v.compactionMBPerSecond = cmdVolume.Flag.Int("compactionMBps", 0, "limit background compaction or copying speed in mega bytes per second")
v.fileSizeLimitMB = cmdVolume.Flag.Int("fileSizeLimitMB", 256, "limit file size to avoid out of memory")
+ v.concurrentUploadLimitMB = cmdVolume.Flag.Int("concurrentUploadLimitMB", 128, "limit total concurrent upload size")
v.pprof = cmdVolume.Flag.Bool("pprof", false, "enable pprof http handlers. precludes --memprofile and --cpuprofile")
v.metricsHttpPort = cmdVolume.Flag.Int("metricsPort", 0, "Prometheus metrics listen port")
v.idxFolder = cmdVolume.Flag.String("dir.idx", "", "directory to store .idx files")
+ v.enableTcp = cmdVolume.Flag.Bool("tcp", false, "<exprimental> enable tcp port")
}
var cmdVolume = &Command{
@@ -109,8 +112,6 @@ func runVolume(cmd *Command, args []string) bool {
util.LoadConfiguration("security", false)
- runtime.GOMAXPROCS(runtime.NumCPU())
-
// If --pprof is set we assume the caller wants to be able to collect
// cpu and memory profiles via go tool pprof
if !*v.pprof {
@@ -238,6 +239,7 @@ func (v VolumeServerOptions) startVolumeServer(volumeFolders, maxVolumeCounts, v
*v.fixJpgOrientation, *v.readRedirect,
*v.compactionMBPerSecond,
*v.fileSizeLimitMB,
+ int64(*v.concurrentUploadLimitMB)*1024*1024,
)
// starting grpc server
grpcS := v.startGrpcService(volumeServer)
@@ -251,6 +253,11 @@ func (v VolumeServerOptions) startVolumeServer(volumeFolders, maxVolumeCounts, v
}
}
+ // starting tcp server
+ if *v.enableTcp {
+ go v.startTcpService(volumeServer)
+ }
+
// starting the cluster http server
clusterHttpServer := v.startClusterHttpService(volumeMux)
@@ -368,3 +375,22 @@ func (v VolumeServerOptions) startClusterHttpService(handler http.Handler) httpd
}()
return clusterHttpServer
}
+
+func (v VolumeServerOptions) startTcpService(volumeServer *weed_server.VolumeServer) {
+ listeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.port+20000)
+ glog.V(0).Infoln("Start Seaweed volume server", util.Version(), "tcp at", listeningAddress)
+ listener, e := util.NewListener(listeningAddress, 0)
+ if e != nil {
+ glog.Fatalf("Volume server listener error on %s:%v", listeningAddress, e)
+ }
+ defer listener.Close()
+
+ for {
+ c, err := listener.Accept()
+ if err != nil {
+ fmt.Println(err)
+ return
+ }
+ go volumeServer.HandleTcpConnection(c)
+ }
+}
diff --git a/weed/command/webdav.go b/weed/command/webdav.go
index 2bd4a3c61..781ea1e36 100644
--- a/weed/command/webdav.go
+++ b/weed/command/webdav.go
@@ -78,7 +78,7 @@ func (wo *WebDavOption) startWebDav() bool {
}
// parse filer grpc address
- filerGrpcAddress, err := pb.ParseFilerGrpcAddress(*wo.filer)
+ filerGrpcAddress, err := pb.ParseServerToGrpcAddress(*wo.filer)
if err != nil {
glog.Fatal(err)
return false
diff --git a/weed/filer/abstract_sql/abstract_sql_store.go b/weed/filer/abstract_sql/abstract_sql_store.go
index 07ce56145..120a1d139 100644
--- a/weed/filer/abstract_sql/abstract_sql_store.go
+++ b/weed/filer/abstract_sql/abstract_sql_store.go
@@ -13,15 +13,15 @@ import (
)
type SqlGenerator interface {
- GetSqlInsert(bucket string) string
- GetSqlUpdate(bucket string) string
- GetSqlFind(bucket string) string
- GetSqlDelete(bucket string) string
- GetSqlDeleteFolderChildren(bucket string) string
- GetSqlListExclusive(bucket string) string
- GetSqlListInclusive(bucket string) string
- GetSqlCreateTable(bucket string) string
- GetSqlDropTable(bucket string) string
+ GetSqlInsert(tableName string) string
+ GetSqlUpdate(tableName string) string
+ GetSqlFind(tableName string) string
+ GetSqlDelete(tableName string) string
+ GetSqlDeleteFolderChildren(tableName string) string
+ GetSqlListExclusive(tableName string) string
+ GetSqlListInclusive(tableName string) string
+ GetSqlCreateTable(tableName string) string
+ GetSqlDropTable(tableName string) string
}
type AbstractSqlStore struct {
@@ -32,6 +32,29 @@ type AbstractSqlStore struct {
dbsLock sync.Mutex
}
+func (store *AbstractSqlStore) OnBucketCreation(bucket string) {
+ store.dbsLock.Lock()
+ defer store.dbsLock.Unlock()
+
+ store.CreateTable(context.Background(), bucket)
+
+ if store.dbs == nil {
+ return
+ }
+ store.dbs[bucket] = true
+}
+func (store *AbstractSqlStore) OnBucketDeletion(bucket string) {
+ store.dbsLock.Lock()
+ defer store.dbsLock.Unlock()
+
+ store.deleteTable(context.Background(), bucket)
+
+ if store.dbs == nil {
+ return
+ }
+ delete(store.dbs, bucket)
+}
+
const (
DEFAULT_TABLE = "filemeta"
)
diff --git a/weed/filer/filechunk_manifest.go b/weed/filer/filechunk_manifest.go
index 99a62c90c..c709dc819 100644
--- a/weed/filer/filechunk_manifest.go
+++ b/weed/filer/filechunk_manifest.go
@@ -97,20 +97,20 @@ func fetchChunk(lookupFileIdFn wdclient.LookupFileIdFunctionType, fileId string,
func retriedFetchChunkData(urlStrings []string, cipherKey []byte, isGzipped bool, isFullChunk bool, offset int64, size int) ([]byte, error) {
var err error
- var buffer bytes.Buffer
var shouldRetry bool
+ receivedData := make([]byte, 0, size)
for waitTime := time.Second; waitTime < util.RetryWaitTime; waitTime += waitTime / 2 {
for _, urlString := range urlStrings {
- shouldRetry, err = util.FastReadUrlAsStream(urlString+"?readDeleted=true", cipherKey, isGzipped, isFullChunk, offset, size, func(data []byte) {
- buffer.Write(data)
+ receivedData = receivedData[:0]
+ shouldRetry, err = util.ReadUrlAsStream(urlString+"?readDeleted=true", cipherKey, isGzipped, isFullChunk, offset, size, func(data []byte) {
+ receivedData = append(receivedData, data...)
})
if !shouldRetry {
break
}
if err != nil {
glog.V(0).Infof("read %s failed, err: %v", urlString, err)
- buffer.Reset()
} else {
break
}
@@ -123,7 +123,8 @@ func retriedFetchChunkData(urlStrings []string, cipherKey []byte, isGzipped bool
}
}
- return buffer.Bytes(), err
+ return receivedData, err
+
}
func MaybeManifestize(saveFunc SaveDataAsChunkFunctionType, inputChunks []*filer_pb.FileChunk) (chunks []*filer_pb.FileChunk, err error) {
diff --git a/weed/filer/filer.go b/weed/filer/filer.go
index e59887763..effdc0e4e 100644
--- a/weed/filer/filer.go
+++ b/weed/filer/filer.go
@@ -241,12 +241,12 @@ func (f *Filer) UpdateEntry(ctx context.Context, oldEntry, entry *Entry) (err er
if oldEntry != nil {
entry.Attr.Crtime = oldEntry.Attr.Crtime
if oldEntry.IsDirectory() && !entry.IsDirectory() {
- glog.Errorf("existing %s is a directory", entry.FullPath)
- return fmt.Errorf("existing %s is a directory", entry.FullPath)
+ glog.Errorf("existing %s is a directory", oldEntry.FullPath)
+ return fmt.Errorf("existing %s is a directory", oldEntry.FullPath)
}
if !oldEntry.IsDirectory() && entry.IsDirectory() {
- glog.Errorf("existing %s is a file", entry.FullPath)
- return fmt.Errorf("existing %s is a file", entry.FullPath)
+ glog.Errorf("existing %s is a file", oldEntry.FullPath)
+ return fmt.Errorf("existing %s is a file", oldEntry.FullPath)
}
}
return f.Store.UpdateEntry(ctx, entry)
diff --git a/weed/filer/filer_delete_entry.go b/weed/filer/filer_delete_entry.go
index 50a669f40..bedf2f4d1 100644
--- a/weed/filer/filer_delete_entry.go
+++ b/weed/filer/filer_delete_entry.go
@@ -11,6 +11,10 @@ import (
type HardLinkId []byte
+const (
+ MsgFailDelNonEmptyFolder = "fail to delete non-empty folder"
+)
+
func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p util.FullPath, isRecursive, ignoreRecursiveError, shouldDeleteChunks, isFromOtherCluster bool, signatures []int32) (err error) {
if p == "/" {
return nil
@@ -77,7 +81,7 @@ func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry
if lastFileName == "" && !isRecursive && len(entries) > 0 {
// only for first iteration in the loop
glog.Errorf("deleting a folder %s has children: %+v ...", entry.FullPath, entries[0].Name())
- return nil, nil, fmt.Errorf("fail to delete non-empty folder: %s", entry.FullPath)
+ return nil, nil, fmt.Errorf("%s: %s", MsgFailDelNonEmptyFolder, entry.FullPath)
}
for _, sub := range entries {
diff --git a/weed/filer/filer_on_meta_event.go b/weed/filer/filer_on_meta_event.go
index 295a5039e..a91faeb24 100644
--- a/weed/filer/filer_on_meta_event.go
+++ b/weed/filer/filer_on_meta_event.go
@@ -11,6 +11,28 @@ 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.onBucketEvents(event)
+}
+
+func (f *Filer) onBucketEvents(event *filer_pb.SubscribeMetadataResponse) {
+ message := event.EventNotification
+ for _, sig := range message.Signatures {
+ if sig == f.Signature {
+ return
+ }
+ }
+ if f.DirBucketsPath == event.Directory {
+ if message.OldEntry == nil && message.NewEntry != nil {
+ f.Store.OnBucketCreation(message.NewEntry.Name)
+ }
+ if message.OldEntry != nil && message.NewEntry == nil {
+ f.Store.OnBucketDeletion(message.OldEntry.Name)
+ }
+ }
+}
+
+func (f *Filer) maybeReloadFilerConfiguration(event *filer_pb.SubscribeMetadataResponse) {
if DirectoryEtcSeaweedFS != event.Directory {
if DirectoryEtcSeaweedFS != event.EventNotification.NewParentPath {
return
@@ -26,12 +48,11 @@ func (f *Filer) onMetadataChangeEvent(event *filer_pb.SubscribeMetadataResponse)
if entry.Name == FilerConfName {
f.reloadFilerConfiguration(entry)
}
-
}
func (f *Filer) readEntry(chunks []*filer_pb.FileChunk) ([]byte, error) {
var buf bytes.Buffer
- err := StreamContent(f.MasterClient, &buf, chunks, 0, math.MaxInt64)
+ err := StreamContent(f.MasterClient, &buf, chunks, 0, math.MaxInt64, false)
if err != nil {
return nil, err
}
diff --git a/weed/filer/filerstore.go b/weed/filer/filerstore.go
index 8955a25c7..a5b2f25de 100644
--- a/weed/filer/filerstore.go
+++ b/weed/filer/filerstore.go
@@ -39,3 +39,8 @@ type FilerStore interface {
Shutdown()
}
+
+type BucketAware interface {
+ OnBucketCreation(bucket string)
+ OnBucketDeletion(bucket string)
+}
diff --git a/weed/filer/filerstore_wrapper.go b/weed/filer/filerstore_wrapper.go
index 64baac371..95848e61b 100644
--- a/weed/filer/filerstore_wrapper.go
+++ b/weed/filer/filerstore_wrapper.go
@@ -21,6 +21,8 @@ type VirtualFilerStore interface {
DeleteHardLink(ctx context.Context, hardLinkId HardLinkId) error
DeleteOneEntry(ctx context.Context, entry *Entry) error
AddPathSpecificStore(path string, storeId string, store FilerStore)
+ OnBucketCreation(bucket string)
+ OnBucketDeletion(bucket string)
}
type FilerStoreWrapper struct {
@@ -40,6 +42,27 @@ func NewFilerStoreWrapper(store FilerStore) *FilerStoreWrapper {
}
}
+func (fsw *FilerStoreWrapper) OnBucketCreation(bucket string) {
+ for _, store := range fsw.storeIdToStore {
+ if ba, ok := store.(BucketAware); ok {
+ ba.OnBucketCreation(bucket)
+ }
+ }
+ if ba, ok := fsw.defaultStore.(BucketAware); ok {
+ ba.OnBucketCreation(bucket)
+ }
+}
+func (fsw *FilerStoreWrapper) OnBucketDeletion(bucket string) {
+ for _, store := range fsw.storeIdToStore {
+ if ba, ok := store.(BucketAware); ok {
+ ba.OnBucketDeletion(bucket)
+ }
+ }
+ if ba, ok := fsw.defaultStore.(BucketAware); ok {
+ ba.OnBucketDeletion(bucket)
+ }
+}
+
func (fsw *FilerStoreWrapper) AddPathSpecificStore(path string, storeId string, store FilerStore) {
fsw.storeIdToStore[storeId] = NewFilerStorePathTranlator(path, store)
err := fsw.pathToStore.Put([]byte(path), storeId)
diff --git a/weed/filer/leveldb/leveldb_store.go b/weed/filer/leveldb/leveldb_store.go
index ac0ad4ba6..ce454f36a 100644
--- a/weed/filer/leveldb/leveldb_store.go
+++ b/weed/filer/leveldb/leveldb_store.go
@@ -8,6 +8,7 @@ import (
leveldb_errors "github.com/syndtr/goleveldb/leveldb/errors"
"github.com/syndtr/goleveldb/leveldb/opt"
leveldb_util "github.com/syndtr/goleveldb/leveldb/util"
+ "os"
"github.com/chrislusf/seaweedfs/weed/filer"
"github.com/chrislusf/seaweedfs/weed/glog"
@@ -38,6 +39,7 @@ func (store *LevelDBStore) Initialize(configuration weed_util.Configuration, pre
func (store *LevelDBStore) initialize(dir string) (err error) {
glog.Infof("filer store dir: %s", dir)
+ os.MkdirAll(dir, 0755)
if err := weed_util.TestFolderWritable(dir); err != nil {
return fmt.Errorf("Check Level Folder %s Writable: %s", dir, err)
}
diff --git a/weed/filer/leveldb2/leveldb2_store.go b/weed/filer/leveldb2/leveldb2_store.go
index c76340da4..124d61c1c 100644
--- a/weed/filer/leveldb2/leveldb2_store.go
+++ b/weed/filer/leveldb2/leveldb2_store.go
@@ -38,6 +38,7 @@ func (store *LevelDB2Store) Initialize(configuration weed_util.Configuration, pr
func (store *LevelDB2Store) initialize(dir string, dbCount int) (err error) {
glog.Infof("filer store leveldb2 dir: %s", dir)
+ os.MkdirAll(dir, 0755)
if err := weed_util.TestFolderWritable(dir); err != nil {
return fmt.Errorf("Check Level Folder %s Writable: %s", dir, err)
}
diff --git a/weed/filer/leveldb3/leveldb3_store.go b/weed/filer/leveldb3/leveldb3_store.go
index c81ea7bbe..d1cdfbbf6 100644
--- a/weed/filer/leveldb3/leveldb3_store.go
+++ b/weed/filer/leveldb3/leveldb3_store.go
@@ -45,6 +45,7 @@ func (store *LevelDB3Store) Initialize(configuration weed_util.Configuration, pr
func (store *LevelDB3Store) initialize(dir string) (err error) {
glog.Infof("filer store leveldb3 dir: %s", dir)
+ os.MkdirAll(dir, 0755)
if err := weed_util.TestFolderWritable(dir); err != nil {
return fmt.Errorf("Check Level Folder %s Writable: %s", dir, err)
}
diff --git a/weed/filer/meta_aggregator.go b/weed/filer/meta_aggregator.go
index 9437e9992..5c368a57e 100644
--- a/weed/filer/meta_aggregator.go
+++ b/weed/filer/meta_aggregator.go
@@ -69,6 +69,7 @@ func (ma *MetaAggregator) subscribeToOneFiler(f *Filer, self string, peer string
peerSignature, err = ma.readFilerStoreSignature(peer)
}
+ // when filer store is not shared by multiple filers
if peerSignature != f.Signature {
if prevTsNs, err := ma.readOffset(f, peer, peerSignature); err == nil {
lastTsNs = prevTsNs
diff --git a/weed/filer/mysql/mysql_sql_gen.go b/weed/filer/mysql/mysql_sql_gen.go
index 4213cf965..93d3e3f9e 100644
--- a/weed/filer/mysql/mysql_sql_gen.go
+++ b/weed/filer/mysql/mysql_sql_gen.go
@@ -2,6 +2,7 @@ package mysql
import (
"fmt"
+
"github.com/chrislusf/seaweedfs/weed/filer/abstract_sql"
_ "github.com/go-sql-driver/mysql"
)
@@ -9,44 +10,49 @@ import (
type SqlGenMysql struct {
CreateTableSqlTemplate string
DropTableSqlTemplate string
+ UpsertQueryTemplate string
}
var (
_ = abstract_sql.SqlGenerator(&SqlGenMysql{})
)
-func (gen *SqlGenMysql) GetSqlInsert(bucket string) string {
- return fmt.Sprintf("INSERT INTO `%s` (dirhash,name,directory,meta) VALUES(?,?,?,?)", bucket)
+func (gen *SqlGenMysql) GetSqlInsert(tableName string) string {
+ if gen.UpsertQueryTemplate != "" {
+ return fmt.Sprintf(gen.UpsertQueryTemplate, tableName)
+ } else {
+ return fmt.Sprintf("INSERT INTO `%s` (dirhash,name,directory,meta) VALUES(?,?,?,?)", tableName)
+ }
}
-func (gen *SqlGenMysql) GetSqlUpdate(bucket string) string {
- return fmt.Sprintf("UPDATE `%s` SET meta=? WHERE dirhash=? AND name=? AND directory=?", bucket)
+func (gen *SqlGenMysql) GetSqlUpdate(tableName string) string {
+ return fmt.Sprintf("UPDATE `%s` SET meta=? WHERE dirhash=? AND name=? AND directory=?", tableName)
}
-func (gen *SqlGenMysql) GetSqlFind(bucket string) string {
- return fmt.Sprintf("SELECT meta FROM `%s` WHERE dirhash=? AND name=? AND directory=?", bucket)
+func (gen *SqlGenMysql) GetSqlFind(tableName string) string {
+ return fmt.Sprintf("SELECT meta FROM `%s` WHERE dirhash=? AND name=? AND directory=?", tableName)
}
-func (gen *SqlGenMysql) GetSqlDelete(bucket string) string {
- return fmt.Sprintf("DELETE FROM `%s` WHERE dirhash=? AND name=? AND directory=?", bucket)
+func (gen *SqlGenMysql) GetSqlDelete(tableName string) string {
+ return fmt.Sprintf("DELETE FROM `%s` WHERE dirhash=? AND name=? AND directory=?", tableName)
}
-func (gen *SqlGenMysql) GetSqlDeleteFolderChildren(bucket string) string {
- return fmt.Sprintf("DELETE FROM `%s` WHERE dirhash=? AND directory=?", bucket)
+func (gen *SqlGenMysql) GetSqlDeleteFolderChildren(tableName string) string {
+ return fmt.Sprintf("DELETE FROM `%s` WHERE dirhash=? AND directory=?", tableName)
}
-func (gen *SqlGenMysql) GetSqlListExclusive(bucket string) string {
- return fmt.Sprintf("SELECT NAME, meta FROM `%s` WHERE dirhash=? AND name>? AND directory=? AND name like ? ORDER BY NAME ASC LIMIT ?", bucket)
+func (gen *SqlGenMysql) GetSqlListExclusive(tableName string) string {
+ return fmt.Sprintf("SELECT NAME, meta FROM `%s` WHERE dirhash=? AND name>? AND directory=? AND name like ? ORDER BY NAME ASC LIMIT ?", tableName)
}
-func (gen *SqlGenMysql) GetSqlListInclusive(bucket string) string {
- return fmt.Sprintf("SELECT NAME, meta FROM `%s` WHERE dirhash=? AND name>=? AND directory=? AND name like ? ORDER BY NAME ASC LIMIT ?", bucket)
+func (gen *SqlGenMysql) GetSqlListInclusive(tableName string) string {
+ return fmt.Sprintf("SELECT NAME, meta FROM `%s` WHERE dirhash=? AND name>=? AND directory=? AND name like ? ORDER BY NAME ASC LIMIT ?", tableName)
}
-func (gen *SqlGenMysql) GetSqlCreateTable(bucket string) string {
- return fmt.Sprintf(gen.CreateTableSqlTemplate, bucket)
+func (gen *SqlGenMysql) GetSqlCreateTable(tableName string) string {
+ return fmt.Sprintf(gen.CreateTableSqlTemplate, tableName)
}
-func (gen *SqlGenMysql) GetSqlDropTable(bucket string) string {
- return fmt.Sprintf(gen.DropTableSqlTemplate, bucket)
+func (gen *SqlGenMysql) GetSqlDropTable(tableName string) string {
+ return fmt.Sprintf(gen.DropTableSqlTemplate, tableName)
}
diff --git a/weed/filer/mysql/mysql_store.go b/weed/filer/mysql/mysql_store.go
index 501ab1d39..fbaa4d5f9 100644
--- a/weed/filer/mysql/mysql_store.go
+++ b/weed/filer/mysql/mysql_store.go
@@ -3,9 +3,10 @@ package mysql
import (
"database/sql"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/filer"
"time"
+ "github.com/chrislusf/seaweedfs/weed/filer"
+
"github.com/chrislusf/seaweedfs/weed/filer/abstract_sql"
"github.com/chrislusf/seaweedfs/weed/util"
_ "github.com/go-sql-driver/mysql"
@@ -29,6 +30,8 @@ func (store *MysqlStore) GetName() string {
func (store *MysqlStore) Initialize(configuration util.Configuration, prefix string) (err error) {
return store.initialize(
+ configuration.GetString(prefix+"upsertQuery"),
+ configuration.GetBool(prefix+"enableUpsert"),
configuration.GetString(prefix+"username"),
configuration.GetString(prefix+"password"),
configuration.GetString(prefix+"hostname"),
@@ -41,13 +44,17 @@ func (store *MysqlStore) Initialize(configuration util.Configuration, prefix str
)
}
-func (store *MysqlStore) initialize(user, password, hostname string, port int, database string, maxIdle, maxOpen,
+func (store *MysqlStore) initialize(upsertQuery string, enableUpsert bool, user, password, hostname string, port int, database string, maxIdle, maxOpen,
maxLifetimeSeconds int, interpolateParams bool) (err error) {
store.SupportBucketTable = false
+ if !enableUpsert {
+ upsertQuery = ""
+ }
store.SqlGenerator = &SqlGenMysql{
CreateTableSqlTemplate: "",
DropTableSqlTemplate: "drop table `%s`",
+ UpsertQueryTemplate: upsertQuery,
}
sqlUrl := fmt.Sprintf(CONNECTION_URL_PATTERN, user, password, hostname, port, database)
diff --git a/weed/filer/mysql2/mysql2_store.go b/weed/filer/mysql2/mysql2_store.go
index c796cd6aa..a1f54455a 100644
--- a/weed/filer/mysql2/mysql2_store.go
+++ b/weed/filer/mysql2/mysql2_store.go
@@ -32,6 +32,8 @@ func (store *MysqlStore2) GetName() string {
func (store *MysqlStore2) Initialize(configuration util.Configuration, prefix string) (err error) {
return store.initialize(
configuration.GetString(prefix+"createTable"),
+ configuration.GetString(prefix+"upsertQuery"),
+ configuration.GetBool(prefix+"enableUpsert"),
configuration.GetString(prefix+"username"),
configuration.GetString(prefix+"password"),
configuration.GetString(prefix+"hostname"),
@@ -44,13 +46,17 @@ func (store *MysqlStore2) Initialize(configuration util.Configuration, prefix st
)
}
-func (store *MysqlStore2) initialize(createTable, user, password, hostname string, port int, database string, maxIdle, maxOpen,
+func (store *MysqlStore2) initialize(createTable, upsertQuery string, enableUpsert bool, user, password, hostname string, port int, database string, maxIdle, maxOpen,
maxLifetimeSeconds int, interpolateParams bool) (err error) {
store.SupportBucketTable = true
+ if !enableUpsert {
+ upsertQuery = ""
+ }
store.SqlGenerator = &mysql.SqlGenMysql{
CreateTableSqlTemplate: createTable,
DropTableSqlTemplate: "drop table `%s`",
+ UpsertQueryTemplate: upsertQuery,
}
sqlUrl := fmt.Sprintf(CONNECTION_URL_PATTERN, user, password, hostname, port, database)
diff --git a/weed/filer/postgres/postgres_sql_gen.go b/weed/filer/postgres/postgres_sql_gen.go
index e13070c3d..6cee3d2da 100644
--- a/weed/filer/postgres/postgres_sql_gen.go
+++ b/weed/filer/postgres/postgres_sql_gen.go
@@ -10,44 +10,49 @@ import (
type SqlGenPostgres struct {
CreateTableSqlTemplate string
DropTableSqlTemplate string
+ UpsertQueryTemplate string
}
var (
_ = abstract_sql.SqlGenerator(&SqlGenPostgres{})
)
-func (gen *SqlGenPostgres) GetSqlInsert(bucket string) string {
- return fmt.Sprintf(`INSERT INTO "%s" (dirhash,name,directory,meta) VALUES($1,$2,$3,$4)`, bucket)
+func (gen *SqlGenPostgres) GetSqlInsert(tableName string) string {
+ if gen.UpsertQueryTemplate != "" {
+ return fmt.Sprintf(gen.UpsertQueryTemplate, tableName)
+ } else {
+ return fmt.Sprintf(`INSERT INTO "%s" (dirhash,name,directory,meta) VALUES($1,$2,$3,$4)`, tableName)
+ }
}
-func (gen *SqlGenPostgres) GetSqlUpdate(bucket string) string {
- return fmt.Sprintf(`UPDATE "%s" SET meta=$1 WHERE dirhash=$2 AND name=$3 AND directory=$4`, bucket)
+func (gen *SqlGenPostgres) GetSqlUpdate(tableName string) string {
+ return fmt.Sprintf(`UPDATE "%s" SET meta=$1 WHERE dirhash=$2 AND name=$3 AND directory=$4`, tableName)
}
-func (gen *SqlGenPostgres) GetSqlFind(bucket string) string {
- return fmt.Sprintf(`SELECT meta FROM "%s" WHERE dirhash=$1 AND name=$2 AND directory=$3`, bucket)
+func (gen *SqlGenPostgres) GetSqlFind(tableName string) string {
+ return fmt.Sprintf(`SELECT meta FROM "%s" WHERE dirhash=$1 AND name=$2 AND directory=$3`, tableName)
}
-func (gen *SqlGenPostgres) GetSqlDelete(bucket string) string {
- return fmt.Sprintf(`DELETE FROM "%s" WHERE dirhash=$1 AND name=$2 AND directory=$3`, bucket)
+func (gen *SqlGenPostgres) GetSqlDelete(tableName string) string {
+ return fmt.Sprintf(`DELETE FROM "%s" WHERE dirhash=$1 AND name=$2 AND directory=$3`, tableName)
}
-func (gen *SqlGenPostgres) GetSqlDeleteFolderChildren(bucket string) string {
- return fmt.Sprintf(`DELETE FROM "%s" WHERE dirhash=$1 AND directory=$2`, bucket)
+func (gen *SqlGenPostgres) GetSqlDeleteFolderChildren(tableName string) string {
+ return fmt.Sprintf(`DELETE FROM "%s" WHERE dirhash=$1 AND directory=$2`, tableName)
}
-func (gen *SqlGenPostgres) GetSqlListExclusive(bucket string) string {
- return fmt.Sprintf(`SELECT NAME, meta FROM "%s" WHERE dirhash=$1 AND name>$2 AND directory=$3 AND name like $4 ORDER BY NAME ASC LIMIT $5`, bucket)
+func (gen *SqlGenPostgres) GetSqlListExclusive(tableName string) string {
+ return fmt.Sprintf(`SELECT NAME, meta FROM "%s" WHERE dirhash=$1 AND name>$2 AND directory=$3 AND name like $4 ORDER BY NAME ASC LIMIT $5`, tableName)
}
-func (gen *SqlGenPostgres) GetSqlListInclusive(bucket string) string {
- return fmt.Sprintf(`SELECT NAME, meta FROM "%s" WHERE dirhash=$1 AND name>=$2 AND directory=$3 AND name like $4 ORDER BY NAME ASC LIMIT $5`, bucket)
+func (gen *SqlGenPostgres) GetSqlListInclusive(tableName string) string {
+ return fmt.Sprintf(`SELECT NAME, meta FROM "%s" WHERE dirhash=$1 AND name>=$2 AND directory=$3 AND name like $4 ORDER BY NAME ASC LIMIT $5`, tableName)
}
-func (gen *SqlGenPostgres) GetSqlCreateTable(bucket string) string {
- return fmt.Sprintf(gen.CreateTableSqlTemplate, bucket)
+func (gen *SqlGenPostgres) GetSqlCreateTable(tableName string) string {
+ return fmt.Sprintf(gen.CreateTableSqlTemplate, tableName)
}
-func (gen *SqlGenPostgres) GetSqlDropTable(bucket string) string {
- return fmt.Sprintf(gen.DropTableSqlTemplate, bucket)
+func (gen *SqlGenPostgres) GetSqlDropTable(tableName string) string {
+ return fmt.Sprintf(gen.DropTableSqlTemplate, tableName)
}
diff --git a/weed/filer/postgres/postgres_store.go b/weed/filer/postgres/postgres_store.go
index 9e4ff7c32..a1e16a92a 100644
--- a/weed/filer/postgres/postgres_store.go
+++ b/weed/filer/postgres/postgres_store.go
@@ -29,6 +29,8 @@ func (store *PostgresStore) GetName() string {
func (store *PostgresStore) Initialize(configuration util.Configuration, prefix string) (err error) {
return store.initialize(
+ configuration.GetString(prefix+"upsertQuery"),
+ configuration.GetBool(prefix+"enableUpsert"),
configuration.GetString(prefix+"username"),
configuration.GetString(prefix+"password"),
configuration.GetString(prefix+"hostname"),
@@ -42,12 +44,16 @@ func (store *PostgresStore) Initialize(configuration util.Configuration, prefix
)
}
-func (store *PostgresStore) initialize(user, password, hostname string, port int, database, schema, sslmode string, maxIdle, maxOpen, maxLifetimeSeconds int) (err error) {
+func (store *PostgresStore) initialize(upsertQuery string, enableUpsert bool, user, password, hostname string, port int, database, schema, sslmode string, maxIdle, maxOpen, maxLifetimeSeconds int) (err error) {
store.SupportBucketTable = false
+ if !enableUpsert {
+ upsertQuery = ""
+ }
store.SqlGenerator = &SqlGenPostgres{
CreateTableSqlTemplate: "",
DropTableSqlTemplate: `drop table "%s"`,
+ UpsertQueryTemplate: upsertQuery,
}
sqlUrl := fmt.Sprintf(CONNECTION_URL_PATTERN, hostname, port, sslmode)
diff --git a/weed/filer/postgres2/postgres2_store.go b/weed/filer/postgres2/postgres2_store.go
index 92893bf7a..0f573d8d0 100644
--- a/weed/filer/postgres2/postgres2_store.go
+++ b/weed/filer/postgres2/postgres2_store.go
@@ -32,6 +32,8 @@ func (store *PostgresStore2) GetName() string {
func (store *PostgresStore2) Initialize(configuration util.Configuration, prefix string) (err error) {
return store.initialize(
configuration.GetString(prefix+"createTable"),
+ configuration.GetString(prefix+"upsertQuery"),
+ configuration.GetBool(prefix+"enableUpsert"),
configuration.GetString(prefix+"username"),
configuration.GetString(prefix+"password"),
configuration.GetString(prefix+"hostname"),
@@ -45,12 +47,16 @@ func (store *PostgresStore2) Initialize(configuration util.Configuration, prefix
)
}
-func (store *PostgresStore2) initialize(createTable, user, password, hostname string, port int, database, schema, sslmode string, maxIdle, maxOpen, maxLifetimeSeconds int) (err error) {
+func (store *PostgresStore2) initialize(createTable, upsertQuery string, enableUpsert bool, user, password, hostname string, port int, database, schema, sslmode string, maxIdle, maxOpen, maxLifetimeSeconds int) (err error) {
store.SupportBucketTable = true
+ if !enableUpsert {
+ upsertQuery = ""
+ }
store.SqlGenerator = &postgres.SqlGenPostgres{
CreateTableSqlTemplate: createTable,
DropTableSqlTemplate: `drop table "%s"`,
+ UpsertQueryTemplate: upsertQuery,
}
sqlUrl := fmt.Sprintf(CONNECTION_URL_PATTERN, hostname, port, sslmode)
diff --git a/weed/filer/read_write.go b/weed/filer/read_write.go
index bf9159b41..d92d526d5 100644
--- a/weed/filer/read_write.go
+++ b/weed/filer/read_write.go
@@ -27,7 +27,7 @@ func ReadEntry(masterClient *wdclient.MasterClient, filerClient filer_pb.Seaweed
return err
}
- return StreamContent(masterClient, byteBuffer, respLookupEntry.Entry.Chunks, 0, math.MaxInt64)
+ return StreamContent(masterClient, byteBuffer, respLookupEntry.Entry.Chunks, 0, math.MaxInt64, false)
}
@@ -35,7 +35,7 @@ func ReadContent(filerAddress string, dir, name string) ([]byte, error) {
target := fmt.Sprintf("http://%s%s/%s", filerAddress, dir, name)
- data, _, err := util.FastGet(target)
+ data, _, err := util.Get(target)
return data, err
}
diff --git a/weed/filer/rocksdb/rocksdb_store.go b/weed/filer/rocksdb/rocksdb_store.go
index 70c301725..379a18c62 100644
--- a/weed/filer/rocksdb/rocksdb_store.go
+++ b/weed/filer/rocksdb/rocksdb_store.go
@@ -8,6 +8,7 @@ import (
"crypto/md5"
"fmt"
"io"
+ "os"
"github.com/tecbot/gorocksdb"
@@ -56,6 +57,7 @@ func (store *RocksDBStore) Initialize(configuration weed_util.Configuration, pre
func (store *RocksDBStore) initialize(dir string) (err error) {
glog.Infof("filer store rocksdb dir: %s", dir)
+ os.MkdirAll(dir, 0755)
if err := weed_util.TestFolderWritable(dir); err != nil {
return fmt.Errorf("Check Level Folder %s Writable: %s", dir, err)
}
diff --git a/weed/filer/stream.go b/weed/filer/stream.go
index 075204b79..661a210ea 100644
--- a/weed/filer/stream.go
+++ b/weed/filer/stream.go
@@ -3,6 +3,7 @@ package filer
import (
"bytes"
"fmt"
+ "golang.org/x/sync/errgroup"
"io"
"math"
"strings"
@@ -13,9 +14,9 @@ import (
"github.com/chrislusf/seaweedfs/weed/wdclient"
)
-func StreamContent(masterClient wdclient.HasLookupFileIdFunction, w io.Writer, chunks []*filer_pb.FileChunk, offset int64, size int64) error {
+func StreamContent(masterClient wdclient.HasLookupFileIdFunction, w io.Writer, chunks []*filer_pb.FileChunk, offset int64, size int64, isCheck bool) error {
- // fmt.Printf("start to stream content for chunks: %+v\n", chunks)
+ glog.V(9).Infof("start to stream content for chunks: %+v\n", chunks)
chunkViews := ViewFromChunks(masterClient.GetLookupFileIdFunction(), chunks, offset, size)
fileId2Url := make(map[string][]string)
@@ -26,19 +27,33 @@ func StreamContent(masterClient wdclient.HasLookupFileIdFunction, w io.Writer, c
if err != nil {
glog.V(1).Infof("operation LookupFileId %s failed, err: %v", chunkView.FileId, err)
return err
+ } else if len(urlStrings) == 0 {
+ glog.Errorf("operation LookupFileId %s failed, err: urls not found", chunkView.FileId)
+ return fmt.Errorf("operation LookupFileId %s failed, err: urls not found", chunkView.FileId)
}
fileId2Url[chunkView.FileId] = urlStrings
}
+ if isCheck {
+ // Pre-check all chunkViews urls
+ gErr := new(errgroup.Group)
+ CheckAllChunkViews(chunkViews, &fileId2Url, gErr)
+ if err := gErr.Wait(); err != nil {
+ glog.Errorf("check all chunks: %v", err)
+ return fmt.Errorf("check all chunks: %v", err)
+ }
+ return nil
+ }
+
for _, chunkView := range chunkViews {
urlStrings := fileId2Url[chunkView.FileId]
-
data, err := retriedFetchChunkData(urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.Offset, int(chunkView.Size))
if err != nil {
glog.Errorf("read chunk: %v", err)
return fmt.Errorf("read chunk: %v", err)
}
+
_, err = w.Write(data)
if err != nil {
glog.Errorf("write chunk: %v", err)
@@ -50,6 +65,17 @@ func StreamContent(masterClient wdclient.HasLookupFileIdFunction, w io.Writer, c
}
+func CheckAllChunkViews(chunkViews []*ChunkView, fileId2Url *map[string][]string, gErr *errgroup.Group) {
+ for _, chunkView := range chunkViews {
+ urlStrings := (*fileId2Url)[chunkView.FileId]
+ glog.V(9).Infof("Check chunk: %+v\n url: %v", chunkView, urlStrings)
+ gErr.Go(func() error {
+ _, err := retriedFetchChunkData(urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.Offset, int(chunkView.Size))
+ return err
+ })
+ }
+}
+
// ---------------- ReadAllReader ----------------------------------
func ReadAll(masterClient *wdclient.MasterClient, chunks []*filer_pb.FileChunk) ([]byte, error) {
@@ -181,7 +207,7 @@ func (c *ChunkStreamReader) fetchChunkToBuffer(chunkView *ChunkView) error {
var buffer bytes.Buffer
var shouldRetry bool
for _, urlString := range urlStrings {
- shouldRetry, err = util.FastReadUrlAsStream(urlString+"?readDeleted=true", chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.Offset, int(chunkView.Size), func(data []byte) {
+ shouldRetry, err = util.ReadUrlAsStream(urlString, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.Offset, int(chunkView.Size), func(data []byte) {
buffer.Write(data)
})
if !shouldRetry {
diff --git a/weed/filesys/dir.go b/weed/filesys/dir.go
index 10a0a2b44..46457f858 100644
--- a/weed/filesys/dir.go
+++ b/weed/filesys/dir.go
@@ -128,6 +128,10 @@ func (dir *Dir) newDirectory(fullpath util.FullPath, entry *filer_pb.Entry) fs.N
func (dir *Dir) Create(ctx context.Context, req *fuse.CreateRequest,
resp *fuse.CreateResponse) (fs.Node, fs.Handle, error) {
+ if dir.wfs.option.ReadOnly {
+ return nil, nil, fuse.EPERM
+ }
+
request, err := dir.doCreateEntry(req.Name, req.Mode, req.Uid, req.Gid, req.Flags&fuse.OpenExclusive != 0)
if err != nil {
@@ -148,6 +152,10 @@ func (dir *Dir) Create(ctx context.Context, req *fuse.CreateRequest,
func (dir *Dir) Mknod(ctx context.Context, req *fuse.MknodRequest) (fs.Node, error) {
+ if dir.wfs.option.ReadOnly {
+ return nil, fuse.EPERM
+ }
+
request, err := dir.doCreateEntry(req.Name, req.Mode, req.Uid, req.Gid, false)
if err != nil {
@@ -202,6 +210,10 @@ func (dir *Dir) doCreateEntry(name string, mode os.FileMode, uid, gid uint32, ex
func (dir *Dir) Mkdir(ctx context.Context, req *fuse.MkdirRequest) (fs.Node, error) {
+ if dir.wfs.option.ReadOnly {
+ return nil, fuse.EPERM
+ }
+
glog.V(4).Infof("mkdir %s: %s", dir.FullPath(), req.Name)
newEntry := &filer_pb.Entry{
@@ -251,10 +263,10 @@ func (dir *Dir) Mkdir(ctx context.Context, req *fuse.MkdirRequest) (fs.Node, err
func (dir *Dir) Lookup(ctx context.Context, req *fuse.LookupRequest, resp *fuse.LookupResponse) (node fs.Node, err error) {
- glog.V(4).Infof("dir Lookup %s: %s by %s", dir.FullPath(), req.Name, req.Header.String())
-
- fullFilePath := util.NewFullPath(dir.FullPath(), req.Name)
dirPath := util.FullPath(dir.FullPath())
+ glog.V(4).Infof("dir Lookup %s: %s by %s", dirPath, req.Name, req.Header.String())
+
+ fullFilePath := dirPath.Child(req.Name)
visitErr := meta_cache.EnsureVisited(dir.wfs.metaCache, dir.wfs, dirPath)
if visitErr != nil {
glog.Errorf("dir Lookup %s: %v", dirPath, visitErr)
@@ -305,7 +317,8 @@ func (dir *Dir) Lookup(ctx context.Context, req *fuse.LookupRequest, resp *fuse.
func (dir *Dir) ReadDirAll(ctx context.Context) (ret []fuse.Dirent, err error) {
- glog.V(4).Infof("dir ReadDirAll %s", dir.FullPath())
+ dirPath := util.FullPath(dir.FullPath())
+ glog.V(4).Infof("dir ReadDirAll %s", dirPath)
processEachEntryFn := func(entry *filer_pb.Entry, isLast bool) error {
if entry.IsDirectory {
@@ -318,12 +331,11 @@ func (dir *Dir) ReadDirAll(ctx context.Context) (ret []fuse.Dirent, err error) {
return nil
}
- dirPath := util.FullPath(dir.FullPath())
if err = meta_cache.EnsureVisited(dir.wfs.metaCache, dir.wfs, dirPath); err != nil {
glog.Errorf("dir ReadDirAll %s: %v", dirPath, err)
return nil, fuse.EIO
}
- listErr := dir.wfs.metaCache.ListDirectoryEntries(context.Background(), util.FullPath(dir.FullPath()), "", false, int64(math.MaxInt32), func(entry *filer.Entry) bool {
+ listErr := dir.wfs.metaCache.ListDirectoryEntries(context.Background(), dirPath, "", false, int64(math.MaxInt32), func(entry *filer.Entry) bool {
processEachEntryFn(entry.ToProtoEntry(), false)
return true
})
@@ -356,6 +368,11 @@ func findFileType(mode uint16) fuse.DirentType {
func (dir *Dir) Remove(ctx context.Context, req *fuse.RemoveRequest) error {
+ if dir.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
+
if !req.Dir {
return dir.removeOneFile(req)
}
@@ -389,12 +406,12 @@ func (dir *Dir) removeOneFile(req *fuse.RemoveRequest) error {
// clear entry inside the file
fsNode := dir.wfs.fsNodeCache.GetFsNode(filePath)
+ dir.wfs.fsNodeCache.DeleteFsNode(filePath)
if fsNode != nil {
if file, ok := fsNode.(*File); ok {
file.clearEntry()
}
}
- dir.wfs.fsNodeCache.DeleteFsNode(filePath)
// remove current file handle if any
dir.wfs.handlesLock.Lock()
@@ -429,6 +446,10 @@ func (dir *Dir) removeFolder(req *fuse.RemoveRequest) error {
func (dir *Dir) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *fuse.SetattrResponse) error {
+ if dir.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
glog.V(4).Infof("%v dir setattr %+v", dir.FullPath(), req)
if err := dir.maybeLoadEntry(); err != nil {
@@ -457,6 +478,10 @@ func (dir *Dir) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *fus
func (dir *Dir) Setxattr(ctx context.Context, req *fuse.SetxattrRequest) error {
+ if dir.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
glog.V(4).Infof("dir Setxattr %s: %s", dir.FullPath(), req.Name)
if err := dir.maybeLoadEntry(); err != nil {
@@ -473,6 +498,10 @@ func (dir *Dir) Setxattr(ctx context.Context, req *fuse.SetxattrRequest) error {
func (dir *Dir) Removexattr(ctx context.Context, req *fuse.RemovexattrRequest) error {
+ if dir.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
glog.V(4).Infof("dir Removexattr %s: %s", dir.FullPath(), req.Name)
if err := dir.maybeLoadEntry(); err != nil {
diff --git a/weed/filesys/dir_link.go b/weed/filesys/dir_link.go
index ba3280f03..6266e492d 100644
--- a/weed/filesys/dir_link.go
+++ b/weed/filesys/dir_link.go
@@ -24,6 +24,10 @@ const (
func (dir *Dir) Link(ctx context.Context, req *fuse.LinkRequest, old fs.Node) (fs.Node, error) {
+ if dir.wfs.option.ReadOnly {
+ return nil, fuse.EPERM
+ }
+
oldFile, ok := old.(*File)
if !ok {
glog.Errorf("old node is not a file: %+v", old)
@@ -35,15 +39,20 @@ func (dir *Dir) Link(ctx context.Context, req *fuse.LinkRequest, old fs.Node) (f
return nil, err
}
+ oldEntry := oldFile.getEntry()
+ if oldEntry == nil {
+ return nil, fuse.EIO
+ }
+
// update old file to hardlink mode
- if len(oldFile.entry.HardLinkId) == 0 {
- oldFile.entry.HardLinkId = append(util.RandomBytes(16), HARD_LINK_MARKER)
- oldFile.entry.HardLinkCounter = 1
+ if len(oldEntry.HardLinkId) == 0 {
+ oldEntry.HardLinkId = append(util.RandomBytes(16), HARD_LINK_MARKER)
+ oldEntry.HardLinkCounter = 1
}
- oldFile.entry.HardLinkCounter++
+ oldEntry.HardLinkCounter++
updateOldEntryRequest := &filer_pb.UpdateEntryRequest{
Directory: oldFile.dir.FullPath(),
- Entry: oldFile.entry,
+ Entry: oldEntry,
Signatures: []int32{dir.wfs.signature},
}
@@ -53,11 +62,11 @@ func (dir *Dir) Link(ctx context.Context, req *fuse.LinkRequest, old fs.Node) (f
Entry: &filer_pb.Entry{
Name: req.NewName,
IsDirectory: false,
- Attributes: oldFile.entry.Attributes,
- Chunks: oldFile.entry.Chunks,
- Extended: oldFile.entry.Extended,
- HardLinkId: oldFile.entry.HardLinkId,
- HardLinkCounter: oldFile.entry.HardLinkCounter,
+ Attributes: oldEntry.Attributes,
+ Chunks: oldEntry.Chunks,
+ Extended: oldEntry.Extended,
+ HardLinkId: oldEntry.HardLinkId,
+ HardLinkCounter: oldEntry.HardLinkCounter,
},
Signatures: []int32{dir.wfs.signature},
}
@@ -83,6 +92,10 @@ func (dir *Dir) Link(ctx context.Context, req *fuse.LinkRequest, old fs.Node) (f
return nil
})
+ if err != nil {
+ return nil, fuse.EIO
+ }
+
// create new file node
newNode := dir.newFile(req.NewName, request.Entry)
newFile := newNode.(*File)
@@ -96,6 +109,10 @@ func (dir *Dir) Link(ctx context.Context, req *fuse.LinkRequest, old fs.Node) (f
func (dir *Dir) Symlink(ctx context.Context, req *fuse.SymlinkRequest) (fs.Node, error) {
+ if dir.wfs.option.ReadOnly {
+ return nil, fuse.EPERM
+ }
+
glog.V(4).Infof("Symlink: %v/%v to %v", dir.FullPath(), req.NewName, req.Target)
request := &filer_pb.CreateEntryRequest{
diff --git a/weed/filesys/dir_rename.go b/weed/filesys/dir_rename.go
index d2acad4b2..28316c3bd 100644
--- a/weed/filesys/dir_rename.go
+++ b/weed/filesys/dir_rename.go
@@ -13,6 +13,10 @@ import (
func (dir *Dir) Rename(ctx context.Context, req *fuse.RenameRequest, newDirectory fs.Node) error {
+ if dir.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
newDir := newDirectory.(*Dir)
newPath := util.NewFullPath(newDir.FullPath(), req.NewName)
diff --git a/weed/filesys/dir_test.go b/weed/filesys/dir_test.go
deleted file mode 100644
index 49c76eb5e..000000000
--- a/weed/filesys/dir_test.go
+++ /dev/null
@@ -1,34 +0,0 @@
-package filesys
-
-import (
- "testing"
-
- "github.com/stretchr/testify/assert"
-)
-
-func TestDirPath(t *testing.T) {
-
- p := &Dir{name: "/some"}
- p = &Dir{name: "path", parent: p}
- p = &Dir{name: "to", parent: p}
- p = &Dir{name: "a", parent: p}
- p = &Dir{name: "file", parent: p}
-
- assert.Equal(t, "/some/path/to/a/file", p.FullPath())
-
- p = &Dir{name: "/some"}
- assert.Equal(t, "/some", p.FullPath())
-
- p = &Dir{name: "/"}
- assert.Equal(t, "/", p.FullPath())
-
- p = &Dir{name: "/"}
- p = &Dir{name: "path", parent: p}
- assert.Equal(t, "/path", p.FullPath())
-
- p = &Dir{name: "/"}
- p = &Dir{name: "path", parent: p}
- p = &Dir{name: "to", parent: p}
- assert.Equal(t, "/path/to", p.FullPath())
-
-}
diff --git a/weed/filesys/dirty_page.go b/weed/filesys/dirty_page.go
index f05a3a56a..8888cff96 100644
--- a/weed/filesys/dirty_page.go
+++ b/weed/filesys/dirty_page.go
@@ -30,7 +30,7 @@ func newDirtyPages(file *File) *ContinuousDirtyPages {
func (pages *ContinuousDirtyPages) AddPage(offset int64, data []byte) {
- glog.V(4).Infof("%s AddPage [%d,%d) of %d bytes", pages.f.fullpath(), offset, offset+int64(len(data)), pages.f.entry.Attributes.FileSize)
+ glog.V(4).Infof("%s AddPage [%d,%d)", pages.f.fullpath(), offset, offset+int64(len(data)))
if len(data) > int(pages.f.wfs.option.ChunkSizeLimit) {
// this is more than what buffer can hold.
@@ -69,7 +69,12 @@ func (pages *ContinuousDirtyPages) saveExistingLargestPageToStorage() (hasSavedD
return false
}
- fileSize := int64(pages.f.entry.Attributes.FileSize)
+ entry := pages.f.getEntry()
+ if entry == nil {
+ return false
+ }
+
+ fileSize := int64(entry.Attributes.FileSize)
chunkSize := min(maxList.Size(), fileSize-maxList.Offset())
if chunkSize == 0 {
diff --git a/weed/filesys/file.go b/weed/filesys/file.go
index a210c5152..2433be590 100644
--- a/weed/filesys/file.go
+++ b/weed/filesys/file.go
@@ -5,6 +5,7 @@ import (
"io"
"os"
"sort"
+ "sync"
"time"
"github.com/seaweedfs/fuse"
@@ -33,6 +34,7 @@ type File struct {
dir *Dir
wfs *WFS
entry *filer_pb.Entry
+ entryLock sync.RWMutex
entryViewCache []filer.VisibleInterval
isOpen int
reader io.ReaderAt
@@ -47,13 +49,17 @@ func (file *File) Attr(ctx context.Context, attr *fuse.Attr) (err error) {
glog.V(4).Infof("file Attr %s, open:%v existing:%v", file.fullpath(), file.isOpen, attr)
- entry := file.entry
+ entry := file.getEntry()
if file.isOpen <= 0 || entry == nil {
if entry, err = file.maybeLoadEntry(ctx); err != nil {
return err
}
}
+ if entry == nil {
+ return fuse.ENOENT
+ }
+
// attr.Inode = file.fullpath().AsInode()
attr.Valid = time.Second
attr.Mode = os.FileMode(entry.Attributes.FileMode)
@@ -104,9 +110,13 @@ func (file *File) Open(ctx context.Context, req *fuse.OpenRequest, resp *fuse.Op
func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *fuse.SetattrResponse) error {
+ if file.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
glog.V(4).Infof("%v file setattr %+v", file.fullpath(), req)
- _, err := file.maybeLoadEntry(ctx)
+ entry, err := file.maybeLoadEntry(ctx)
if err != nil {
return err
}
@@ -123,12 +133,12 @@ func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *f
if req.Valid.Size() {
- glog.V(4).Infof("%v file setattr set size=%v chunks=%d", file.fullpath(), req.Size, len(file.entry.Chunks))
- if req.Size < filer.FileSize(file.entry) {
+ glog.V(4).Infof("%v file setattr set size=%v chunks=%d", file.fullpath(), req.Size, len(entry.Chunks))
+ if req.Size < filer.FileSize(entry) {
// fmt.Printf("truncate %v \n", fullPath)
var chunks []*filer_pb.FileChunk
var truncatedChunks []*filer_pb.FileChunk
- for _, chunk := range file.entry.Chunks {
+ for _, chunk := range entry.Chunks {
int64Size := int64(chunk.Size)
if chunk.Offset+int64Size > int64(req.Size) {
// this chunk is truncated
@@ -143,36 +153,36 @@ func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *f
}
}
}
- file.entry.Chunks = chunks
+ entry.Chunks = chunks
file.entryViewCache, _ = filer.NonOverlappingVisibleIntervals(file.wfs.LookupFn(), chunks)
- file.reader = nil
+ file.setReader(nil)
}
- file.entry.Attributes.FileSize = req.Size
+ entry.Attributes.FileSize = req.Size
file.dirtyMetadata = true
}
if req.Valid.Mode() {
- file.entry.Attributes.FileMode = uint32(req.Mode)
+ entry.Attributes.FileMode = uint32(req.Mode)
file.dirtyMetadata = true
}
if req.Valid.Uid() {
- file.entry.Attributes.Uid = req.Uid
+ entry.Attributes.Uid = req.Uid
file.dirtyMetadata = true
}
if req.Valid.Gid() {
- file.entry.Attributes.Gid = req.Gid
+ entry.Attributes.Gid = req.Gid
file.dirtyMetadata = true
}
if req.Valid.Crtime() {
- file.entry.Attributes.Crtime = req.Crtime.Unix()
+ entry.Attributes.Crtime = req.Crtime.Unix()
file.dirtyMetadata = true
}
if req.Valid.Mtime() {
- file.entry.Attributes.Mtime = req.Mtime.Unix()
+ entry.Attributes.Mtime = req.Mtime.Unix()
file.dirtyMetadata = true
}
@@ -188,12 +198,16 @@ func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *f
return nil
}
- return file.saveEntry(file.entry)
+ return file.saveEntry(entry)
}
func (file *File) Setxattr(ctx context.Context, req *fuse.SetxattrRequest) error {
+ if file.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
glog.V(4).Infof("file Setxattr %s: %s", file.fullpath(), req.Name)
entry, err := file.maybeLoadEntry(ctx)
@@ -211,6 +225,10 @@ func (file *File) Setxattr(ctx context.Context, req *fuse.SetxattrRequest) error
func (file *File) Removexattr(ctx context.Context, req *fuse.RemovexattrRequest) error {
+ if file.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
glog.V(4).Infof("file Removexattr %s: %s", file.fullpath(), req.Name)
entry, err := file.maybeLoadEntry(ctx)
@@ -255,10 +273,12 @@ func (file *File) Forget() {
t := util.NewFullPath(file.dir.FullPath(), file.Name)
glog.V(4).Infof("Forget file %s", t)
file.wfs.fsNodeCache.DeleteFsNode(t)
+ file.wfs.ReleaseHandle(t, 0)
+ file.setReader(nil)
}
func (file *File) maybeLoadEntry(ctx context.Context) (entry *filer_pb.Entry, err error) {
- entry = file.entry
+ entry = file.getEntry()
if file.isOpen > 0 {
return entry, nil
}
@@ -299,8 +319,13 @@ func (file *File) addChunks(chunks []*filer_pb.FileChunk) {
}
}
+ entry := file.getEntry()
+ if entry == nil {
+ return
+ }
+
// pick out-of-order chunks from existing chunks
- for _, chunk := range file.entry.Chunks {
+ for _, chunk := range entry.Chunks {
if lessThan(earliestChunk, chunk) {
chunks = append(chunks, chunk)
}
@@ -316,23 +341,37 @@ func (file *File) addChunks(chunks []*filer_pb.FileChunk) {
file.entryViewCache = filer.MergeIntoVisibles(file.entryViewCache, chunk)
}
- file.reader = nil
+ file.setReader(nil)
- glog.V(4).Infof("%s existing %d chunks adds %d more", file.fullpath(), len(file.entry.Chunks), len(chunks))
+ glog.V(4).Infof("%s existing %d chunks adds %d more", file.fullpath(), len(entry.Chunks), len(chunks))
- file.entry.Chunks = append(file.entry.Chunks, newChunks...)
+ entry.Chunks = append(entry.Chunks, newChunks...)
+}
+
+func (file *File) setReader(reader io.ReaderAt) {
+ r := file.reader
+ if r != nil {
+ if closer, ok := r.(io.Closer); ok {
+ closer.Close()
+ }
+ }
+ file.reader = reader
}
func (file *File) setEntry(entry *filer_pb.Entry) {
+ file.entryLock.Lock()
+ defer file.entryLock.Unlock()
file.entry = entry
file.entryViewCache, _ = filer.NonOverlappingVisibleIntervals(file.wfs.LookupFn(), entry.Chunks)
- file.reader = nil
+ file.setReader(nil)
}
func (file *File) clearEntry() {
+ file.entryLock.Lock()
+ defer file.entryLock.Unlock()
file.entry = nil
file.entryViewCache = nil
- file.reader = nil
+ file.setReader(nil)
}
func (file *File) saveEntry(entry *filer_pb.Entry) error {
@@ -359,3 +398,9 @@ func (file *File) saveEntry(entry *filer_pb.Entry) error {
return nil
})
}
+
+func (file *File) getEntry() *filer_pb.Entry {
+ file.entryLock.RLock()
+ defer file.entryLock.RUnlock()
+ return file.entry
+}
diff --git a/weed/filesys/filehandle.go b/weed/filesys/filehandle.go
index fb073c9cd..4419888c4 100644
--- a/weed/filesys/filehandle.go
+++ b/weed/filesys/filehandle.go
@@ -23,7 +23,7 @@ type FileHandle struct {
dirtyPages *ContinuousDirtyPages
contentType string
handle uint64
- sync.RWMutex
+ sync.Mutex
f *File
RequestId fuse.RequestID // unique ID for request
@@ -40,8 +40,9 @@ func newFileHandle(file *File, uid, gid uint32) *FileHandle {
Uid: uid,
Gid: gid,
}
- if fh.f.entry != nil {
- fh.f.entry.Attributes.FileSize = filer.FileSize(fh.f.entry)
+ entry := fh.f.getEntry()
+ if entry != nil {
+ entry.Attributes.FileSize = filer.FileSize(entry)
}
return fh
@@ -58,8 +59,8 @@ var _ = fs.HandleReleaser(&FileHandle{})
func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fuse.ReadResponse) error {
glog.V(4).Infof("%s read fh %d: [%d,%d) size %d resp.Data cap=%d", fh.f.fullpath(), fh.handle, req.Offset, req.Offset+int64(req.Size), req.Size, cap(resp.Data))
- fh.RLock()
- defer fh.RUnlock()
+ fh.Lock()
+ defer fh.Unlock()
if req.Size <= 0 {
return nil
@@ -104,26 +105,32 @@ func (fh *FileHandle) readFromDirtyPages(buff []byte, startOffset int64) (maxSto
func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
- fileSize := int64(filer.FileSize(fh.f.entry))
+ entry := fh.f.getEntry()
+ if entry == nil {
+ return 0, io.EOF
+ }
+
+ fileSize := int64(filer.FileSize(entry))
+ fileFullPath := fh.f.fullpath()
if fileSize == 0 {
- glog.V(1).Infof("empty fh %v", fh.f.fullpath())
+ glog.V(1).Infof("empty fh %v", fileFullPath)
return 0, io.EOF
}
- if offset+int64(len(buff)) <= int64(len(fh.f.entry.Content)) {
- totalRead := copy(buff, fh.f.entry.Content[offset:])
- glog.V(4).Infof("file handle read cached %s [%d,%d] %d", fh.f.fullpath(), offset, offset+int64(totalRead), totalRead)
+ if offset+int64(len(buff)) <= int64(len(entry.Content)) {
+ totalRead := copy(buff, entry.Content[offset:])
+ glog.V(4).Infof("file handle read cached %s [%d,%d] %d", fileFullPath, offset, offset+int64(totalRead), totalRead)
return int64(totalRead), nil
}
var chunkResolveErr error
if fh.f.entryViewCache == nil {
- fh.f.entryViewCache, chunkResolveErr = filer.NonOverlappingVisibleIntervals(fh.f.wfs.LookupFn(), fh.f.entry.Chunks)
+ fh.f.entryViewCache, chunkResolveErr = filer.NonOverlappingVisibleIntervals(fh.f.wfs.LookupFn(), entry.Chunks)
if chunkResolveErr != nil {
return 0, fmt.Errorf("fail to resolve chunk manifest: %v", chunkResolveErr)
}
- fh.f.reader = nil
+ fh.f.setReader(nil)
}
reader := fh.f.reader
@@ -131,15 +138,15 @@ func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
chunkViews := filer.ViewFromVisibleIntervals(fh.f.entryViewCache, 0, math.MaxInt64)
reader = filer.NewChunkReaderAtFromClient(fh.f.wfs.LookupFn(), chunkViews, fh.f.wfs.chunkCache, fileSize)
}
- fh.f.reader = reader
+ fh.f.setReader(reader)
totalRead, err := reader.ReadAt(buff, offset)
if err != nil && err != io.EOF {
- glog.Errorf("file handle read %s: %v", fh.f.fullpath(), err)
+ glog.Errorf("file handle read %s: %v", fileFullPath, err)
}
- glog.V(4).Infof("file handle read %s [%d,%d] %d : %v", fh.f.fullpath(), offset, offset+int64(totalRead), totalRead, err)
+ glog.V(4).Infof("file handle read %s [%d,%d] %d : %v", fileFullPath, offset, offset+int64(totalRead), totalRead, err)
return int64(totalRead), err
}
@@ -147,6 +154,10 @@ func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
// Write to the file handle
func (fh *FileHandle) Write(ctx context.Context, req *fuse.WriteRequest, resp *fuse.WriteResponse) error {
+ if fh.f.wfs.option.ReadOnly {
+ return fuse.EPERM
+ }
+
fh.Lock()
defer fh.Unlock()
@@ -158,8 +169,13 @@ func (fh *FileHandle) Write(ctx context.Context, req *fuse.WriteRequest, resp *f
copy(data, req.Data)
}
- fh.f.entry.Content = nil
- fh.f.entry.Attributes.FileSize = uint64(max(req.Offset+int64(len(data)), int64(fh.f.entry.Attributes.FileSize)))
+ entry := fh.f.getEntry()
+ if entry == nil {
+ return fuse.EIO
+ }
+
+ entry.Content = nil
+ entry.Attributes.FileSize = uint64(max(req.Offset+int64(len(data)), int64(entry.Attributes.FileSize)))
glog.V(4).Infof("%v write [%d,%d) %d", fh.f.fullpath(), req.Offset, req.Offset+int64(len(req.Data)), len(req.Data))
fh.dirtyPages.AddPage(req.Offset, data)
@@ -195,12 +211,7 @@ func (fh *FileHandle) Release(ctx context.Context, req *fuse.ReleaseRequest) err
fh.f.isOpen--
fh.f.wfs.ReleaseHandle(fh.f.fullpath(), fuse.HandleID(fh.handle))
- if closer, ok := fh.f.reader.(io.Closer); ok {
- if closer != nil {
- closer.Close()
- }
- }
- fh.f.reader = nil
+ fh.f.setReader(nil)
}
return nil
@@ -242,35 +253,40 @@ func (fh *FileHandle) doFlush(ctx context.Context, header fuse.Header) error {
err := fh.f.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
- if fh.f.entry.Attributes != nil {
- fh.f.entry.Attributes.Mime = fh.contentType
- if fh.f.entry.Attributes.Uid == 0 {
- fh.f.entry.Attributes.Uid = header.Uid
+ entry := fh.f.getEntry()
+ if entry == nil {
+ return nil
+ }
+
+ if entry.Attributes != nil {
+ entry.Attributes.Mime = fh.contentType
+ if entry.Attributes.Uid == 0 {
+ entry.Attributes.Uid = header.Uid
}
- if fh.f.entry.Attributes.Gid == 0 {
- fh.f.entry.Attributes.Gid = header.Gid
+ if entry.Attributes.Gid == 0 {
+ entry.Attributes.Gid = header.Gid
}
- if fh.f.entry.Attributes.Crtime == 0 {
- fh.f.entry.Attributes.Crtime = time.Now().Unix()
+ if entry.Attributes.Crtime == 0 {
+ entry.Attributes.Crtime = time.Now().Unix()
}
- fh.f.entry.Attributes.Mtime = time.Now().Unix()
- fh.f.entry.Attributes.FileMode = uint32(os.FileMode(fh.f.entry.Attributes.FileMode) &^ fh.f.wfs.option.Umask)
- fh.f.entry.Attributes.Collection = fh.dirtyPages.collection
- fh.f.entry.Attributes.Replication = fh.dirtyPages.replication
+ entry.Attributes.Mtime = time.Now().Unix()
+ entry.Attributes.FileMode = uint32(os.FileMode(entry.Attributes.FileMode) &^ fh.f.wfs.option.Umask)
+ entry.Attributes.Collection = fh.dirtyPages.collection
+ entry.Attributes.Replication = fh.dirtyPages.replication
}
request := &filer_pb.CreateEntryRequest{
Directory: fh.f.dir.FullPath(),
- Entry: fh.f.entry,
+ Entry: entry,
Signatures: []int32{fh.f.wfs.signature},
}
- glog.V(4).Infof("%s set chunks: %v", fh.f.fullpath(), len(fh.f.entry.Chunks))
- for i, chunk := range fh.f.entry.Chunks {
+ glog.V(4).Infof("%s set chunks: %v", fh.f.fullpath(), len(entry.Chunks))
+ for i, chunk := range entry.Chunks {
glog.V(4).Infof("%s chunks %d: %v [%d,%d)", fh.f.fullpath(), i, chunk.GetFileIdString(), chunk.Offset, chunk.Offset+int64(chunk.Size))
}
- manifestChunks, nonManifestChunks := filer.SeparateManifestChunks(fh.f.entry.Chunks)
+ manifestChunks, nonManifestChunks := filer.SeparateManifestChunks(entry.Chunks)
chunks, _ := filer.CompactFileChunks(fh.f.wfs.LookupFn(), nonManifestChunks)
chunks, manifestErr := filer.MaybeManifestize(fh.f.wfs.saveDataAsChunk(fh.f.fullpath()), chunks)
@@ -278,7 +294,7 @@ func (fh *FileHandle) doFlush(ctx context.Context, header fuse.Header) error {
// not good, but should be ok
glog.V(0).Infof("MaybeManifestize: %v", manifestErr)
}
- fh.f.entry.Chunks = append(chunks, manifestChunks...)
+ entry.Chunks = append(chunks, manifestChunks...)
fh.f.wfs.mapPbIdFromLocalToFiler(request.Entry)
defer fh.f.wfs.mapPbIdFromFilerToLocal(request.Entry)
diff --git a/weed/filesys/fscache.go b/weed/filesys/fscache.go
index fdec8253c..6b1012090 100644
--- a/weed/filesys/fscache.go
+++ b/weed/filesys/fscache.go
@@ -124,8 +124,9 @@ func (c *FsCache) Move(oldPath util.FullPath, newPath util.FullPath) *FsNode {
}
if f, ok := src.node.(*File); ok {
f.Name = target.name
- if f.entry != nil {
- f.entry.Name = f.Name
+ entry := f.getEntry()
+ if entry != nil {
+ entry.Name = f.Name
}
}
parent.disconnectChild(target)
diff --git a/weed/filesys/wfs.go b/weed/filesys/wfs.go
index c6d9080a1..ba5eb4b6b 100644
--- a/weed/filesys/wfs.go
+++ b/weed/filesys/wfs.go
@@ -43,6 +43,7 @@ type Option struct {
DataCenter string
EntryCacheTtl time.Duration
Umask os.FileMode
+ ReadOnly bool
MountUid uint32
MountGid uint32
diff --git a/weed/operation/buffer_pool.go b/weed/operation/buffer_pool.go
new file mode 100644
index 000000000..9cbe4787f
--- /dev/null
+++ b/weed/operation/buffer_pool.go
@@ -0,0 +1,24 @@
+package operation
+
+import (
+ "github.com/valyala/bytebufferpool"
+ "sync/atomic"
+)
+
+var bufferCounter int64
+
+func GetBuffer() *bytebufferpool.ByteBuffer {
+ defer func() {
+ atomic.AddInt64(&bufferCounter, 1)
+ // println("+", bufferCounter)
+ }()
+ return bytebufferpool.Get()
+}
+
+func PutBuffer(buf *bytebufferpool.ByteBuffer) {
+ defer func() {
+ atomic.AddInt64(&bufferCounter, -1)
+ // println("-", bufferCounter)
+ }()
+ bytebufferpool.Put(buf)
+}
diff --git a/weed/operation/upload_content.go b/weed/operation/upload_content.go
index 70428bb07..e891ae03b 100644
--- a/weed/operation/upload_content.go
+++ b/weed/operation/upload_content.go
@@ -19,7 +19,6 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/valyala/bytebufferpool"
)
type UploadResult struct {
@@ -31,6 +30,7 @@ type UploadResult struct {
Mime string `json:"mime,omitempty"`
Gzip uint32 `json:"gzip,omitempty"`
ContentMd5 string `json:"contentMd5,omitempty"`
+ RetryCount int `json:"-"`
}
func (uploadResult *UploadResult) ToPbFileChunk(fileId string, offset int64) *filer_pb.FileChunk {
@@ -96,6 +96,7 @@ func retriedUploadData(uploadUrl string, filename string, cipher bool, data []by
for i := 0; i < 3; i++ {
uploadResult, err = doUploadData(uploadUrl, filename, cipher, data, isInputCompressed, mtype, pairMap, jwt)
if err == nil {
+ uploadResult.RetryCount = i
return
} else {
glog.Warningf("uploading to %s: %v", uploadUrl, err)
@@ -188,8 +189,8 @@ func doUploadData(uploadUrl string, filename string, cipher bool, data []byte, i
}
func upload_content(uploadUrl string, fillBufferFunction func(w io.Writer) error, filename string, isGzipped bool, originalDataSize int, mtype string, pairMap map[string]string, jwt security.EncodedJwt) (*UploadResult, error) {
- buf := bytebufferpool.Get()
- defer bytebufferpool.Put(buf)
+ buf := GetBuffer()
+ defer PutBuffer(buf)
body_writer := multipart.NewWriter(buf)
h := make(textproto.MIMEHeader)
h.Set("Content-Disposition", fmt.Sprintf(`form-data; name="file"; filename="%s"`, fileNameEscaper.Replace(filename)))
diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go
index 607a2f26e..902c39514 100644
--- a/weed/pb/filer_pb/filer.pb.go
+++ b/weed/pb/filer_pb/filer.pb.go
@@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
-// protoc-gen-go v1.24.0
+// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: filer.proto
diff --git a/weed/pb/filer_pb/filer_client.go b/weed/pb/filer_pb/filer_client.go
index 079fbd671..65bd85c84 100644
--- a/weed/pb/filer_pb/filer_client.go
+++ b/weed/pb/filer_pb/filer_client.go
@@ -5,6 +5,7 @@ import (
"errors"
"fmt"
"io"
+ "math"
"os"
"strings"
"time"
@@ -101,12 +102,16 @@ func SeaweedList(client SeaweedFilerClient, parentDirectoryPath, prefix string,
}
func doSeaweedList(client SeaweedFilerClient, fullDirPath util.FullPath, prefix string, fn EachEntryFunciton, startFrom string, inclusive bool, limit uint32) (err error) {
-
+ // Redundancy limit to make it correctly judge whether it is the last file.
+ redLimit := limit
+ if limit != math.MaxInt32 && limit != 0 {
+ redLimit = limit + 1
+ }
request := &ListEntriesRequest{
Directory: string(fullDirPath),
Prefix: prefix,
StartFromFileName: startFrom,
- Limit: limit,
+ Limit: redLimit,
InclusiveStartFrom: inclusive,
}
@@ -119,6 +124,7 @@ func doSeaweedList(client SeaweedFilerClient, fullDirPath util.FullPath, prefix
}
var prevEntry *Entry
+ count := 0
for {
resp, recvErr := stream.Recv()
if recvErr != nil {
@@ -139,6 +145,10 @@ func doSeaweedList(client SeaweedFilerClient, fullDirPath util.FullPath, prefix
}
}
prevEntry = resp.Entry
+ count++
+ if count > int(limit) && limit != 0 {
+ prevEntry = nil
+ }
}
return nil
@@ -172,6 +182,26 @@ func Exists(filerClient FilerClient, parentDirectoryPath string, entryName strin
return
}
+func Touch(filerClient FilerClient, parentDirectoryPath string, entryName string, entry *Entry) (err error) {
+
+ return filerClient.WithFilerClient(func(client SeaweedFilerClient) error {
+
+ request := &UpdateEntryRequest{
+ Directory: parentDirectoryPath,
+ Entry: entry,
+ }
+
+ glog.V(4).Infof("touch entry %v/%v: %v", parentDirectoryPath, entryName, request)
+ if err := UpdateEntry(client, request); err != nil {
+ glog.V(0).Infof("touch exists entry %v: %v", request, err)
+ return fmt.Errorf("touch exists entry %s/%s: %v", parentDirectoryPath, entryName, err)
+ }
+
+ return nil
+ })
+
+}
+
func Mkdir(filerClient FilerClient, parentDirectoryPath string, dirName string, fn func(entry *Entry)) error {
return filerClient.WithFilerClient(func(client SeaweedFilerClient) error {
diff --git a/weed/pb/grpc_client_server.go b/weed/pb/grpc_client_server.go
index 910114313..9efcd9bdc 100644
--- a/weed/pb/grpc_client_server.go
+++ b/weed/pb/grpc_client_server.go
@@ -3,6 +3,7 @@ package pb
import (
"context"
"fmt"
+ "github.com/chrislusf/seaweedfs/weed/glog"
"net/http"
"strconv"
"strings"
@@ -108,51 +109,55 @@ func WithCachedGrpcClient(fn func(*grpc.ClientConn) error, address string, opts
}
func ParseServerToGrpcAddress(server string) (serverGrpcAddress string, err error) {
- colonIndex := strings.LastIndex(server, ":")
- if colonIndex < 0 {
- return "", fmt.Errorf("server should have hostname:port format: %v", server)
- }
+ return ParseServerAddress(server, 10000)
+}
- port, parseErr := strconv.ParseUint(server[colonIndex+1:], 10, 64)
+func ParseServerAddress(server string, deltaPort int) (newServerAddress string, err error) {
+
+ host, port, parseErr := hostAndPort(server)
if parseErr != nil {
return "", fmt.Errorf("server port parse error: %v", parseErr)
}
- grpcPort := int(port) + 10000
+ newPort := int(port) + deltaPort
- return fmt.Sprintf("%s:%d", server[:colonIndex], grpcPort), nil
+ return fmt.Sprintf("%s:%d", host, newPort), nil
}
-func ServerToGrpcAddress(server string) (serverGrpcAddress string) {
- hostnameAndPort := strings.Split(server, ":")
- if len(hostnameAndPort) != 2 {
- return fmt.Sprintf("unexpected server address: %s", server)
+func hostAndPort(address string) (host string, port uint64, err error) {
+ colonIndex := strings.LastIndex(address, ":")
+ if colonIndex < 0 {
+ return "", 0, fmt.Errorf("server should have hostname:port format: %v", address)
+ }
+ port, err = strconv.ParseUint(address[colonIndex+1:], 10, 64)
+ if err != nil {
+ return "", 0, fmt.Errorf("server port parse error: %v", err)
}
- port, parseErr := strconv.ParseUint(hostnameAndPort[1], 10, 64)
+ return address[:colonIndex], port, err
+}
+
+func ServerToGrpcAddress(server string) (serverGrpcAddress string) {
+
+ host, port, parseErr := hostAndPort(server)
if parseErr != nil {
- return fmt.Sprintf("failed to parse port for %s:%s", hostnameAndPort[0], hostnameAndPort[1])
+ glog.Fatalf("server address %s parse error: %v", server, parseErr)
}
grpcPort := int(port) + 10000
- return fmt.Sprintf("%s:%d", hostnameAndPort[0], grpcPort)
+ return fmt.Sprintf("%s:%d", host, grpcPort)
}
func GrpcAddressToServerAddress(grpcAddress string) (serverAddress string) {
- hostnameAndPort := strings.Split(grpcAddress, ":")
- if len(hostnameAndPort) != 2 {
- return fmt.Sprintf("unexpected grpcAddress: %s", grpcAddress)
- }
-
- grpcPort, parseErr := strconv.ParseUint(hostnameAndPort[1], 10, 64)
+ host, grpcPort, parseErr := hostAndPort(grpcAddress)
if parseErr != nil {
- return fmt.Sprintf("failed to parse port for %s:%s", hostnameAndPort[0], hostnameAndPort[1])
+ glog.Fatalf("server grpc address %s parse error: %v", grpcAddress, parseErr)
}
port := int(grpcPort) - 10000
- return fmt.Sprintf("%s:%d", hostnameAndPort[0], port)
+ return fmt.Sprintf("%s:%d", host, port)
}
func WithMasterClient(master string, grpcDialOption grpc.DialOption, fn func(client master_pb.SeaweedClient) error) error {
@@ -197,19 +202,3 @@ func WithGrpcFilerClient(filerGrpcAddress string, grpcDialOption grpc.DialOption
}, filerGrpcAddress, grpcDialOption)
}
-
-func ParseFilerGrpcAddress(filer string) (filerGrpcAddress string, err error) {
- hostnameAndPort := strings.Split(filer, ":")
- if len(hostnameAndPort) != 2 {
- return "", fmt.Errorf("filer should have hostname:port format: %v", hostnameAndPort)
- }
-
- filerPort, parseErr := strconv.ParseUint(hostnameAndPort[1], 10, 64)
- if parseErr != nil {
- return "", fmt.Errorf("filer port parse error: %v", parseErr)
- }
-
- filerGrpcPort := int(filerPort) + 10000
-
- return fmt.Sprintf("%s:%d", hostnameAndPort[0], filerGrpcPort), nil
-}
diff --git a/weed/pb/iam_pb/iam.pb.go b/weed/pb/iam_pb/iam.pb.go
index 93bc854cc..7d0b6281b 100644
--- a/weed/pb/iam_pb/iam.pb.go
+++ b/weed/pb/iam_pb/iam.pb.go
@@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
-// protoc-gen-go v1.24.0
+// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: iam.proto
diff --git a/weed/pb/master_pb/master.pb.go b/weed/pb/master_pb/master.pb.go
index 9b49d2df5..7e1f282dd 100644
--- a/weed/pb/master_pb/master.pb.go
+++ b/weed/pb/master_pb/master.pb.go
@@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
-// protoc-gen-go v1.24.0
+// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: master.proto
diff --git a/weed/pb/messaging_pb/messaging.pb.go b/weed/pb/messaging_pb/messaging.pb.go
index 90b4b724a..591406347 100644
--- a/weed/pb/messaging_pb/messaging.pb.go
+++ b/weed/pb/messaging_pb/messaging.pb.go
@@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
-// protoc-gen-go v1.24.0
+// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: messaging.proto
diff --git a/weed/pb/volume_info.go b/weed/pb/volume_info.go
index c4f733f5c..cae9e018f 100644
--- a/weed/pb/volume_info.go
+++ b/weed/pb/volume_info.go
@@ -15,40 +15,49 @@ import (
)
// MaybeLoadVolumeInfo load the file data as *volume_server_pb.VolumeInfo, the returned volumeInfo will not be nil
-func MaybeLoadVolumeInfo(fileName string) (*volume_server_pb.VolumeInfo, bool, error) {
+func MaybeLoadVolumeInfo(fileName string) (volumeInfo *volume_server_pb.VolumeInfo, hasRemoteFile bool, hasVolumeInfoFile bool, err error) {
- volumeInfo := &volume_server_pb.VolumeInfo{}
+ volumeInfo = &volume_server_pb.VolumeInfo{}
glog.V(1).Infof("maybeLoadVolumeInfo checks %s", fileName)
if exists, canRead, _, _, _ := util.CheckFile(fileName); !exists || !canRead {
if !exists {
- return volumeInfo, false, nil
+ return
}
+ hasVolumeInfoFile = true
if !canRead {
glog.Warningf("can not read %s", fileName)
- return volumeInfo, false, fmt.Errorf("can not read %s", fileName)
+ err = fmt.Errorf("can not read %s", fileName)
+ return
}
- return volumeInfo, false, nil
+ return
}
+ hasVolumeInfoFile = true
+
glog.V(1).Infof("maybeLoadVolumeInfo reads %s", fileName)
tierData, readErr := ioutil.ReadFile(fileName)
if readErr != nil {
glog.Warningf("fail to read %s : %v", fileName, readErr)
- return volumeInfo, false, fmt.Errorf("fail to read %s : %v", fileName, readErr)
+ err = fmt.Errorf("fail to read %s : %v", fileName, readErr)
+ return
+
}
glog.V(1).Infof("maybeLoadVolumeInfo Unmarshal volume info %v", fileName)
- if err := jsonpb.Unmarshal(bytes.NewReader(tierData), volumeInfo); err != nil {
+ if err = jsonpb.Unmarshal(bytes.NewReader(tierData), volumeInfo); err != nil {
glog.Warningf("unmarshal error: %v", err)
- return volumeInfo, false, fmt.Errorf("unmarshal error: %v", err)
+ err = fmt.Errorf("unmarshal error: %v", err)
+ return
}
if len(volumeInfo.GetFiles()) == 0 {
- return volumeInfo, false, nil
+ return
}
- return volumeInfo, true, nil
+ hasRemoteFile = true
+
+ return
}
func SaveVolumeInfo(fileName string, volumeInfo *volume_server_pb.VolumeInfo) error {
diff --git a/weed/pb/volume_server.proto b/weed/pb/volume_server.proto
index 7faf5d4f5..f9836c402 100644
--- a/weed/pb/volume_server.proto
+++ b/weed/pb/volume_server.proto
@@ -52,6 +52,11 @@ service VolumeServer {
rpc CopyFile (CopyFileRequest) returns (stream CopyFileResponse) {
}
+ rpc ReadNeedleBlob (ReadNeedleBlobRequest) returns (ReadNeedleBlobResponse) {
+ }
+ rpc WriteNeedleBlob (WriteNeedleBlobRequest) returns (WriteNeedleBlobResponse) {
+ }
+
rpc VolumeTailSender (VolumeTailSenderRequest) returns (stream VolumeTailSenderResponse) {
}
rpc VolumeTailReceiver (VolumeTailReceiverRequest) returns (VolumeTailReceiverResponse) {
@@ -253,6 +258,25 @@ message CopyFileResponse {
bytes file_content = 1;
}
+message ReadNeedleBlobRequest {
+ uint32 volume_id = 1;
+ uint64 needle_id = 2;
+ int64 offset = 3; // actual offset
+ int32 size = 4;
+}
+message ReadNeedleBlobResponse {
+ bytes needle_blob = 1;
+}
+
+message WriteNeedleBlobRequest {
+ uint32 volume_id = 1;
+ uint64 needle_id = 2;
+ int32 size = 3;
+ bytes needle_blob = 4;
+}
+message WriteNeedleBlobResponse {
+}
+
message VolumeTailSenderRequest {
uint32 volume_id = 1;
uint64 since_ns = 2;
diff --git a/weed/pb/volume_server_pb/volume_server.pb.go b/weed/pb/volume_server_pb/volume_server.pb.go
index dbcc738e5..c642142ba 100644
--- a/weed/pb/volume_server_pb/volume_server.pb.go
+++ b/weed/pb/volume_server_pb/volume_server.pb.go
@@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
-// protoc-gen-go v1.24.0
+// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: volume_server.proto
@@ -1973,6 +1973,233 @@ func (x *CopyFileResponse) GetFileContent() []byte {
return nil
}
+type ReadNeedleBlobRequest 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"` // actual offset
+ Size int32 `protobuf:"varint,4,opt,name=size,proto3" json:"size,omitempty"`
+}
+
+func (x *ReadNeedleBlobRequest) Reset() {
+ *x = ReadNeedleBlobRequest{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_volume_server_proto_msgTypes[38]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *ReadNeedleBlobRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ReadNeedleBlobRequest) ProtoMessage() {}
+
+func (x *ReadNeedleBlobRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_volume_server_proto_msgTypes[38]
+ 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 ReadNeedleBlobRequest.ProtoReflect.Descriptor instead.
+func (*ReadNeedleBlobRequest) Descriptor() ([]byte, []int) {
+ return file_volume_server_proto_rawDescGZIP(), []int{38}
+}
+
+func (x *ReadNeedleBlobRequest) GetVolumeId() uint32 {
+ if x != nil {
+ return x.VolumeId
+ }
+ return 0
+}
+
+func (x *ReadNeedleBlobRequest) GetNeedleId() uint64 {
+ if x != nil {
+ return x.NeedleId
+ }
+ return 0
+}
+
+func (x *ReadNeedleBlobRequest) GetOffset() int64 {
+ if x != nil {
+ return x.Offset
+ }
+ return 0
+}
+
+func (x *ReadNeedleBlobRequest) GetSize() int32 {
+ if x != nil {
+ return x.Size
+ }
+ return 0
+}
+
+type ReadNeedleBlobResponse struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ NeedleBlob []byte `protobuf:"bytes,1,opt,name=needle_blob,json=needleBlob,proto3" json:"needle_blob,omitempty"`
+}
+
+func (x *ReadNeedleBlobResponse) Reset() {
+ *x = ReadNeedleBlobResponse{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_volume_server_proto_msgTypes[39]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *ReadNeedleBlobResponse) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ReadNeedleBlobResponse) ProtoMessage() {}
+
+func (x *ReadNeedleBlobResponse) ProtoReflect() protoreflect.Message {
+ mi := &file_volume_server_proto_msgTypes[39]
+ 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 ReadNeedleBlobResponse.ProtoReflect.Descriptor instead.
+func (*ReadNeedleBlobResponse) Descriptor() ([]byte, []int) {
+ return file_volume_server_proto_rawDescGZIP(), []int{39}
+}
+
+func (x *ReadNeedleBlobResponse) GetNeedleBlob() []byte {
+ if x != nil {
+ return x.NeedleBlob
+ }
+ return nil
+}
+
+type WriteNeedleBlobRequest 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"`
+ Size int32 `protobuf:"varint,3,opt,name=size,proto3" json:"size,omitempty"`
+ NeedleBlob []byte `protobuf:"bytes,4,opt,name=needle_blob,json=needleBlob,proto3" json:"needle_blob,omitempty"`
+}
+
+func (x *WriteNeedleBlobRequest) Reset() {
+ *x = WriteNeedleBlobRequest{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_volume_server_proto_msgTypes[40]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *WriteNeedleBlobRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*WriteNeedleBlobRequest) ProtoMessage() {}
+
+func (x *WriteNeedleBlobRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_volume_server_proto_msgTypes[40]
+ 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 WriteNeedleBlobRequest.ProtoReflect.Descriptor instead.
+func (*WriteNeedleBlobRequest) Descriptor() ([]byte, []int) {
+ return file_volume_server_proto_rawDescGZIP(), []int{40}
+}
+
+func (x *WriteNeedleBlobRequest) GetVolumeId() uint32 {
+ if x != nil {
+ return x.VolumeId
+ }
+ return 0
+}
+
+func (x *WriteNeedleBlobRequest) GetNeedleId() uint64 {
+ if x != nil {
+ return x.NeedleId
+ }
+ return 0
+}
+
+func (x *WriteNeedleBlobRequest) GetSize() int32 {
+ if x != nil {
+ return x.Size
+ }
+ return 0
+}
+
+func (x *WriteNeedleBlobRequest) GetNeedleBlob() []byte {
+ if x != nil {
+ return x.NeedleBlob
+ }
+ return nil
+}
+
+type WriteNeedleBlobResponse struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+}
+
+func (x *WriteNeedleBlobResponse) Reset() {
+ *x = WriteNeedleBlobResponse{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_volume_server_proto_msgTypes[41]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *WriteNeedleBlobResponse) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*WriteNeedleBlobResponse) ProtoMessage() {}
+
+func (x *WriteNeedleBlobResponse) ProtoReflect() protoreflect.Message {
+ mi := &file_volume_server_proto_msgTypes[41]
+ 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 WriteNeedleBlobResponse.ProtoReflect.Descriptor instead.
+func (*WriteNeedleBlobResponse) Descriptor() ([]byte, []int) {
+ return file_volume_server_proto_rawDescGZIP(), []int{41}
+}
+
type VolumeTailSenderRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1986,7 +2213,7 @@ type VolumeTailSenderRequest struct {
func (x *VolumeTailSenderRequest) Reset() {
*x = VolumeTailSenderRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[38]
+ mi := &file_volume_server_proto_msgTypes[42]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1999,7 +2226,7 @@ func (x *VolumeTailSenderRequest) String() string {
func (*VolumeTailSenderRequest) ProtoMessage() {}
func (x *VolumeTailSenderRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[38]
+ mi := &file_volume_server_proto_msgTypes[42]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2012,7 +2239,7 @@ func (x *VolumeTailSenderRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeTailSenderRequest.ProtoReflect.Descriptor instead.
func (*VolumeTailSenderRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{38}
+ return file_volume_server_proto_rawDescGZIP(), []int{42}
}
func (x *VolumeTailSenderRequest) GetVolumeId() uint32 {
@@ -2049,7 +2276,7 @@ type VolumeTailSenderResponse struct {
func (x *VolumeTailSenderResponse) Reset() {
*x = VolumeTailSenderResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[39]
+ mi := &file_volume_server_proto_msgTypes[43]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2062,7 +2289,7 @@ func (x *VolumeTailSenderResponse) String() string {
func (*VolumeTailSenderResponse) ProtoMessage() {}
func (x *VolumeTailSenderResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[39]
+ mi := &file_volume_server_proto_msgTypes[43]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2075,7 +2302,7 @@ func (x *VolumeTailSenderResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeTailSenderResponse.ProtoReflect.Descriptor instead.
func (*VolumeTailSenderResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{39}
+ return file_volume_server_proto_rawDescGZIP(), []int{43}
}
func (x *VolumeTailSenderResponse) GetNeedleHeader() []byte {
@@ -2113,7 +2340,7 @@ type VolumeTailReceiverRequest struct {
func (x *VolumeTailReceiverRequest) Reset() {
*x = VolumeTailReceiverRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[40]
+ mi := &file_volume_server_proto_msgTypes[44]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2126,7 +2353,7 @@ func (x *VolumeTailReceiverRequest) String() string {
func (*VolumeTailReceiverRequest) ProtoMessage() {}
func (x *VolumeTailReceiverRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[40]
+ mi := &file_volume_server_proto_msgTypes[44]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2139,7 +2366,7 @@ func (x *VolumeTailReceiverRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeTailReceiverRequest.ProtoReflect.Descriptor instead.
func (*VolumeTailReceiverRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{40}
+ return file_volume_server_proto_rawDescGZIP(), []int{44}
}
func (x *VolumeTailReceiverRequest) GetVolumeId() uint32 {
@@ -2179,7 +2406,7 @@ type VolumeTailReceiverResponse struct {
func (x *VolumeTailReceiverResponse) Reset() {
*x = VolumeTailReceiverResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[41]
+ mi := &file_volume_server_proto_msgTypes[45]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2192,7 +2419,7 @@ func (x *VolumeTailReceiverResponse) String() string {
func (*VolumeTailReceiverResponse) ProtoMessage() {}
func (x *VolumeTailReceiverResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[41]
+ mi := &file_volume_server_proto_msgTypes[45]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2205,7 +2432,7 @@ func (x *VolumeTailReceiverResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeTailReceiverResponse.ProtoReflect.Descriptor instead.
func (*VolumeTailReceiverResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{41}
+ return file_volume_server_proto_rawDescGZIP(), []int{45}
}
type VolumeEcShardsGenerateRequest struct {
@@ -2220,7 +2447,7 @@ type VolumeEcShardsGenerateRequest struct {
func (x *VolumeEcShardsGenerateRequest) Reset() {
*x = VolumeEcShardsGenerateRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[42]
+ mi := &file_volume_server_proto_msgTypes[46]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2233,7 +2460,7 @@ func (x *VolumeEcShardsGenerateRequest) String() string {
func (*VolumeEcShardsGenerateRequest) ProtoMessage() {}
func (x *VolumeEcShardsGenerateRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[42]
+ mi := &file_volume_server_proto_msgTypes[46]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2246,7 +2473,7 @@ func (x *VolumeEcShardsGenerateRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsGenerateRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsGenerateRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{42}
+ return file_volume_server_proto_rawDescGZIP(), []int{46}
}
func (x *VolumeEcShardsGenerateRequest) GetVolumeId() uint32 {
@@ -2272,7 +2499,7 @@ type VolumeEcShardsGenerateResponse struct {
func (x *VolumeEcShardsGenerateResponse) Reset() {
*x = VolumeEcShardsGenerateResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[43]
+ mi := &file_volume_server_proto_msgTypes[47]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2285,7 +2512,7 @@ func (x *VolumeEcShardsGenerateResponse) String() string {
func (*VolumeEcShardsGenerateResponse) ProtoMessage() {}
func (x *VolumeEcShardsGenerateResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[43]
+ mi := &file_volume_server_proto_msgTypes[47]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2298,7 +2525,7 @@ func (x *VolumeEcShardsGenerateResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsGenerateResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsGenerateResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{43}
+ return file_volume_server_proto_rawDescGZIP(), []int{47}
}
type VolumeEcShardsRebuildRequest struct {
@@ -2313,7 +2540,7 @@ type VolumeEcShardsRebuildRequest struct {
func (x *VolumeEcShardsRebuildRequest) Reset() {
*x = VolumeEcShardsRebuildRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[44]
+ mi := &file_volume_server_proto_msgTypes[48]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2326,7 +2553,7 @@ func (x *VolumeEcShardsRebuildRequest) String() string {
func (*VolumeEcShardsRebuildRequest) ProtoMessage() {}
func (x *VolumeEcShardsRebuildRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[44]
+ mi := &file_volume_server_proto_msgTypes[48]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2339,7 +2566,7 @@ func (x *VolumeEcShardsRebuildRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsRebuildRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsRebuildRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{44}
+ return file_volume_server_proto_rawDescGZIP(), []int{48}
}
func (x *VolumeEcShardsRebuildRequest) GetVolumeId() uint32 {
@@ -2367,7 +2594,7 @@ type VolumeEcShardsRebuildResponse struct {
func (x *VolumeEcShardsRebuildResponse) Reset() {
*x = VolumeEcShardsRebuildResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[45]
+ mi := &file_volume_server_proto_msgTypes[49]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2380,7 +2607,7 @@ func (x *VolumeEcShardsRebuildResponse) String() string {
func (*VolumeEcShardsRebuildResponse) ProtoMessage() {}
func (x *VolumeEcShardsRebuildResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[45]
+ mi := &file_volume_server_proto_msgTypes[49]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2393,7 +2620,7 @@ func (x *VolumeEcShardsRebuildResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsRebuildResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsRebuildResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{45}
+ return file_volume_server_proto_rawDescGZIP(), []int{49}
}
func (x *VolumeEcShardsRebuildResponse) GetRebuiltShardIds() []uint32 {
@@ -2420,7 +2647,7 @@ type VolumeEcShardsCopyRequest struct {
func (x *VolumeEcShardsCopyRequest) Reset() {
*x = VolumeEcShardsCopyRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[46]
+ mi := &file_volume_server_proto_msgTypes[50]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2433,7 +2660,7 @@ func (x *VolumeEcShardsCopyRequest) String() string {
func (*VolumeEcShardsCopyRequest) ProtoMessage() {}
func (x *VolumeEcShardsCopyRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[46]
+ mi := &file_volume_server_proto_msgTypes[50]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2446,7 +2673,7 @@ func (x *VolumeEcShardsCopyRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsCopyRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsCopyRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{46}
+ return file_volume_server_proto_rawDescGZIP(), []int{50}
}
func (x *VolumeEcShardsCopyRequest) GetVolumeId() uint32 {
@@ -2507,7 +2734,7 @@ type VolumeEcShardsCopyResponse struct {
func (x *VolumeEcShardsCopyResponse) Reset() {
*x = VolumeEcShardsCopyResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[47]
+ mi := &file_volume_server_proto_msgTypes[51]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2520,7 +2747,7 @@ func (x *VolumeEcShardsCopyResponse) String() string {
func (*VolumeEcShardsCopyResponse) ProtoMessage() {}
func (x *VolumeEcShardsCopyResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[47]
+ mi := &file_volume_server_proto_msgTypes[51]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2533,7 +2760,7 @@ func (x *VolumeEcShardsCopyResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsCopyResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsCopyResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{47}
+ return file_volume_server_proto_rawDescGZIP(), []int{51}
}
type VolumeEcShardsDeleteRequest struct {
@@ -2549,7 +2776,7 @@ type VolumeEcShardsDeleteRequest struct {
func (x *VolumeEcShardsDeleteRequest) Reset() {
*x = VolumeEcShardsDeleteRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[48]
+ mi := &file_volume_server_proto_msgTypes[52]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2562,7 +2789,7 @@ func (x *VolumeEcShardsDeleteRequest) String() string {
func (*VolumeEcShardsDeleteRequest) ProtoMessage() {}
func (x *VolumeEcShardsDeleteRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[48]
+ mi := &file_volume_server_proto_msgTypes[52]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2575,7 +2802,7 @@ func (x *VolumeEcShardsDeleteRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsDeleteRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsDeleteRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{48}
+ return file_volume_server_proto_rawDescGZIP(), []int{52}
}
func (x *VolumeEcShardsDeleteRequest) GetVolumeId() uint32 {
@@ -2608,7 +2835,7 @@ type VolumeEcShardsDeleteResponse struct {
func (x *VolumeEcShardsDeleteResponse) Reset() {
*x = VolumeEcShardsDeleteResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[49]
+ mi := &file_volume_server_proto_msgTypes[53]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2621,7 +2848,7 @@ func (x *VolumeEcShardsDeleteResponse) String() string {
func (*VolumeEcShardsDeleteResponse) ProtoMessage() {}
func (x *VolumeEcShardsDeleteResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[49]
+ mi := &file_volume_server_proto_msgTypes[53]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2634,7 +2861,7 @@ func (x *VolumeEcShardsDeleteResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsDeleteResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsDeleteResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{49}
+ return file_volume_server_proto_rawDescGZIP(), []int{53}
}
type VolumeEcShardsMountRequest struct {
@@ -2650,7 +2877,7 @@ type VolumeEcShardsMountRequest struct {
func (x *VolumeEcShardsMountRequest) Reset() {
*x = VolumeEcShardsMountRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[50]
+ mi := &file_volume_server_proto_msgTypes[54]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2663,7 +2890,7 @@ func (x *VolumeEcShardsMountRequest) String() string {
func (*VolumeEcShardsMountRequest) ProtoMessage() {}
func (x *VolumeEcShardsMountRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[50]
+ mi := &file_volume_server_proto_msgTypes[54]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2676,7 +2903,7 @@ func (x *VolumeEcShardsMountRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsMountRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsMountRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{50}
+ return file_volume_server_proto_rawDescGZIP(), []int{54}
}
func (x *VolumeEcShardsMountRequest) GetVolumeId() uint32 {
@@ -2709,7 +2936,7 @@ type VolumeEcShardsMountResponse struct {
func (x *VolumeEcShardsMountResponse) Reset() {
*x = VolumeEcShardsMountResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[51]
+ mi := &file_volume_server_proto_msgTypes[55]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2722,7 +2949,7 @@ func (x *VolumeEcShardsMountResponse) String() string {
func (*VolumeEcShardsMountResponse) ProtoMessage() {}
func (x *VolumeEcShardsMountResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[51]
+ mi := &file_volume_server_proto_msgTypes[55]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2735,7 +2962,7 @@ func (x *VolumeEcShardsMountResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsMountResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsMountResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{51}
+ return file_volume_server_proto_rawDescGZIP(), []int{55}
}
type VolumeEcShardsUnmountRequest struct {
@@ -2750,7 +2977,7 @@ type VolumeEcShardsUnmountRequest struct {
func (x *VolumeEcShardsUnmountRequest) Reset() {
*x = VolumeEcShardsUnmountRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[52]
+ mi := &file_volume_server_proto_msgTypes[56]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2763,7 +2990,7 @@ func (x *VolumeEcShardsUnmountRequest) String() string {
func (*VolumeEcShardsUnmountRequest) ProtoMessage() {}
func (x *VolumeEcShardsUnmountRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[52]
+ mi := &file_volume_server_proto_msgTypes[56]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2776,7 +3003,7 @@ func (x *VolumeEcShardsUnmountRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsUnmountRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsUnmountRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{52}
+ return file_volume_server_proto_rawDescGZIP(), []int{56}
}
func (x *VolumeEcShardsUnmountRequest) GetVolumeId() uint32 {
@@ -2802,7 +3029,7 @@ type VolumeEcShardsUnmountResponse struct {
func (x *VolumeEcShardsUnmountResponse) Reset() {
*x = VolumeEcShardsUnmountResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[53]
+ mi := &file_volume_server_proto_msgTypes[57]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2815,7 +3042,7 @@ func (x *VolumeEcShardsUnmountResponse) String() string {
func (*VolumeEcShardsUnmountResponse) ProtoMessage() {}
func (x *VolumeEcShardsUnmountResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[53]
+ mi := &file_volume_server_proto_msgTypes[57]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2828,7 +3055,7 @@ func (x *VolumeEcShardsUnmountResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsUnmountResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsUnmountResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{53}
+ return file_volume_server_proto_rawDescGZIP(), []int{57}
}
type VolumeEcShardReadRequest struct {
@@ -2846,7 +3073,7 @@ type VolumeEcShardReadRequest struct {
func (x *VolumeEcShardReadRequest) Reset() {
*x = VolumeEcShardReadRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[54]
+ mi := &file_volume_server_proto_msgTypes[58]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2859,7 +3086,7 @@ func (x *VolumeEcShardReadRequest) String() string {
func (*VolumeEcShardReadRequest) ProtoMessage() {}
func (x *VolumeEcShardReadRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[54]
+ mi := &file_volume_server_proto_msgTypes[58]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2872,7 +3099,7 @@ func (x *VolumeEcShardReadRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardReadRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardReadRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{54}
+ return file_volume_server_proto_rawDescGZIP(), []int{58}
}
func (x *VolumeEcShardReadRequest) GetVolumeId() uint32 {
@@ -2922,7 +3149,7 @@ type VolumeEcShardReadResponse struct {
func (x *VolumeEcShardReadResponse) Reset() {
*x = VolumeEcShardReadResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[55]
+ mi := &file_volume_server_proto_msgTypes[59]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2935,7 +3162,7 @@ func (x *VolumeEcShardReadResponse) String() string {
func (*VolumeEcShardReadResponse) ProtoMessage() {}
func (x *VolumeEcShardReadResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[55]
+ mi := &file_volume_server_proto_msgTypes[59]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2948,7 +3175,7 @@ func (x *VolumeEcShardReadResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardReadResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardReadResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{55}
+ return file_volume_server_proto_rawDescGZIP(), []int{59}
}
func (x *VolumeEcShardReadResponse) GetData() []byte {
@@ -2979,7 +3206,7 @@ type VolumeEcBlobDeleteRequest struct {
func (x *VolumeEcBlobDeleteRequest) Reset() {
*x = VolumeEcBlobDeleteRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[56]
+ mi := &file_volume_server_proto_msgTypes[60]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2992,7 +3219,7 @@ func (x *VolumeEcBlobDeleteRequest) String() string {
func (*VolumeEcBlobDeleteRequest) ProtoMessage() {}
func (x *VolumeEcBlobDeleteRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[56]
+ mi := &file_volume_server_proto_msgTypes[60]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3005,7 +3232,7 @@ func (x *VolumeEcBlobDeleteRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcBlobDeleteRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcBlobDeleteRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{56}
+ return file_volume_server_proto_rawDescGZIP(), []int{60}
}
func (x *VolumeEcBlobDeleteRequest) GetVolumeId() uint32 {
@@ -3045,7 +3272,7 @@ type VolumeEcBlobDeleteResponse struct {
func (x *VolumeEcBlobDeleteResponse) Reset() {
*x = VolumeEcBlobDeleteResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[57]
+ mi := &file_volume_server_proto_msgTypes[61]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3058,7 +3285,7 @@ func (x *VolumeEcBlobDeleteResponse) String() string {
func (*VolumeEcBlobDeleteResponse) ProtoMessage() {}
func (x *VolumeEcBlobDeleteResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[57]
+ mi := &file_volume_server_proto_msgTypes[61]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3071,7 +3298,7 @@ func (x *VolumeEcBlobDeleteResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcBlobDeleteResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcBlobDeleteResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{57}
+ return file_volume_server_proto_rawDescGZIP(), []int{61}
}
type VolumeEcShardsToVolumeRequest struct {
@@ -3086,7 +3313,7 @@ type VolumeEcShardsToVolumeRequest struct {
func (x *VolumeEcShardsToVolumeRequest) Reset() {
*x = VolumeEcShardsToVolumeRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[58]
+ mi := &file_volume_server_proto_msgTypes[62]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3099,7 +3326,7 @@ func (x *VolumeEcShardsToVolumeRequest) String() string {
func (*VolumeEcShardsToVolumeRequest) ProtoMessage() {}
func (x *VolumeEcShardsToVolumeRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[58]
+ mi := &file_volume_server_proto_msgTypes[62]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3112,7 +3339,7 @@ func (x *VolumeEcShardsToVolumeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsToVolumeRequest.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsToVolumeRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{58}
+ return file_volume_server_proto_rawDescGZIP(), []int{62}
}
func (x *VolumeEcShardsToVolumeRequest) GetVolumeId() uint32 {
@@ -3138,7 +3365,7 @@ type VolumeEcShardsToVolumeResponse struct {
func (x *VolumeEcShardsToVolumeResponse) Reset() {
*x = VolumeEcShardsToVolumeResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[59]
+ mi := &file_volume_server_proto_msgTypes[63]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3151,7 +3378,7 @@ func (x *VolumeEcShardsToVolumeResponse) String() string {
func (*VolumeEcShardsToVolumeResponse) ProtoMessage() {}
func (x *VolumeEcShardsToVolumeResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[59]
+ mi := &file_volume_server_proto_msgTypes[63]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3164,7 +3391,7 @@ func (x *VolumeEcShardsToVolumeResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeEcShardsToVolumeResponse.ProtoReflect.Descriptor instead.
func (*VolumeEcShardsToVolumeResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{59}
+ return file_volume_server_proto_rawDescGZIP(), []int{63}
}
type ReadVolumeFileStatusRequest struct {
@@ -3178,7 +3405,7 @@ type ReadVolumeFileStatusRequest struct {
func (x *ReadVolumeFileStatusRequest) Reset() {
*x = ReadVolumeFileStatusRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[60]
+ mi := &file_volume_server_proto_msgTypes[64]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3191,7 +3418,7 @@ func (x *ReadVolumeFileStatusRequest) String() string {
func (*ReadVolumeFileStatusRequest) ProtoMessage() {}
func (x *ReadVolumeFileStatusRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[60]
+ mi := &file_volume_server_proto_msgTypes[64]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3204,7 +3431,7 @@ func (x *ReadVolumeFileStatusRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReadVolumeFileStatusRequest.ProtoReflect.Descriptor instead.
func (*ReadVolumeFileStatusRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{60}
+ return file_volume_server_proto_rawDescGZIP(), []int{64}
}
func (x *ReadVolumeFileStatusRequest) GetVolumeId() uint32 {
@@ -3233,7 +3460,7 @@ type ReadVolumeFileStatusResponse struct {
func (x *ReadVolumeFileStatusResponse) Reset() {
*x = ReadVolumeFileStatusResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[61]
+ mi := &file_volume_server_proto_msgTypes[65]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3246,7 +3473,7 @@ func (x *ReadVolumeFileStatusResponse) String() string {
func (*ReadVolumeFileStatusResponse) ProtoMessage() {}
func (x *ReadVolumeFileStatusResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[61]
+ mi := &file_volume_server_proto_msgTypes[65]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3259,7 +3486,7 @@ func (x *ReadVolumeFileStatusResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReadVolumeFileStatusResponse.ProtoReflect.Descriptor instead.
func (*ReadVolumeFileStatusResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{61}
+ return file_volume_server_proto_rawDescGZIP(), []int{65}
}
func (x *ReadVolumeFileStatusResponse) GetVolumeId() uint32 {
@@ -3342,7 +3569,7 @@ type DiskStatus struct {
func (x *DiskStatus) Reset() {
*x = DiskStatus{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[62]
+ mi := &file_volume_server_proto_msgTypes[66]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3355,7 +3582,7 @@ func (x *DiskStatus) String() string {
func (*DiskStatus) ProtoMessage() {}
func (x *DiskStatus) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[62]
+ mi := &file_volume_server_proto_msgTypes[66]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3368,7 +3595,7 @@ func (x *DiskStatus) ProtoReflect() protoreflect.Message {
// Deprecated: Use DiskStatus.ProtoReflect.Descriptor instead.
func (*DiskStatus) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{62}
+ return file_volume_server_proto_rawDescGZIP(), []int{66}
}
func (x *DiskStatus) GetDir() string {
@@ -3437,7 +3664,7 @@ type MemStatus struct {
func (x *MemStatus) Reset() {
*x = MemStatus{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[63]
+ mi := &file_volume_server_proto_msgTypes[67]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3450,7 +3677,7 @@ func (x *MemStatus) String() string {
func (*MemStatus) ProtoMessage() {}
func (x *MemStatus) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[63]
+ mi := &file_volume_server_proto_msgTypes[67]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3463,7 +3690,7 @@ func (x *MemStatus) ProtoReflect() protoreflect.Message {
// Deprecated: Use MemStatus.ProtoReflect.Descriptor instead.
func (*MemStatus) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{63}
+ return file_volume_server_proto_rawDescGZIP(), []int{67}
}
func (x *MemStatus) GetGoroutines() int32 {
@@ -3533,7 +3760,7 @@ type RemoteFile struct {
func (x *RemoteFile) Reset() {
*x = RemoteFile{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[64]
+ mi := &file_volume_server_proto_msgTypes[68]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3546,7 +3773,7 @@ func (x *RemoteFile) String() string {
func (*RemoteFile) ProtoMessage() {}
func (x *RemoteFile) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[64]
+ mi := &file_volume_server_proto_msgTypes[68]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3559,7 +3786,7 @@ func (x *RemoteFile) ProtoReflect() protoreflect.Message {
// Deprecated: Use RemoteFile.ProtoReflect.Descriptor instead.
func (*RemoteFile) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{64}
+ return file_volume_server_proto_rawDescGZIP(), []int{68}
}
func (x *RemoteFile) GetBackendType() string {
@@ -3624,7 +3851,7 @@ type VolumeInfo struct {
func (x *VolumeInfo) Reset() {
*x = VolumeInfo{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[65]
+ mi := &file_volume_server_proto_msgTypes[69]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3637,7 +3864,7 @@ func (x *VolumeInfo) String() string {
func (*VolumeInfo) ProtoMessage() {}
func (x *VolumeInfo) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[65]
+ mi := &file_volume_server_proto_msgTypes[69]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3650,7 +3877,7 @@ func (x *VolumeInfo) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeInfo.ProtoReflect.Descriptor instead.
func (*VolumeInfo) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{65}
+ return file_volume_server_proto_rawDescGZIP(), []int{69}
}
func (x *VolumeInfo) GetFiles() []*RemoteFile {
@@ -3688,7 +3915,7 @@ type VolumeTierMoveDatToRemoteRequest struct {
func (x *VolumeTierMoveDatToRemoteRequest) Reset() {
*x = VolumeTierMoveDatToRemoteRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[66]
+ mi := &file_volume_server_proto_msgTypes[70]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3701,7 +3928,7 @@ func (x *VolumeTierMoveDatToRemoteRequest) String() string {
func (*VolumeTierMoveDatToRemoteRequest) ProtoMessage() {}
func (x *VolumeTierMoveDatToRemoteRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[66]
+ mi := &file_volume_server_proto_msgTypes[70]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3714,7 +3941,7 @@ func (x *VolumeTierMoveDatToRemoteRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeTierMoveDatToRemoteRequest.ProtoReflect.Descriptor instead.
func (*VolumeTierMoveDatToRemoteRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{66}
+ return file_volume_server_proto_rawDescGZIP(), []int{70}
}
func (x *VolumeTierMoveDatToRemoteRequest) GetVolumeId() uint32 {
@@ -3757,7 +3984,7 @@ type VolumeTierMoveDatToRemoteResponse struct {
func (x *VolumeTierMoveDatToRemoteResponse) Reset() {
*x = VolumeTierMoveDatToRemoteResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[67]
+ mi := &file_volume_server_proto_msgTypes[71]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3770,7 +3997,7 @@ func (x *VolumeTierMoveDatToRemoteResponse) String() string {
func (*VolumeTierMoveDatToRemoteResponse) ProtoMessage() {}
func (x *VolumeTierMoveDatToRemoteResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[67]
+ mi := &file_volume_server_proto_msgTypes[71]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3783,7 +4010,7 @@ func (x *VolumeTierMoveDatToRemoteResponse) ProtoReflect() protoreflect.Message
// Deprecated: Use VolumeTierMoveDatToRemoteResponse.ProtoReflect.Descriptor instead.
func (*VolumeTierMoveDatToRemoteResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{67}
+ return file_volume_server_proto_rawDescGZIP(), []int{71}
}
func (x *VolumeTierMoveDatToRemoteResponse) GetProcessed() int64 {
@@ -3813,7 +4040,7 @@ type VolumeTierMoveDatFromRemoteRequest struct {
func (x *VolumeTierMoveDatFromRemoteRequest) Reset() {
*x = VolumeTierMoveDatFromRemoteRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[68]
+ mi := &file_volume_server_proto_msgTypes[72]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3826,7 +4053,7 @@ func (x *VolumeTierMoveDatFromRemoteRequest) String() string {
func (*VolumeTierMoveDatFromRemoteRequest) ProtoMessage() {}
func (x *VolumeTierMoveDatFromRemoteRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[68]
+ mi := &file_volume_server_proto_msgTypes[72]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3839,7 +4066,7 @@ func (x *VolumeTierMoveDatFromRemoteRequest) ProtoReflect() protoreflect.Message
// Deprecated: Use VolumeTierMoveDatFromRemoteRequest.ProtoReflect.Descriptor instead.
func (*VolumeTierMoveDatFromRemoteRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{68}
+ return file_volume_server_proto_rawDescGZIP(), []int{72}
}
func (x *VolumeTierMoveDatFromRemoteRequest) GetVolumeId() uint32 {
@@ -3875,7 +4102,7 @@ type VolumeTierMoveDatFromRemoteResponse struct {
func (x *VolumeTierMoveDatFromRemoteResponse) Reset() {
*x = VolumeTierMoveDatFromRemoteResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[69]
+ mi := &file_volume_server_proto_msgTypes[73]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3888,7 +4115,7 @@ func (x *VolumeTierMoveDatFromRemoteResponse) String() string {
func (*VolumeTierMoveDatFromRemoteResponse) ProtoMessage() {}
func (x *VolumeTierMoveDatFromRemoteResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[69]
+ mi := &file_volume_server_proto_msgTypes[73]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3901,7 +4128,7 @@ func (x *VolumeTierMoveDatFromRemoteResponse) ProtoReflect() protoreflect.Messag
// Deprecated: Use VolumeTierMoveDatFromRemoteResponse.ProtoReflect.Descriptor instead.
func (*VolumeTierMoveDatFromRemoteResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{69}
+ return file_volume_server_proto_rawDescGZIP(), []int{73}
}
func (x *VolumeTierMoveDatFromRemoteResponse) GetProcessed() int64 {
@@ -3927,7 +4154,7 @@ type VolumeServerStatusRequest struct {
func (x *VolumeServerStatusRequest) Reset() {
*x = VolumeServerStatusRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[70]
+ mi := &file_volume_server_proto_msgTypes[74]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3940,7 +4167,7 @@ func (x *VolumeServerStatusRequest) String() string {
func (*VolumeServerStatusRequest) ProtoMessage() {}
func (x *VolumeServerStatusRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[70]
+ mi := &file_volume_server_proto_msgTypes[74]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3953,7 +4180,7 @@ func (x *VolumeServerStatusRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeServerStatusRequest.ProtoReflect.Descriptor instead.
func (*VolumeServerStatusRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{70}
+ return file_volume_server_proto_rawDescGZIP(), []int{74}
}
type VolumeServerStatusResponse struct {
@@ -3968,7 +4195,7 @@ type VolumeServerStatusResponse struct {
func (x *VolumeServerStatusResponse) Reset() {
*x = VolumeServerStatusResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[71]
+ mi := &file_volume_server_proto_msgTypes[75]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3981,7 +4208,7 @@ func (x *VolumeServerStatusResponse) String() string {
func (*VolumeServerStatusResponse) ProtoMessage() {}
func (x *VolumeServerStatusResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[71]
+ mi := &file_volume_server_proto_msgTypes[75]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3994,7 +4221,7 @@ func (x *VolumeServerStatusResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeServerStatusResponse.ProtoReflect.Descriptor instead.
func (*VolumeServerStatusResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{71}
+ return file_volume_server_proto_rawDescGZIP(), []int{75}
}
func (x *VolumeServerStatusResponse) GetDiskStatuses() []*DiskStatus {
@@ -4020,7 +4247,7 @@ type VolumeServerLeaveRequest struct {
func (x *VolumeServerLeaveRequest) Reset() {
*x = VolumeServerLeaveRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[72]
+ mi := &file_volume_server_proto_msgTypes[76]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4033,7 +4260,7 @@ func (x *VolumeServerLeaveRequest) String() string {
func (*VolumeServerLeaveRequest) ProtoMessage() {}
func (x *VolumeServerLeaveRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[72]
+ mi := &file_volume_server_proto_msgTypes[76]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4046,7 +4273,7 @@ func (x *VolumeServerLeaveRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeServerLeaveRequest.ProtoReflect.Descriptor instead.
func (*VolumeServerLeaveRequest) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{72}
+ return file_volume_server_proto_rawDescGZIP(), []int{76}
}
type VolumeServerLeaveResponse struct {
@@ -4058,7 +4285,7 @@ type VolumeServerLeaveResponse struct {
func (x *VolumeServerLeaveResponse) Reset() {
*x = VolumeServerLeaveResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[73]
+ mi := &file_volume_server_proto_msgTypes[77]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4071,7 +4298,7 @@ func (x *VolumeServerLeaveResponse) String() string {
func (*VolumeServerLeaveResponse) ProtoMessage() {}
func (x *VolumeServerLeaveResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[73]
+ mi := &file_volume_server_proto_msgTypes[77]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4084,7 +4311,7 @@ func (x *VolumeServerLeaveResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use VolumeServerLeaveResponse.ProtoReflect.Descriptor instead.
func (*VolumeServerLeaveResponse) Descriptor() ([]byte, []int) {
- return file_volume_server_proto_rawDescGZIP(), []int{73}
+ return file_volume_server_proto_rawDescGZIP(), []int{77}
}
// select on volume servers
@@ -4103,7 +4330,7 @@ type QueryRequest struct {
func (x *QueryRequest) Reset() {
*x = QueryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[74]
+ mi := &file_volume_server_proto_msgTypes[78]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4116,7 +4343,7 @@ func (x *QueryRequest) String() string {
func (*QueryRequest) ProtoMessage() {}
func (x *QueryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[74]
+ mi := &file_volume_server_proto_msgTypes[78]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4129,7 +4356,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{74}
+ return file_volume_server_proto_rawDescGZIP(), []int{78}
}
func (x *QueryRequest) GetSelections() []string {
@@ -4178,7 +4405,7 @@ type QueriedStripe struct {
func (x *QueriedStripe) Reset() {
*x = QueriedStripe{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[75]
+ mi := &file_volume_server_proto_msgTypes[79]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4191,7 +4418,7 @@ func (x *QueriedStripe) String() string {
func (*QueriedStripe) ProtoMessage() {}
func (x *QueriedStripe) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[75]
+ mi := &file_volume_server_proto_msgTypes[79]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4204,7 +4431,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{75}
+ return file_volume_server_proto_rawDescGZIP(), []int{79}
}
func (x *QueriedStripe) GetRecords() []byte {
@@ -4226,7 +4453,7 @@ type VolumeNeedleStatusRequest struct {
func (x *VolumeNeedleStatusRequest) Reset() {
*x = VolumeNeedleStatusRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[76]
+ mi := &file_volume_server_proto_msgTypes[80]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4239,7 +4466,7 @@ func (x *VolumeNeedleStatusRequest) String() string {
func (*VolumeNeedleStatusRequest) ProtoMessage() {}
func (x *VolumeNeedleStatusRequest) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[76]
+ mi := &file_volume_server_proto_msgTypes[80]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4252,7 +4479,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{76}
+ return file_volume_server_proto_rawDescGZIP(), []int{80}
}
func (x *VolumeNeedleStatusRequest) GetVolumeId() uint32 {
@@ -4285,7 +4512,7 @@ type VolumeNeedleStatusResponse struct {
func (x *VolumeNeedleStatusResponse) Reset() {
*x = VolumeNeedleStatusResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[77]
+ mi := &file_volume_server_proto_msgTypes[81]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4298,7 +4525,7 @@ func (x *VolumeNeedleStatusResponse) String() string {
func (*VolumeNeedleStatusResponse) ProtoMessage() {}
func (x *VolumeNeedleStatusResponse) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[77]
+ mi := &file_volume_server_proto_msgTypes[81]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4311,7 +4538,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{77}
+ return file_volume_server_proto_rawDescGZIP(), []int{81}
}
func (x *VolumeNeedleStatusResponse) GetNeedleId() uint64 {
@@ -4369,7 +4596,7 @@ type QueryRequest_Filter struct {
func (x *QueryRequest_Filter) Reset() {
*x = QueryRequest_Filter{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[78]
+ mi := &file_volume_server_proto_msgTypes[82]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4382,7 +4609,7 @@ func (x *QueryRequest_Filter) String() string {
func (*QueryRequest_Filter) ProtoMessage() {}
func (x *QueryRequest_Filter) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[78]
+ mi := &file_volume_server_proto_msgTypes[82]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4395,7 +4622,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{74, 0}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 0}
}
func (x *QueryRequest_Filter) GetField() string {
@@ -4434,7 +4661,7 @@ type QueryRequest_InputSerialization struct {
func (x *QueryRequest_InputSerialization) Reset() {
*x = QueryRequest_InputSerialization{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[79]
+ mi := &file_volume_server_proto_msgTypes[83]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4447,7 +4674,7 @@ func (x *QueryRequest_InputSerialization) String() string {
func (*QueryRequest_InputSerialization) ProtoMessage() {}
func (x *QueryRequest_InputSerialization) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[79]
+ mi := &file_volume_server_proto_msgTypes[83]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4460,7 +4687,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{74, 1}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 1}
}
func (x *QueryRequest_InputSerialization) GetCompressionType() string {
@@ -4503,7 +4730,7 @@ type QueryRequest_OutputSerialization struct {
func (x *QueryRequest_OutputSerialization) Reset() {
*x = QueryRequest_OutputSerialization{}
if protoimpl.UnsafeEnabled {
- mi := &file_volume_server_proto_msgTypes[80]
+ mi := &file_volume_server_proto_msgTypes[84]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4516,7 +4743,7 @@ func (x *QueryRequest_OutputSerialization) String() string {
func (*QueryRequest_OutputSerialization) ProtoMessage() {}
func (x *QueryRequest_OutputSerialization) ProtoReflect() protoreflect.Message {
- mi := &file_volume_server_proto_msgTypes[80]
+ mi := &file_volume_server_proto_msgTypes[84]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4529,7 +4756,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{74, 2}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 2}
}
func (x *QueryRequest_OutputSerialization) GetCsvOutput() *QueryRequest_OutputSerialization_CSVOutput {
@@ -4564,7 +4791,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[81]
+ mi := &file_volume_server_proto_msgTypes[85]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4577,7 +4804,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[81]
+ mi := &file_volume_server_proto_msgTypes[85]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4590,7 +4817,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{74, 1, 0}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 0}
}
func (x *QueryRequest_InputSerialization_CSVInput) GetFileHeaderInfo() string {
@@ -4653,7 +4880,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[82]
+ mi := &file_volume_server_proto_msgTypes[86]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4666,7 +4893,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[82]
+ mi := &file_volume_server_proto_msgTypes[86]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4679,7 +4906,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{74, 1, 1}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 1}
}
func (x *QueryRequest_InputSerialization_JSONInput) GetType() string {
@@ -4698,7 +4925,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[83]
+ mi := &file_volume_server_proto_msgTypes[87]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4711,7 +4938,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[83]
+ mi := &file_volume_server_proto_msgTypes[87]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4724,7 +4951,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{74, 1, 2}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 2}
}
type QueryRequest_OutputSerialization_CSVOutput struct {
@@ -4742,7 +4969,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[84]
+ mi := &file_volume_server_proto_msgTypes[88]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4755,7 +4982,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[84]
+ mi := &file_volume_server_proto_msgTypes[88]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4768,7 +4995,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{74, 2, 0}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 2, 0}
}
func (x *QueryRequest_OutputSerialization_CSVOutput) GetQuoteFields() string {
@@ -4817,7 +5044,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[85]
+ mi := &file_volume_server_proto_msgTypes[89]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4830,7 +5057,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[85]
+ mi := &file_volume_server_proto_msgTypes[89]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4843,7 +5070,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{74, 2, 1}
+ return file_volume_server_proto_rawDescGZIP(), []int{78, 2, 1}
}
func (x *QueryRequest_OutputSerialization_JSONOutput) GetRecordDelimiter() string {
@@ -5037,510 +5264,545 @@ var file_volume_server_proto_rawDesc = []byte{
0x6e, 0x64, 0x22, 0x35, 0x0a, 0x10, 0x43, 0x6f, 0x70, 0x79, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x63,
0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x66, 0x69,
- 0x6c, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x17, 0x56, 0x6f,
- 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69, 0x6c, 0x53, 0x65, 0x6e, 0x64, 0x65, 0x72, 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, 0x19, 0x0a, 0x08, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x4e, 0x73, 0x12, 0x30, 0x0a,
- 0x14, 0x69, 0x64, 0x6c, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x65,
- 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x69, 0x64, 0x6c,
- 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22,
- 0x84, 0x01, 0x0a, 0x18, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69, 0x6c, 0x53, 0x65,
- 0x6e, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d,
- 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
- 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x62, 0x6f, 0x64, 0x79,
- 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6f,
- 0x64, 0x79, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x68,
- 0x75, 0x6e, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x4c, 0x61, 0x73,
- 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x22, 0xb7, 0x01, 0x0a, 0x19, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
- 0x65, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71,
+ 0x6c, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x22, 0x7d, 0x0a, 0x15, 0x52, 0x65, 0x61,
+ 0x64, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62, 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, 0x05, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x22, 0x39, 0x0a, 0x16, 0x52, 0x65, 0x61, 0x64,
+ 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x62, 0x6c, 0x6f,
+ 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42,
+ 0x6c, 0x6f, 0x62, 0x22, 0x87, 0x01, 0x0a, 0x16, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65,
+ 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62, 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, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65,
+ 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b,
+ 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x18, 0x04, 0x20, 0x01, 0x28,
+ 0x0c, 0x52, 0x0a, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62, 0x22, 0x19, 0x0a,
+ 0x17, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x83, 0x01, 0x0a, 0x17, 0x56, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69, 0x6c, 0x53, 0x65, 0x6e, 0x64, 0x65, 0x72, 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, 0x19, 0x0a, 0x08, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20,
0x01, 0x28, 0x04, 0x52, 0x07, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x4e, 0x73, 0x12, 0x30, 0x0a, 0x14,
0x69, 0x64, 0x6c, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x65, 0x63,
0x6f, 0x6e, 0x64, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x69, 0x64, 0x6c, 0x65,
- 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30,
- 0x0a, 0x14, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
- 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x73, 0x6f,
- 0x75, 0x72, 0x63, 0x65, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72,
- 0x22, 0x1c, 0x0a, 0x1a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65,
- 0x63, 0x65, 0x69, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5c,
- 0x0a, 0x1d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73,
- 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 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, 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, 0x22, 0x20, 0x0a, 0x1e,
- 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x47, 0x65,
- 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5b,
- 0x0a, 0x1c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73,
- 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b,
+ 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x84,
+ 0x01, 0x0a, 0x18, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69, 0x6c, 0x53, 0x65, 0x6e,
+ 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x6e,
+ 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01,
+ 0x28, 0x0c, 0x52, 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
+ 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x62, 0x6f, 0x64, 0x79, 0x18,
+ 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6f, 0x64,
+ 0x79, 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x68, 0x75,
+ 0x6e, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x4c, 0x61, 0x73, 0x74,
+ 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x22, 0xb7, 0x01, 0x0a, 0x19, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
+ 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x72, 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, 0x19, 0x0a, 0x08, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01,
+ 0x28, 0x04, 0x52, 0x07, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x4e, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x69,
+ 0x64, 0x6c, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x65, 0x63, 0x6f,
+ 0x6e, 0x64, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x69, 0x64, 0x6c, 0x65, 0x54,
+ 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a,
+ 0x14, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73,
+ 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x73, 0x6f, 0x75,
+ 0x72, 0x63, 0x65, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x22,
+ 0x1c, 0x0a, 0x1a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x63,
+ 0x65, 0x69, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5c, 0x0a,
+ 0x1d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x47,
+ 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 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, 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, 0x22, 0x4b, 0x0a, 0x1d, 0x56,
- 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x62,
- 0x75, 0x69, 0x6c, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x11,
- 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x74, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x64,
- 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0f, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x74,
- 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73, 0x22, 0x8b, 0x02, 0x0a, 0x19, 0x56, 0x6f, 0x6c,
- 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x43, 0x6f, 0x70, 0x79, 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, 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, 0x1b, 0x0a, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x64, 0x73,
- 0x18, 0x03, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73,
- 0x12, 0x22, 0x0a, 0x0d, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x65, 0x63, 0x78, 0x5f, 0x66, 0x69, 0x6c,
- 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x45, 0x63, 0x78,
- 0x46, 0x69, 0x6c, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x64,
- 0x61, 0x74, 0x61, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,
- 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x22,
- 0x0a, 0x0d, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x65, 0x63, 0x6a, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18,
- 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x45, 0x63, 0x6a, 0x46, 0x69,
- 0x6c, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x76, 0x69, 0x66, 0x5f, 0x66,
- 0x69, 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x56,
- 0x69, 0x66, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x1c, 0x0a, 0x1a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x43, 0x6f, 0x70, 0x79, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x77, 0x0a, 0x1b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63,
- 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75,
+ 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x20, 0x0a, 0x1e, 0x56,
+ 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x47, 0x65, 0x6e,
+ 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5b, 0x0a,
+ 0x1c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52,
+ 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 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, 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, 0x22, 0x4b, 0x0a, 0x1d, 0x56, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x62, 0x75,
+ 0x69, 0x6c, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x72,
+ 0x65, 0x62, 0x75, 0x69, 0x6c, 0x74, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x64, 0x73,
+ 0x18, 0x01, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0f, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x74, 0x53,
+ 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73, 0x22, 0x8b, 0x02, 0x0a, 0x19, 0x56, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x43, 0x6f, 0x70, 0x79, 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, 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, 0x1b, 0x0a, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18,
+ 0x03, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73, 0x12,
+ 0x22, 0x0a, 0x0d, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x65, 0x63, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65,
+ 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x45, 0x63, 0x78, 0x46,
+ 0x69, 0x6c, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x64, 0x61,
+ 0x74, 0x61, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73,
+ 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x22, 0x0a,
+ 0x0d, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x65, 0x63, 0x6a, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x06,
+ 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x45, 0x63, 0x6a, 0x46, 0x69, 0x6c,
+ 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x76, 0x69, 0x66, 0x5f, 0x66, 0x69,
+ 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x56, 0x69,
+ 0x66, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x1c, 0x0a, 0x1a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45,
+ 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x43, 0x6f, 0x70, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x77, 0x0a, 0x1b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53,
+ 0x68, 0x61, 0x72, 0x64, 0x73, 0x44, 0x65, 0x6c, 0x65, 0x74, 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,
+ 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,
+ 0x1b, 0x0a, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03,
+ 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73, 0x22, 0x1e, 0x0a, 0x1c,
+ 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x44, 0x65,
+ 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x76, 0x0a, 0x1a,
+ 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x4d, 0x6f,
+ 0x75, 0x6e, 0x74, 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, 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, 0x1b, 0x0a, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64,
+ 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61, 0x72,
+ 0x64, 0x49, 0x64, 0x73, 0x22, 0x1d, 0x0a, 0x1b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63,
+ 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x58, 0x0a, 0x1c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53,
+ 0x68, 0x61, 0x72, 0x64, 0x73, 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 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, 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, 0x1b, 0x0a, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20,
- 0x03, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73, 0x22, 0x1e, 0x0a,
- 0x1c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x44,
- 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x76, 0x0a,
- 0x1a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x4d,
- 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x76,
+ 0x03, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73, 0x22, 0x1f, 0x0a,
+ 0x1d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x55,
+ 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x99,
+ 0x01, 0x0a, 0x18, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64,
+ 0x52, 0x65, 0x61, 0x64, 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, 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, 0x1b, 0x0a, 0x09, 0x73, 0x68, 0x61, 0x72,
- 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61,
- 0x72, 0x64, 0x49, 0x64, 0x73, 0x22, 0x1d, 0x0a, 0x1b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45,
- 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x58, 0x0a, 0x1c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63,
- 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 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, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03,
- 0x20, 0x03, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x73, 0x22, 0x1f,
- 0x0a, 0x1d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73,
- 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
- 0x99, 0x01, 0x0a, 0x18, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72,
- 0x64, 0x52, 0x65, 0x61, 0x64, 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, 0x19, 0x0a, 0x08, 0x73, 0x68, 0x61,
- 0x72, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x73, 0x68, 0x61,
- 0x72, 0x64, 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, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x05, 0x20, 0x01,
- 0x28, 0x04, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x4e, 0x0a, 0x19, 0x56,
- 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x61, 0x64,
- 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1d, 0x0a, 0x0a,
- 0x69, 0x73, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08,
- 0x52, 0x09, 0x69, 0x73, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x22, 0x8d, 0x01, 0x0a, 0x19,
- 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x42, 0x6c, 0x6f, 0x62, 0x44, 0x65, 0x6c, 0x65,
- 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c,
+ 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x68, 0x61, 0x72,
+ 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x73, 0x68, 0x61, 0x72,
+ 0x64, 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,
+ 0x19, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28,
+ 0x04, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x4e, 0x0a, 0x19, 0x56, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x61, 0x64, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18,
+ 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x69,
+ 0x73, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52,
+ 0x09, 0x69, 0x73, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x22, 0x8d, 0x01, 0x0a, 0x19, 0x56,
+ 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x42, 0x6c, 0x6f, 0x62, 0x44, 0x65, 0x6c, 0x65, 0x74,
+ 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, 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, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b, 0x65,
+ 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x4b, 0x65, 0x79,
+ 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28,
+ 0x0d, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x1c, 0x0a, 0x1a, 0x56, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x42, 0x6c, 0x6f, 0x62, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5c, 0x0a, 0x1d, 0x56, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x54, 0x6f, 0x56, 0x6f, 0x6c, 0x75,
+ 0x6d, 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, 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, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6b,
- 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x4b, 0x65,
- 0x79, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01,
- 0x28, 0x0d, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x1c, 0x0a, 0x1a, 0x56,
- 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x63, 0x42, 0x6c, 0x6f, 0x62, 0x44, 0x65, 0x6c, 0x65, 0x74,
- 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5c, 0x0a, 0x1d, 0x56, 0x6f, 0x6c,
- 0x75, 0x6d, 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x54, 0x6f, 0x56, 0x6f, 0x6c,
- 0x75, 0x6d, 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, 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, 0x22, 0x20, 0x0a, 0x1e, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
- 0x65, 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x54, 0x6f, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
- 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3a, 0x0a, 0x1b, 0x52, 0x65, 0x61,
- 0x64, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75,
- 0x73, 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, 0x22, 0x8a, 0x03, 0x0a, 0x1c, 0x52, 0x65, 0x61, 0x64, 0x56, 0x6f,
- 0x6c, 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 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, 0x3b, 0x0a, 0x1a, 0x69, 0x64, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f,
+ 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x20, 0x0a, 0x1e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
+ 0x45, 0x63, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x54, 0x6f, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3a, 0x0a, 0x1b, 0x52, 0x65, 0x61, 0x64,
+ 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
+ 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, 0x22, 0x8a, 0x03, 0x0a, 0x1c, 0x52, 0x65, 0x61, 0x64, 0x56, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 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, 0x3b, 0x0a, 0x1a, 0x69, 0x64, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x74,
+ 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73,
+ 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x17, 0x69, 0x64, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x54,
+ 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12,
+ 0x22, 0x0a, 0x0d, 0x69, 0x64, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65,
+ 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x69, 0x64, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x53,
+ 0x69, 0x7a, 0x65, 0x12, 0x3b, 0x0a, 0x1a, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f,
0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64,
- 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x17, 0x69, 0x64, 0x78, 0x46, 0x69, 0x6c, 0x65,
+ 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x17, 0x64, 0x61, 0x74, 0x46, 0x69, 0x6c, 0x65,
0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73,
- 0x12, 0x22, 0x0a, 0x0d, 0x69, 0x64, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a,
- 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x69, 0x64, 0x78, 0x46, 0x69, 0x6c, 0x65,
- 0x53, 0x69, 0x7a, 0x65, 0x12, 0x3b, 0x0a, 0x1a, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65,
- 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e,
- 0x64, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x17, 0x64, 0x61, 0x74, 0x46, 0x69, 0x6c,
- 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64,
- 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69,
- 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x64, 0x61, 0x74, 0x46, 0x69, 0x6c,
- 0x65, 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, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69,
- 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28,
- 0x0d, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x76,
- 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
- 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
- 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79,
- 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79,
- 0x70, 0x65, 0x22, 0xbb, 0x01, 0x0a, 0x0a, 0x44, 0x69, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75,
- 0x73, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03,
- 0x64, 0x69, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04,
- 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x64, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x04, 0x52, 0x04, 0x75, 0x73, 0x65, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x72, 0x65,
- 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x66, 0x72, 0x65, 0x65, 0x12, 0x21, 0x0a,
- 0x0c, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x66, 0x72, 0x65, 0x65, 0x18, 0x05, 0x20,
- 0x01, 0x28, 0x02, 0x52, 0x0b, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x46, 0x72, 0x65, 0x65,
- 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x75, 0x73, 0x65, 0x64,
- 0x18, 0x06, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0b, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x55,
- 0x73, 0x65, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65,
- 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65,
- 0x22, 0xa3, 0x01, 0x0a, 0x09, 0x4d, 0x65, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1e,
- 0x0a, 0x0a, 0x67, 0x6f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x05, 0x52, 0x0a, 0x67, 0x6f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x65, 0x73, 0x12, 0x10,
- 0x0a, 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x03, 0x61, 0x6c, 0x6c,
- 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04,
- 0x75, 0x73, 0x65, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x72, 0x65, 0x65, 0x18, 0x04, 0x20, 0x01,
- 0x28, 0x04, 0x52, 0x04, 0x66, 0x72, 0x65, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x65, 0x6c, 0x66,
- 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x65, 0x6c, 0x66, 0x12, 0x12, 0x0a, 0x04,
- 0x68, 0x65, 0x61, 0x70, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x68, 0x65, 0x61, 0x70,
- 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52,
- 0x05, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x22, 0xd8, 0x01, 0x0a, 0x0a, 0x52, 0x65, 0x6d, 0x6f, 0x74,
- 0x65, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64,
- 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x62, 0x61, 0x63,
- 0x6b, 0x65, 0x6e, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x63, 0x6b,
- 0x65, 0x6e, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x62, 0x61,
- 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x49, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x03,
- 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66,
- 0x73, 0x65, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65,
- 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05,
- 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23,
- 0x0a, 0x0d, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18,
- 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54,
- 0x69, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e,
- 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f,
- 0x6e, 0x22, 0x7c, 0x0a, 0x0a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12,
- 0x32, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c,
- 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70,
- 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69,
- 0x6c, 0x65, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 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, 0x22,
- 0xc8, 0x01, 0x0a, 0x20, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x69, 0x65, 0x72, 0x4d, 0x6f,
- 0x76, 0x65, 0x44, 0x61, 0x74, 0x54, 0x6f, 0x52, 0x65, 0x6d, 0x6f, 0x74, 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, 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, 0x38, 0x0a, 0x18, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e,
- 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x16, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e,
- 0x42, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x13, 0x6b,
- 0x65, 0x65, 0x70, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69,
- 0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x6b, 0x65, 0x65, 0x70, 0x4c, 0x6f,
- 0x63, 0x61, 0x6c, 0x44, 0x61, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x73, 0x0a, 0x21, 0x56, 0x6f,
- 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x69, 0x65, 0x72, 0x4d, 0x6f, 0x76, 0x65, 0x44, 0x61, 0x74, 0x54,
- 0x6f, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
- 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x03, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x30, 0x0a,
- 0x13, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e,
- 0x74, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x13, 0x70, 0x72, 0x6f, 0x63,
- 0x65, 0x73, 0x73, 0x65, 0x64, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x22,
- 0x92, 0x01, 0x0a, 0x22, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x69, 0x65, 0x72, 0x4d, 0x6f,
- 0x76, 0x65, 0x44, 0x61, 0x74, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x6d, 0x6f, 0x74, 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, 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, 0x2f, 0x0a, 0x14, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x72, 0x65, 0x6d, 0x6f,
- 0x74, 0x65, 0x5f, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28,
- 0x08, 0x52, 0x11, 0x6b, 0x65, 0x65, 0x70, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x44, 0x61, 0x74,
- 0x46, 0x69, 0x6c, 0x65, 0x22, 0x75, 0x0a, 0x23, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x69,
- 0x65, 0x72, 0x4d, 0x6f, 0x76, 0x65, 0x44, 0x61, 0x74, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x6d,
- 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x70,
- 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09,
- 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x30, 0x0a, 0x13, 0x70, 0x72, 0x6f,
- 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65,
- 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x13, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65,
- 0x64, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x22, 0x1b, 0x0a, 0x19, 0x56,
- 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75,
- 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xa1, 0x01, 0x0a, 0x1a, 0x56, 0x6f, 0x6c,
- 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x41, 0x0a, 0x0d, 0x64, 0x69, 0x73, 0x6b, 0x5f,
- 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c,
- 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70,
- 0x62, 0x2e, 0x44, 0x69, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0c, 0x64, 0x69,
- 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x12, 0x40, 0x0a, 0x0d, 0x6d, 0x65,
- 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28,
- 0x0b, 0x32, 0x1b, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65,
- 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4d, 0x65, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0c,
- 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x1a, 0x0a, 0x18,
- 0x56, 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, 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, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x66,
- 0x69, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x66,
- 0x72, 0x6f, 0x6d, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x66, 0x69,
- 0x6c, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 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, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65,
- 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x13, 0x69, 0x6e, 0x70,
- 0x75, 0x74, 0x5f, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e,
- 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69,
- 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x69, 0x6e, 0x70, 0x75, 0x74,
- 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x65, 0x0a,
- 0x14, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a,
- 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 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, 0x2e, 0x4f, 0x75, 0x74, 0x70,
- 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,
- 0x13, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61,
- 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4e, 0x0a, 0x06, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x14,
- 0x0a, 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x66,
- 0x69, 0x65, 0x6c, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18,
- 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x12, 0x14,
- 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76,
- 0x61, 0x6c, 0x75, 0x65, 0x1a, 0xd5, 0x05, 0x0a, 0x12, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65,
- 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x63,
- 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,
- 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x57, 0x0a, 0x09, 0x63, 0x73, 0x76, 0x5f, 0x69, 0x6e,
- 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x76, 0x6f, 0x6c, 0x75,
+ 0x12, 0x22, 0x0a, 0x0d, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a,
+ 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x64, 0x61, 0x74, 0x46, 0x69, 0x6c, 0x65,
+ 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, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f,
+ 0x6e, 0x5f, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d,
+ 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x76, 0x69,
+ 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
+ 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
+ 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70,
+ 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70,
+ 0x65, 0x22, 0xbb, 0x01, 0x0a, 0x0a, 0x44, 0x69, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
+ 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x64,
+ 0x69, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52,
+ 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01,
+ 0x28, 0x04, 0x52, 0x04, 0x75, 0x73, 0x65, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x72, 0x65, 0x65,
+ 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x66, 0x72, 0x65, 0x65, 0x12, 0x21, 0x0a, 0x0c,
+ 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x66, 0x72, 0x65, 0x65, 0x18, 0x05, 0x20, 0x01,
+ 0x28, 0x02, 0x52, 0x0b, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x46, 0x72, 0x65, 0x65, 0x12,
+ 0x21, 0x0a, 0x0c, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18,
+ 0x06, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0b, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x55, 0x73,
+ 0x65, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18,
+ 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x22,
+ 0xa3, 0x01, 0x0a, 0x09, 0x4d, 0x65, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1e, 0x0a,
+ 0x0a, 0x67, 0x6f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x05, 0x52, 0x0a, 0x67, 0x6f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x65, 0x73, 0x12, 0x10, 0x0a,
+ 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12,
+ 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x75,
+ 0x73, 0x65, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x72, 0x65, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28,
+ 0x04, 0x52, 0x04, 0x66, 0x72, 0x65, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x65, 0x6c, 0x66, 0x18,
+ 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x65, 0x6c, 0x66, 0x12, 0x12, 0x0a, 0x04, 0x68,
+ 0x65, 0x61, 0x70, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x68, 0x65, 0x61, 0x70, 0x12,
+ 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05,
+ 0x73, 0x74, 0x61, 0x63, 0x6b, 0x22, 0xd8, 0x01, 0x0a, 0x0a, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65,
+ 0x46, 0x69, 0x6c, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x5f,
+ 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x62, 0x61, 0x63, 0x6b,
+ 0x65, 0x6e, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x63, 0x6b, 0x65,
+ 0x6e, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x62, 0x61, 0x63,
+ 0x6b, 0x65, 0x6e, 0x64, 0x49, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73,
+ 0x65, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74,
+ 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 0x20,
+ 0x01, 0x28, 0x04, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a,
+ 0x0d, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06,
+ 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69,
+ 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18,
+ 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e,
+ 0x22, 0x7c, 0x0a, 0x0a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x32,
+ 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e,
+ 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62,
+ 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c,
+ 0x65, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20,
+ 0x01, 0x28, 0x0d, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 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, 0x22, 0xc8,
+ 0x01, 0x0a, 0x20, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x69, 0x65, 0x72, 0x4d, 0x6f, 0x76,
+ 0x65, 0x44, 0x61, 0x74, 0x54, 0x6f, 0x52, 0x65, 0x6d, 0x6f, 0x74, 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, 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, 0x38, 0x0a, 0x18, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
+ 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01,
+ 0x28, 0x09, 0x52, 0x16, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42,
+ 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x13, 0x6b, 0x65,
+ 0x65, 0x70, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69, 0x6c,
+ 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x6b, 0x65, 0x65, 0x70, 0x4c, 0x6f, 0x63,
+ 0x61, 0x6c, 0x44, 0x61, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x73, 0x0a, 0x21, 0x56, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x65, 0x54, 0x69, 0x65, 0x72, 0x4d, 0x6f, 0x76, 0x65, 0x44, 0x61, 0x74, 0x54, 0x6f,
+ 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c,
+ 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x03, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x30, 0x0a, 0x13,
+ 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74,
+ 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x13, 0x70, 0x72, 0x6f, 0x63, 0x65,
+ 0x73, 0x73, 0x65, 0x64, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x22, 0x92,
+ 0x01, 0x0a, 0x22, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x69, 0x65, 0x72, 0x4d, 0x6f, 0x76,
+ 0x65, 0x44, 0x61, 0x74, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x6d, 0x6f, 0x74, 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, 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, 0x2f, 0x0a, 0x14, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x74,
+ 0x65, 0x5f, 0x64, 0x61, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08,
+ 0x52, 0x11, 0x6b, 0x65, 0x65, 0x70, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x44, 0x61, 0x74, 0x46,
+ 0x69, 0x6c, 0x65, 0x22, 0x75, 0x0a, 0x23, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x69, 0x65,
+ 0x72, 0x4d, 0x6f, 0x76, 0x65, 0x44, 0x61, 0x74, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x6d, 0x6f,
+ 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72,
+ 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x70,
+ 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x30, 0x0a, 0x13, 0x70, 0x72, 0x6f, 0x63,
+ 0x65, 0x73, 0x73, 0x65, 0x64, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18,
+ 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x13, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64,
+ 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x22, 0x1b, 0x0a, 0x19, 0x56, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
+ 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xa1, 0x01, 0x0a, 0x1a, 0x56, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x41, 0x0a, 0x0d, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x73,
+ 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e,
+ 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62,
+ 0x2e, 0x44, 0x69, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0c, 0x64, 0x69, 0x73,
+ 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x12, 0x40, 0x0a, 0x0d, 0x6d, 0x65, 0x6d,
+ 0x6f, 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x1b, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72,
+ 0x5f, 0x70, 0x62, 0x2e, 0x4d, 0x65, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0c, 0x6d,
+ 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x1a, 0x0a, 0x18, 0x56,
+ 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, 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,
+ 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x66, 0x69,
+ 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x72,
+ 0x6f, 0x6d, 0x46, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x66, 0x69, 0x6c,
+ 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 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, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53,
- 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x53, 0x56,
- 0x49, 0x6e, 0x70, 0x75, 0x74, 0x52, 0x08, 0x63, 0x73, 0x76, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12,
- 0x5a, 0x0a, 0x0a, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x0b, 0x32, 0x3b, 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, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69,
- 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x49, 0x6e, 0x70, 0x75, 0x74,
- 0x52, 0x09, 0x6a, 0x73, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x63, 0x0a, 0x0d, 0x70,
- 0x61, 0x72, 0x71, 0x75, 0x65, 0x74, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76,
+ 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72,
+ 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x13, 0x69, 0x6e, 0x70, 0x75,
+ 0x74, 0x5f, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18,
+ 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61,
+ 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x53,
+ 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x65, 0x0a, 0x14,
+ 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61,
+ 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 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, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75,
+ 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13,
+ 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74,
+ 0x69, 0x6f, 0x6e, 0x1a, 0x4e, 0x0a, 0x06, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x14, 0x0a,
+ 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x66, 0x69,
+ 0x65, 0x6c, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x6e, 0x64, 0x12, 0x14, 0x0a,
+ 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,
+ 0x6c, 0x75, 0x65, 0x1a, 0xd5, 0x05, 0x0a, 0x12, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72,
+ 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f,
+ 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,
+ 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x57, 0x0a, 0x09, 0x63, 0x73, 0x76, 0x5f, 0x69, 0x6e, 0x70,
+ 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 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, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65,
+ 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x53, 0x56, 0x49,
+ 0x6e, 0x70, 0x75, 0x74, 0x52, 0x08, 0x63, 0x73, 0x76, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a,
+ 0x0a, 0x0a, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01,
+ 0x28, 0x0b, 0x32, 0x3b, 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, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a,
- 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x61, 0x72, 0x71, 0x75, 0x65, 0x74, 0x49, 0x6e, 0x70,
- 0x75, 0x74, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x71, 0x75, 0x65, 0x74, 0x49, 0x6e, 0x70, 0x75, 0x74,
- 0x1a, 0xc8, 0x02, 0x0a, 0x08, 0x43, 0x53, 0x56, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x28, 0x0a,
- 0x10, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x66,
- 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x69, 0x6c, 0x65, 0x48, 0x65, 0x61,
- 0x64, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x6f, 0x72,
- 0x64, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x0f, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74,
- 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x64, 0x65, 0x6c, 0x69,
- 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x69, 0x65,
- 0x6c, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x71,
- 0x75, 0x6f, 0x74, 0x65, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x6f, 0x65, 0x72, 0x18,
- 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x43, 0x68, 0x61, 0x72,
- 0x61, 0x63, 0x74, 0x6f, 0x65, 0x72, 0x12, 0x34, 0x0a, 0x16, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f,
- 0x65, 0x73, 0x63, 0x61, 0x70, 0x65, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x65, 0x72,
- 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x45, 0x73, 0x63,
- 0x61, 0x70, 0x65, 0x43, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08,
- 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
- 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x41, 0x0a, 0x1d, 0x61, 0x6c, 0x6c, 0x6f,
- 0x77, 0x5f, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f,
- 0x64, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52,
- 0x1a, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x51, 0x75, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x63, 0x6f,
- 0x72, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x1a, 0x1f, 0x0a, 0x09, 0x4a,
- 0x53, 0x4f, 0x4e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x1a, 0x0e, 0x0a, 0x0c,
- 0x50, 0x61, 0x72, 0x71, 0x75, 0x65, 0x74, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x1a, 0xf1, 0x03, 0x0a,
- 0x13, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61,
- 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, 0x0a, 0x63, 0x73, 0x76, 0x5f, 0x6f, 0x75, 0x74, 0x70,
- 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 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, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53,
- 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x53, 0x56,
- 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x09, 0x63, 0x73, 0x76, 0x4f, 0x75, 0x74, 0x70, 0x75,
- 0x74, 0x12, 0x5e, 0x0a, 0x0b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74,
- 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 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, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72,
- 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x4f,
- 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x0a, 0x6a, 0x73, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75,
- 0x74, 0x1a, 0xe3, 0x01, 0x0a, 0x09, 0x43, 0x53, 0x56, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12,
- 0x21, 0x0a, 0x0c, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x46, 0x69, 0x65, 0x6c,
- 0x64, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x64, 0x65, 0x6c,
- 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65,
- 0x63, 0x6f, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, 0x27, 0x0a,
- 0x0f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72,
- 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x65, 0x6c,
- 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f,
- 0x63, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x6f, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0f, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x43, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x6f, 0x65,
- 0x72, 0x12, 0x34, 0x0a, 0x16, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f, 0x65, 0x73, 0x63, 0x61, 0x70,
- 0x65, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x14, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x45, 0x73, 0x63, 0x61, 0x70, 0x65, 0x43, 0x68,
- 0x61, 0x72, 0x61, 0x63, 0x74, 0x65, 0x72, 0x1a, 0x37, 0x0a, 0x0a, 0x4a, 0x53, 0x4f, 0x4e, 0x4f,
- 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f,
- 0x64, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
- 0x0f, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72,
- 0x22, 0x29, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x69, 0x65, 0x64, 0x53, 0x74, 0x72, 0x69, 0x70,
- 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x0c, 0x52, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x22, 0x55, 0x0a, 0x19, 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, 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, 0x22, 0xae, 0x01, 0x0a, 0x1a, 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, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x16,
- 0x0a, 0x06, 0x63, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06,
- 0x63, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03,
- 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x61,
- 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28,
- 0x04, 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, 0xd8, 0x1f, 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, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 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, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x11, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75,
- 0x6d, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x2a, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x63, 0x75, 0x75,
- 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 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, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c,
- 0x75, 0x6d, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x00, 0x12, 0x74, 0x0a, 0x13, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75,
- 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x12, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75,
- 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x63,
- 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74,
- 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, 0x56, 0x61, 0x63, 0x75, 0x75,
- 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x61, 0x63, 0x75,
- 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x12, 0x2b,
- 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70,
- 0x62, 0x2e, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f,
- 0x6d, 0x6d, 0x69, 0x74, 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,
- 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69,
- 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x74, 0x0a, 0x13, 0x56,
- 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x6e,
- 0x75, 0x70, 0x12, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76,
+ 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x52,
+ 0x09, 0x6a, 0x73, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x63, 0x0a, 0x0d, 0x70, 0x61,
+ 0x72, 0x71, 0x75, 0x65, 0x74, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x3e, 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, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61,
+ 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x61, 0x72, 0x71, 0x75, 0x65, 0x74, 0x49, 0x6e, 0x70, 0x75,
+ 0x74, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x71, 0x75, 0x65, 0x74, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x1a,
+ 0xc8, 0x02, 0x0a, 0x08, 0x43, 0x53, 0x56, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x28, 0x0a, 0x10,
+ 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x66, 0x6f,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x69, 0x6c, 0x65, 0x48, 0x65, 0x61, 0x64,
+ 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64,
+ 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x0f, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65,
+ 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x6d,
+ 0x69, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x69, 0x65, 0x6c,
+ 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x71, 0x75,
+ 0x6f, 0x74, 0x65, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x6f, 0x65, 0x72, 0x18, 0x04,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x43, 0x68, 0x61, 0x72, 0x61,
+ 0x63, 0x74, 0x6f, 0x65, 0x72, 0x12, 0x34, 0x0a, 0x16, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f, 0x65,
+ 0x73, 0x63, 0x61, 0x70, 0x65, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x65, 0x72, 0x18,
+ 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x45, 0x73, 0x63, 0x61,
+ 0x70, 0x65, 0x43, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x63,
+ 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63,
+ 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x41, 0x0a, 0x1d, 0x61, 0x6c, 0x6c, 0x6f, 0x77,
+ 0x5f, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x64,
+ 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1a,
+ 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x51, 0x75, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x63, 0x6f, 0x72,
+ 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x1a, 0x1f, 0x0a, 0x09, 0x4a, 0x53,
+ 0x4f, 0x4e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18,
+ 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x1a, 0x0e, 0x0a, 0x0c, 0x50,
+ 0x61, 0x72, 0x71, 0x75, 0x65, 0x74, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x1a, 0xf1, 0x03, 0x0a, 0x13,
+ 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74,
+ 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, 0x0a, 0x63, 0x73, 0x76, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75,
+ 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 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, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x65,
+ 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x53, 0x56, 0x4f,
+ 0x75, 0x74, 0x70, 0x75, 0x74, 0x52, 0x09, 0x63, 0x73, 0x76, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74,
+ 0x12, 0x5e, 0x0a, 0x0b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18,
+ 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 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, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x65, 0x72, 0x69,
+ 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x75,
+ 0x74, 0x70, 0x75, 0x74, 0x52, 0x0a, 0x6a, 0x73, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74,
+ 0x1a, 0xe3, 0x01, 0x0a, 0x09, 0x43, 0x53, 0x56, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x21,
+ 0x0a, 0x0c, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x46, 0x69, 0x65, 0x6c, 0x64,
+ 0x73, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x64, 0x65, 0x6c, 0x69,
+ 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x63,
+ 0x6f, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f,
+ 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18,
+ 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x65, 0x6c, 0x69,
+ 0x6d, 0x69, 0x74, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f, 0x63,
+ 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x6f, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52,
+ 0x0f, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x43, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x6f, 0x65, 0x72,
+ 0x12, 0x34, 0x0a, 0x16, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x5f, 0x65, 0x73, 0x63, 0x61, 0x70, 0x65,
+ 0x5f, 0x63, 0x68, 0x61, 0x72, 0x61, 0x63, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x14, 0x71, 0x75, 0x6f, 0x74, 0x65, 0x45, 0x73, 0x63, 0x61, 0x70, 0x65, 0x43, 0x68, 0x61,
+ 0x72, 0x61, 0x63, 0x74, 0x65, 0x72, 0x1a, 0x37, 0x0a, 0x0a, 0x4a, 0x53, 0x4f, 0x4e, 0x4f, 0x75,
+ 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x64,
+ 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f,
+ 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x72, 0x22,
+ 0x29, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x69, 0x65, 0x64, 0x53, 0x74, 0x72, 0x69, 0x70, 0x65,
+ 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x0c, 0x52, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x22, 0x55, 0x0a, 0x19, 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, 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, 0x22, 0xae, 0x01, 0x0a, 0x1a, 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,
+ 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
+ 0x01, 0x28, 0x04, 0x52, 0x08, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a,
+ 0x06, 0x63, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x63,
+ 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x0d, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x61, 0x73,
+ 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04,
+ 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,
+ 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,
+ 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 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, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x00, 0x12, 0x6e, 0x0a, 0x11, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x2a, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
+ 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d,
+ 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 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, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75,
- 0x6d, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 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, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
- 0x00, 0x12, 0x6b, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65,
- 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73,
- 0x65, 0x72, 0x76, 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, 0x2a, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 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, 0x65,
- 0x0a, 0x0e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x12, 0x27, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72,
- 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x56, 0x6f, 0x6c, 0x75,
- 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x76, 0x6f, 0x6c, 0x75,
- 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x6c, 0x6c,
- 0x6f, 0x63, 0x61, 0x74, 0x65, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x10, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53,
- 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x29, 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, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 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, 0x79,
- 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x00, 0x12, 0x7c, 0x0a, 0x15, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x6e, 0x63, 0x72,
- 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x2e, 0x2e, 0x76, 0x6f,
+ 0x6d, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x00, 0x12, 0x74, 0x0a, 0x13, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x12, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x63, 0x75,
+ 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 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, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d,
+ 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x61, 0x63, 0x75, 0x75,
+ 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x12, 0x2b, 0x2e,
+ 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62,
+ 0x2e, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d,
+ 0x6d, 0x69, 0x74, 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, 0x61,
+ 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x74, 0x0a, 0x13, 0x56, 0x61,
+ 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x75,
+ 0x70, 0x12, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65,
+ 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 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, 0x56, 0x61, 0x63, 0x75, 0x75, 0x6d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43,
+ 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
+ 0x12, 0x6b, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
+ 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65,
+ 0x72, 0x76, 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,
+ 0x2a, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 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, 0x65, 0x0a,
+ 0x0e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12,
+ 0x27, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f,
+ 0x70, 0x62, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x6c, 0x6c, 0x6f,
+ 0x63, 0x61, 0x74, 0x65, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x10, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x79,
+ 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x29, 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, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x1a, 0x2a, 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, 0x79, 0x6e,
+ 0x63, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x00, 0x12, 0x7c, 0x0a, 0x15, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x6e, 0x63, 0x72, 0x65,
+ 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x2e, 0x2e, 0x76, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x43,
+ 0x6f, 0x70, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x76, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x43,
+ 0x6f, 0x70, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12,
+ 0x5c, 0x0a, 0x0b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x24,
+ 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70,
+ 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65,
+ 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x6f,
+ 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a,
+ 0x0d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x26,
+ 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70,
+ 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x52,
+ 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
+ 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
+ 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x00, 0x12, 0x5f, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74,
+ 0x65, 0x12, 0x25, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65,
+ 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74,
+ 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x00, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x61, 0x72, 0x6b,
+ 0x52, 0x65, 0x61, 0x64, 0x6f, 0x6e, 0x6c, 0x79, 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, 0x4d, 0x61, 0x72, 0x6b, 0x52, 0x65, 0x61, 0x64, 0x6f, 0x6e, 0x6c, 0x79, 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, 0x4d,
+ 0x61, 0x72, 0x6b, 0x52, 0x65, 0x61, 0x64, 0x6f, 0x6e, 0x6c, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d,
+ 0x61, 0x72, 0x6b, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 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, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c,
- 0x43, 0x6f, 0x70, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x76, 0x6f,
+ 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x61, 0x72, 0x6b, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c,
+ 0x65, 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, 0x4d, 0x61, 0x72, 0x6b, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x0f, 0x56, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x12, 0x28, 0x2e, 0x76, 0x6f,
0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56,
- 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c,
- 0x43, 0x6f, 0x70, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01,
- 0x12, 0x5c, 0x0a, 0x0b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x12,
- 0x24, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f,
- 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73,
- 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d,
- 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62,
- 0x0a, 0x0d, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x12,
- 0x26, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f,
- 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
- 0x65, 0x55, 0x6e, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x44, 0x65, 0x6c, 0x65,
- 0x74, 0x65, 0x12, 0x25, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76,
- 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x44, 0x65, 0x6c, 0x65,
- 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x76, 0x6f, 0x6c, 0x75,
+ 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73,
+ 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43,
+ 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x74, 0x61, 0x74,
+ 0x75, 0x73, 0x12, 0x25, 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, 0x74, 0x61, 0x74,
+ 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x76, 0x6f, 0x6c, 0x75,
0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c,
- 0x75, 0x6d, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, 0x61, 0x72,
- 0x6b, 0x52, 0x65, 0x61, 0x64, 0x6f, 0x6e, 0x6c, 0x79, 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, 0x4d, 0x61, 0x72, 0x6b, 0x52, 0x65, 0x61, 0x64, 0x6f, 0x6e, 0x6c, 0x79, 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,
- 0x4d, 0x61, 0x72, 0x6b, 0x52, 0x65, 0x61, 0x64, 0x6f, 0x6e, 0x6c, 0x79, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x4d, 0x61, 0x72, 0x6b, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 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, 0x4d, 0x61, 0x72, 0x6b, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62,
- 0x6c, 0x65, 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, 0x4d, 0x61, 0x72, 0x6b, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x0f, 0x56, 0x6f, 0x6c,
- 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x12, 0x28, 0x2e, 0x76,
- 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
- 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
+ 0x75, 0x6d, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+ 0x65, 0x22, 0x00, 0x12, 0x59, 0x0a, 0x0a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x70,
+ 0x79, 0x12, 0x23, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65,
+ 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x52,
+ 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x74, 0x61,
- 0x74, 0x75, 0x73, 0x12, 0x25, 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, 0x74, 0x61,
- 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 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, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
- 0x73, 0x65, 0x22, 0x00, 0x12, 0x59, 0x0a, 0x0a, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f,
- 0x70, 0x79, 0x12, 0x23, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76,
- 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x43, 0x6f, 0x70, 0x79,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
- 0x65, 0x43, 0x6f, 0x70, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
- 0x77, 0x0a, 0x14, 0x52, 0x65, 0x61, 0x64, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c,
- 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2d, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56,
- 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
+ 0x43, 0x6f, 0x70, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77,
+ 0x0a, 0x14, 0x52, 0x65, 0x61, 0x64, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65,
+ 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2d, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x6f,
0x6c, 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x08, 0x43, 0x6f, 0x70, 0x79,
- 0x46, 0x69, 0x6c, 0x65, 0x12, 0x21, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65,
- 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x46, 0x69, 0x6c, 0x65,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x46,
- 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73,
+ 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x08, 0x43, 0x6f, 0x70, 0x79, 0x46,
+ 0x69, 0x6c, 0x65, 0x12, 0x21, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72,
+ 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x46, 0x69, 0x6c, 0x65, 0x52,
+ 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
+ 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x46, 0x69,
+ 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x65,
+ 0x0a, 0x0e, 0x52, 0x65, 0x61, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62,
+ 0x12, 0x27, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72,
+ 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c,
+ 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x76, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61,
+ 0x64, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x0f, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65,
+ 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62, 0x12, 0x28, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x57, 0x72, 0x69, 0x74,
+ 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x42, 0x6c, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76,
+ 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c,
+ 0x65, 0x42, 0x6c, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x6d, 0x0a, 0x10, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69, 0x6c, 0x53, 0x65, 0x6e,
0x64, 0x65, 0x72, 0x12, 0x29, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72,
0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x61, 0x69,
@@ -5686,7 +5948,7 @@ func file_volume_server_proto_rawDescGZIP() []byte {
return file_volume_server_proto_rawDescData
}
-var file_volume_server_proto_msgTypes = make([]protoimpl.MessageInfo, 86)
+var file_volume_server_proto_msgTypes = make([]protoimpl.MessageInfo, 90)
var file_volume_server_proto_goTypes = []interface{}{
(*BatchDeleteRequest)(nil), // 0: volume_server_pb.BatchDeleteRequest
(*BatchDeleteResponse)(nil), // 1: volume_server_pb.BatchDeleteResponse
@@ -5726,68 +5988,72 @@ var file_volume_server_proto_goTypes = []interface{}{
(*VolumeCopyResponse)(nil), // 35: volume_server_pb.VolumeCopyResponse
(*CopyFileRequest)(nil), // 36: volume_server_pb.CopyFileRequest
(*CopyFileResponse)(nil), // 37: volume_server_pb.CopyFileResponse
- (*VolumeTailSenderRequest)(nil), // 38: volume_server_pb.VolumeTailSenderRequest
- (*VolumeTailSenderResponse)(nil), // 39: volume_server_pb.VolumeTailSenderResponse
- (*VolumeTailReceiverRequest)(nil), // 40: volume_server_pb.VolumeTailReceiverRequest
- (*VolumeTailReceiverResponse)(nil), // 41: volume_server_pb.VolumeTailReceiverResponse
- (*VolumeEcShardsGenerateRequest)(nil), // 42: volume_server_pb.VolumeEcShardsGenerateRequest
- (*VolumeEcShardsGenerateResponse)(nil), // 43: volume_server_pb.VolumeEcShardsGenerateResponse
- (*VolumeEcShardsRebuildRequest)(nil), // 44: volume_server_pb.VolumeEcShardsRebuildRequest
- (*VolumeEcShardsRebuildResponse)(nil), // 45: volume_server_pb.VolumeEcShardsRebuildResponse
- (*VolumeEcShardsCopyRequest)(nil), // 46: volume_server_pb.VolumeEcShardsCopyRequest
- (*VolumeEcShardsCopyResponse)(nil), // 47: volume_server_pb.VolumeEcShardsCopyResponse
- (*VolumeEcShardsDeleteRequest)(nil), // 48: volume_server_pb.VolumeEcShardsDeleteRequest
- (*VolumeEcShardsDeleteResponse)(nil), // 49: volume_server_pb.VolumeEcShardsDeleteResponse
- (*VolumeEcShardsMountRequest)(nil), // 50: volume_server_pb.VolumeEcShardsMountRequest
- (*VolumeEcShardsMountResponse)(nil), // 51: volume_server_pb.VolumeEcShardsMountResponse
- (*VolumeEcShardsUnmountRequest)(nil), // 52: volume_server_pb.VolumeEcShardsUnmountRequest
- (*VolumeEcShardsUnmountResponse)(nil), // 53: volume_server_pb.VolumeEcShardsUnmountResponse
- (*VolumeEcShardReadRequest)(nil), // 54: volume_server_pb.VolumeEcShardReadRequest
- (*VolumeEcShardReadResponse)(nil), // 55: volume_server_pb.VolumeEcShardReadResponse
- (*VolumeEcBlobDeleteRequest)(nil), // 56: volume_server_pb.VolumeEcBlobDeleteRequest
- (*VolumeEcBlobDeleteResponse)(nil), // 57: volume_server_pb.VolumeEcBlobDeleteResponse
- (*VolumeEcShardsToVolumeRequest)(nil), // 58: volume_server_pb.VolumeEcShardsToVolumeRequest
- (*VolumeEcShardsToVolumeResponse)(nil), // 59: volume_server_pb.VolumeEcShardsToVolumeResponse
- (*ReadVolumeFileStatusRequest)(nil), // 60: volume_server_pb.ReadVolumeFileStatusRequest
- (*ReadVolumeFileStatusResponse)(nil), // 61: volume_server_pb.ReadVolumeFileStatusResponse
- (*DiskStatus)(nil), // 62: volume_server_pb.DiskStatus
- (*MemStatus)(nil), // 63: volume_server_pb.MemStatus
- (*RemoteFile)(nil), // 64: volume_server_pb.RemoteFile
- (*VolumeInfo)(nil), // 65: volume_server_pb.VolumeInfo
- (*VolumeTierMoveDatToRemoteRequest)(nil), // 66: volume_server_pb.VolumeTierMoveDatToRemoteRequest
- (*VolumeTierMoveDatToRemoteResponse)(nil), // 67: volume_server_pb.VolumeTierMoveDatToRemoteResponse
- (*VolumeTierMoveDatFromRemoteRequest)(nil), // 68: volume_server_pb.VolumeTierMoveDatFromRemoteRequest
- (*VolumeTierMoveDatFromRemoteResponse)(nil), // 69: volume_server_pb.VolumeTierMoveDatFromRemoteResponse
- (*VolumeServerStatusRequest)(nil), // 70: volume_server_pb.VolumeServerStatusRequest
- (*VolumeServerStatusResponse)(nil), // 71: volume_server_pb.VolumeServerStatusResponse
- (*VolumeServerLeaveRequest)(nil), // 72: volume_server_pb.VolumeServerLeaveRequest
- (*VolumeServerLeaveResponse)(nil), // 73: volume_server_pb.VolumeServerLeaveResponse
- (*QueryRequest)(nil), // 74: volume_server_pb.QueryRequest
- (*QueriedStripe)(nil), // 75: volume_server_pb.QueriedStripe
- (*VolumeNeedleStatusRequest)(nil), // 76: volume_server_pb.VolumeNeedleStatusRequest
- (*VolumeNeedleStatusResponse)(nil), // 77: volume_server_pb.VolumeNeedleStatusResponse
- (*QueryRequest_Filter)(nil), // 78: volume_server_pb.QueryRequest.Filter
- (*QueryRequest_InputSerialization)(nil), // 79: volume_server_pb.QueryRequest.InputSerialization
- (*QueryRequest_OutputSerialization)(nil), // 80: volume_server_pb.QueryRequest.OutputSerialization
- (*QueryRequest_InputSerialization_CSVInput)(nil), // 81: volume_server_pb.QueryRequest.InputSerialization.CSVInput
- (*QueryRequest_InputSerialization_JSONInput)(nil), // 82: volume_server_pb.QueryRequest.InputSerialization.JSONInput
- (*QueryRequest_InputSerialization_ParquetInput)(nil), // 83: volume_server_pb.QueryRequest.InputSerialization.ParquetInput
- (*QueryRequest_OutputSerialization_CSVOutput)(nil), // 84: volume_server_pb.QueryRequest.OutputSerialization.CSVOutput
- (*QueryRequest_OutputSerialization_JSONOutput)(nil), // 85: volume_server_pb.QueryRequest.OutputSerialization.JSONOutput
+ (*ReadNeedleBlobRequest)(nil), // 38: volume_server_pb.ReadNeedleBlobRequest
+ (*ReadNeedleBlobResponse)(nil), // 39: volume_server_pb.ReadNeedleBlobResponse
+ (*WriteNeedleBlobRequest)(nil), // 40: volume_server_pb.WriteNeedleBlobRequest
+ (*WriteNeedleBlobResponse)(nil), // 41: volume_server_pb.WriteNeedleBlobResponse
+ (*VolumeTailSenderRequest)(nil), // 42: volume_server_pb.VolumeTailSenderRequest
+ (*VolumeTailSenderResponse)(nil), // 43: volume_server_pb.VolumeTailSenderResponse
+ (*VolumeTailReceiverRequest)(nil), // 44: volume_server_pb.VolumeTailReceiverRequest
+ (*VolumeTailReceiverResponse)(nil), // 45: volume_server_pb.VolumeTailReceiverResponse
+ (*VolumeEcShardsGenerateRequest)(nil), // 46: volume_server_pb.VolumeEcShardsGenerateRequest
+ (*VolumeEcShardsGenerateResponse)(nil), // 47: volume_server_pb.VolumeEcShardsGenerateResponse
+ (*VolumeEcShardsRebuildRequest)(nil), // 48: volume_server_pb.VolumeEcShardsRebuildRequest
+ (*VolumeEcShardsRebuildResponse)(nil), // 49: volume_server_pb.VolumeEcShardsRebuildResponse
+ (*VolumeEcShardsCopyRequest)(nil), // 50: volume_server_pb.VolumeEcShardsCopyRequest
+ (*VolumeEcShardsCopyResponse)(nil), // 51: volume_server_pb.VolumeEcShardsCopyResponse
+ (*VolumeEcShardsDeleteRequest)(nil), // 52: volume_server_pb.VolumeEcShardsDeleteRequest
+ (*VolumeEcShardsDeleteResponse)(nil), // 53: volume_server_pb.VolumeEcShardsDeleteResponse
+ (*VolumeEcShardsMountRequest)(nil), // 54: volume_server_pb.VolumeEcShardsMountRequest
+ (*VolumeEcShardsMountResponse)(nil), // 55: volume_server_pb.VolumeEcShardsMountResponse
+ (*VolumeEcShardsUnmountRequest)(nil), // 56: volume_server_pb.VolumeEcShardsUnmountRequest
+ (*VolumeEcShardsUnmountResponse)(nil), // 57: volume_server_pb.VolumeEcShardsUnmountResponse
+ (*VolumeEcShardReadRequest)(nil), // 58: volume_server_pb.VolumeEcShardReadRequest
+ (*VolumeEcShardReadResponse)(nil), // 59: volume_server_pb.VolumeEcShardReadResponse
+ (*VolumeEcBlobDeleteRequest)(nil), // 60: volume_server_pb.VolumeEcBlobDeleteRequest
+ (*VolumeEcBlobDeleteResponse)(nil), // 61: volume_server_pb.VolumeEcBlobDeleteResponse
+ (*VolumeEcShardsToVolumeRequest)(nil), // 62: volume_server_pb.VolumeEcShardsToVolumeRequest
+ (*VolumeEcShardsToVolumeResponse)(nil), // 63: volume_server_pb.VolumeEcShardsToVolumeResponse
+ (*ReadVolumeFileStatusRequest)(nil), // 64: volume_server_pb.ReadVolumeFileStatusRequest
+ (*ReadVolumeFileStatusResponse)(nil), // 65: volume_server_pb.ReadVolumeFileStatusResponse
+ (*DiskStatus)(nil), // 66: volume_server_pb.DiskStatus
+ (*MemStatus)(nil), // 67: volume_server_pb.MemStatus
+ (*RemoteFile)(nil), // 68: volume_server_pb.RemoteFile
+ (*VolumeInfo)(nil), // 69: volume_server_pb.VolumeInfo
+ (*VolumeTierMoveDatToRemoteRequest)(nil), // 70: volume_server_pb.VolumeTierMoveDatToRemoteRequest
+ (*VolumeTierMoveDatToRemoteResponse)(nil), // 71: volume_server_pb.VolumeTierMoveDatToRemoteResponse
+ (*VolumeTierMoveDatFromRemoteRequest)(nil), // 72: volume_server_pb.VolumeTierMoveDatFromRemoteRequest
+ (*VolumeTierMoveDatFromRemoteResponse)(nil), // 73: volume_server_pb.VolumeTierMoveDatFromRemoteResponse
+ (*VolumeServerStatusRequest)(nil), // 74: volume_server_pb.VolumeServerStatusRequest
+ (*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
}
var file_volume_server_proto_depIdxs = []int32{
2, // 0: volume_server_pb.BatchDeleteResponse.results:type_name -> volume_server_pb.DeleteResult
- 64, // 1: volume_server_pb.VolumeInfo.files:type_name -> volume_server_pb.RemoteFile
- 62, // 2: volume_server_pb.VolumeServerStatusResponse.disk_statuses:type_name -> volume_server_pb.DiskStatus
- 63, // 3: volume_server_pb.VolumeServerStatusResponse.memory_status:type_name -> volume_server_pb.MemStatus
- 78, // 4: volume_server_pb.QueryRequest.filter:type_name -> volume_server_pb.QueryRequest.Filter
- 79, // 5: volume_server_pb.QueryRequest.input_serialization:type_name -> volume_server_pb.QueryRequest.InputSerialization
- 80, // 6: volume_server_pb.QueryRequest.output_serialization:type_name -> volume_server_pb.QueryRequest.OutputSerialization
- 81, // 7: volume_server_pb.QueryRequest.InputSerialization.csv_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.CSVInput
- 82, // 8: volume_server_pb.QueryRequest.InputSerialization.json_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.JSONInput
- 83, // 9: volume_server_pb.QueryRequest.InputSerialization.parquet_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.ParquetInput
- 84, // 10: volume_server_pb.QueryRequest.OutputSerialization.csv_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.CSVOutput
- 85, // 11: volume_server_pb.QueryRequest.OutputSerialization.json_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.JSONOutput
+ 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
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
@@ -5805,63 +6071,67 @@ var file_volume_server_proto_depIdxs = []int32{
30, // 26: volume_server_pb.VolumeServer.VolumeConfigure:input_type -> volume_server_pb.VolumeConfigureRequest
32, // 27: volume_server_pb.VolumeServer.VolumeStatus:input_type -> volume_server_pb.VolumeStatusRequest
34, // 28: volume_server_pb.VolumeServer.VolumeCopy:input_type -> volume_server_pb.VolumeCopyRequest
- 60, // 29: volume_server_pb.VolumeServer.ReadVolumeFileStatus:input_type -> volume_server_pb.ReadVolumeFileStatusRequest
+ 64, // 29: volume_server_pb.VolumeServer.ReadVolumeFileStatus:input_type -> volume_server_pb.ReadVolumeFileStatusRequest
36, // 30: volume_server_pb.VolumeServer.CopyFile:input_type -> volume_server_pb.CopyFileRequest
- 38, // 31: volume_server_pb.VolumeServer.VolumeTailSender:input_type -> volume_server_pb.VolumeTailSenderRequest
- 40, // 32: volume_server_pb.VolumeServer.VolumeTailReceiver:input_type -> volume_server_pb.VolumeTailReceiverRequest
- 42, // 33: volume_server_pb.VolumeServer.VolumeEcShardsGenerate:input_type -> volume_server_pb.VolumeEcShardsGenerateRequest
- 44, // 34: volume_server_pb.VolumeServer.VolumeEcShardsRebuild:input_type -> volume_server_pb.VolumeEcShardsRebuildRequest
- 46, // 35: volume_server_pb.VolumeServer.VolumeEcShardsCopy:input_type -> volume_server_pb.VolumeEcShardsCopyRequest
- 48, // 36: volume_server_pb.VolumeServer.VolumeEcShardsDelete:input_type -> volume_server_pb.VolumeEcShardsDeleteRequest
- 50, // 37: volume_server_pb.VolumeServer.VolumeEcShardsMount:input_type -> volume_server_pb.VolumeEcShardsMountRequest
- 52, // 38: volume_server_pb.VolumeServer.VolumeEcShardsUnmount:input_type -> volume_server_pb.VolumeEcShardsUnmountRequest
- 54, // 39: volume_server_pb.VolumeServer.VolumeEcShardRead:input_type -> volume_server_pb.VolumeEcShardReadRequest
- 56, // 40: volume_server_pb.VolumeServer.VolumeEcBlobDelete:input_type -> volume_server_pb.VolumeEcBlobDeleteRequest
- 58, // 41: volume_server_pb.VolumeServer.VolumeEcShardsToVolume:input_type -> volume_server_pb.VolumeEcShardsToVolumeRequest
- 66, // 42: volume_server_pb.VolumeServer.VolumeTierMoveDatToRemote:input_type -> volume_server_pb.VolumeTierMoveDatToRemoteRequest
- 68, // 43: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:input_type -> volume_server_pb.VolumeTierMoveDatFromRemoteRequest
- 70, // 44: volume_server_pb.VolumeServer.VolumeServerStatus:input_type -> volume_server_pb.VolumeServerStatusRequest
- 72, // 45: volume_server_pb.VolumeServer.VolumeServerLeave:input_type -> volume_server_pb.VolumeServerLeaveRequest
- 74, // 46: volume_server_pb.VolumeServer.Query:input_type -> volume_server_pb.QueryRequest
- 76, // 47: volume_server_pb.VolumeServer.VolumeNeedleStatus:input_type -> volume_server_pb.VolumeNeedleStatusRequest
- 1, // 48: volume_server_pb.VolumeServer.BatchDelete:output_type -> volume_server_pb.BatchDeleteResponse
- 5, // 49: volume_server_pb.VolumeServer.VacuumVolumeCheck:output_type -> volume_server_pb.VacuumVolumeCheckResponse
- 7, // 50: volume_server_pb.VolumeServer.VacuumVolumeCompact:output_type -> volume_server_pb.VacuumVolumeCompactResponse
- 9, // 51: volume_server_pb.VolumeServer.VacuumVolumeCommit:output_type -> volume_server_pb.VacuumVolumeCommitResponse
- 11, // 52: volume_server_pb.VolumeServer.VacuumVolumeCleanup:output_type -> volume_server_pb.VacuumVolumeCleanupResponse
- 13, // 53: volume_server_pb.VolumeServer.DeleteCollection:output_type -> volume_server_pb.DeleteCollectionResponse
- 15, // 54: volume_server_pb.VolumeServer.AllocateVolume:output_type -> volume_server_pb.AllocateVolumeResponse
- 17, // 55: volume_server_pb.VolumeServer.VolumeSyncStatus:output_type -> volume_server_pb.VolumeSyncStatusResponse
- 19, // 56: volume_server_pb.VolumeServer.VolumeIncrementalCopy:output_type -> volume_server_pb.VolumeIncrementalCopyResponse
- 21, // 57: volume_server_pb.VolumeServer.VolumeMount:output_type -> volume_server_pb.VolumeMountResponse
- 23, // 58: volume_server_pb.VolumeServer.VolumeUnmount:output_type -> volume_server_pb.VolumeUnmountResponse
- 25, // 59: volume_server_pb.VolumeServer.VolumeDelete:output_type -> volume_server_pb.VolumeDeleteResponse
- 27, // 60: volume_server_pb.VolumeServer.VolumeMarkReadonly:output_type -> volume_server_pb.VolumeMarkReadonlyResponse
- 29, // 61: volume_server_pb.VolumeServer.VolumeMarkWritable:output_type -> volume_server_pb.VolumeMarkWritableResponse
- 31, // 62: volume_server_pb.VolumeServer.VolumeConfigure:output_type -> volume_server_pb.VolumeConfigureResponse
- 33, // 63: volume_server_pb.VolumeServer.VolumeStatus:output_type -> volume_server_pb.VolumeStatusResponse
- 35, // 64: volume_server_pb.VolumeServer.VolumeCopy:output_type -> volume_server_pb.VolumeCopyResponse
- 61, // 65: volume_server_pb.VolumeServer.ReadVolumeFileStatus:output_type -> volume_server_pb.ReadVolumeFileStatusResponse
- 37, // 66: volume_server_pb.VolumeServer.CopyFile:output_type -> volume_server_pb.CopyFileResponse
- 39, // 67: volume_server_pb.VolumeServer.VolumeTailSender:output_type -> volume_server_pb.VolumeTailSenderResponse
- 41, // 68: volume_server_pb.VolumeServer.VolumeTailReceiver:output_type -> volume_server_pb.VolumeTailReceiverResponse
- 43, // 69: volume_server_pb.VolumeServer.VolumeEcShardsGenerate:output_type -> volume_server_pb.VolumeEcShardsGenerateResponse
- 45, // 70: volume_server_pb.VolumeServer.VolumeEcShardsRebuild:output_type -> volume_server_pb.VolumeEcShardsRebuildResponse
- 47, // 71: volume_server_pb.VolumeServer.VolumeEcShardsCopy:output_type -> volume_server_pb.VolumeEcShardsCopyResponse
- 49, // 72: volume_server_pb.VolumeServer.VolumeEcShardsDelete:output_type -> volume_server_pb.VolumeEcShardsDeleteResponse
- 51, // 73: volume_server_pb.VolumeServer.VolumeEcShardsMount:output_type -> volume_server_pb.VolumeEcShardsMountResponse
- 53, // 74: volume_server_pb.VolumeServer.VolumeEcShardsUnmount:output_type -> volume_server_pb.VolumeEcShardsUnmountResponse
- 55, // 75: volume_server_pb.VolumeServer.VolumeEcShardRead:output_type -> volume_server_pb.VolumeEcShardReadResponse
- 57, // 76: volume_server_pb.VolumeServer.VolumeEcBlobDelete:output_type -> volume_server_pb.VolumeEcBlobDeleteResponse
- 59, // 77: volume_server_pb.VolumeServer.VolumeEcShardsToVolume:output_type -> volume_server_pb.VolumeEcShardsToVolumeResponse
- 67, // 78: volume_server_pb.VolumeServer.VolumeTierMoveDatToRemote:output_type -> volume_server_pb.VolumeTierMoveDatToRemoteResponse
- 69, // 79: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:output_type -> volume_server_pb.VolumeTierMoveDatFromRemoteResponse
- 71, // 80: volume_server_pb.VolumeServer.VolumeServerStatus:output_type -> volume_server_pb.VolumeServerStatusResponse
- 73, // 81: volume_server_pb.VolumeServer.VolumeServerLeave:output_type -> volume_server_pb.VolumeServerLeaveResponse
- 75, // 82: volume_server_pb.VolumeServer.Query:output_type -> volume_server_pb.QueriedStripe
- 77, // 83: volume_server_pb.VolumeServer.VolumeNeedleStatus:output_type -> volume_server_pb.VolumeNeedleStatusResponse
- 48, // [48:84] is the sub-list for method output_type
- 12, // [12:48] is the sub-list for method input_type
+ 38, // 31: volume_server_pb.VolumeServer.ReadNeedleBlob:input_type -> volume_server_pb.ReadNeedleBlobRequest
+ 40, // 32: volume_server_pb.VolumeServer.WriteNeedleBlob:input_type -> volume_server_pb.WriteNeedleBlobRequest
+ 42, // 33: volume_server_pb.VolumeServer.VolumeTailSender:input_type -> volume_server_pb.VolumeTailSenderRequest
+ 44, // 34: volume_server_pb.VolumeServer.VolumeTailReceiver:input_type -> volume_server_pb.VolumeTailReceiverRequest
+ 46, // 35: volume_server_pb.VolumeServer.VolumeEcShardsGenerate:input_type -> volume_server_pb.VolumeEcShardsGenerateRequest
+ 48, // 36: volume_server_pb.VolumeServer.VolumeEcShardsRebuild:input_type -> volume_server_pb.VolumeEcShardsRebuildRequest
+ 50, // 37: volume_server_pb.VolumeServer.VolumeEcShardsCopy:input_type -> volume_server_pb.VolumeEcShardsCopyRequest
+ 52, // 38: volume_server_pb.VolumeServer.VolumeEcShardsDelete:input_type -> volume_server_pb.VolumeEcShardsDeleteRequest
+ 54, // 39: volume_server_pb.VolumeServer.VolumeEcShardsMount:input_type -> volume_server_pb.VolumeEcShardsMountRequest
+ 56, // 40: volume_server_pb.VolumeServer.VolumeEcShardsUnmount:input_type -> volume_server_pb.VolumeEcShardsUnmountRequest
+ 58, // 41: volume_server_pb.VolumeServer.VolumeEcShardRead:input_type -> volume_server_pb.VolumeEcShardReadRequest
+ 60, // 42: volume_server_pb.VolumeServer.VolumeEcBlobDelete:input_type -> volume_server_pb.VolumeEcBlobDeleteRequest
+ 62, // 43: volume_server_pb.VolumeServer.VolumeEcShardsToVolume:input_type -> volume_server_pb.VolumeEcShardsToVolumeRequest
+ 70, // 44: volume_server_pb.VolumeServer.VolumeTierMoveDatToRemote:input_type -> volume_server_pb.VolumeTierMoveDatToRemoteRequest
+ 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
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
@@ -6330,7 +6600,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTailSenderRequest); i {
+ switch v := v.(*ReadNeedleBlobRequest); i {
case 0:
return &v.state
case 1:
@@ -6342,7 +6612,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTailSenderResponse); i {
+ switch v := v.(*ReadNeedleBlobResponse); i {
case 0:
return &v.state
case 1:
@@ -6354,7 +6624,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTailReceiverRequest); i {
+ switch v := v.(*WriteNeedleBlobRequest); i {
case 0:
return &v.state
case 1:
@@ -6366,7 +6636,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTailReceiverResponse); i {
+ switch v := v.(*WriteNeedleBlobResponse); i {
case 0:
return &v.state
case 1:
@@ -6378,7 +6648,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsGenerateRequest); i {
+ switch v := v.(*VolumeTailSenderRequest); i {
case 0:
return &v.state
case 1:
@@ -6390,7 +6660,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsGenerateResponse); i {
+ switch v := v.(*VolumeTailSenderResponse); i {
case 0:
return &v.state
case 1:
@@ -6402,7 +6672,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsRebuildRequest); i {
+ switch v := v.(*VolumeTailReceiverRequest); i {
case 0:
return &v.state
case 1:
@@ -6414,7 +6684,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsRebuildResponse); i {
+ switch v := v.(*VolumeTailReceiverResponse); i {
case 0:
return &v.state
case 1:
@@ -6426,7 +6696,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsCopyRequest); i {
+ switch v := v.(*VolumeEcShardsGenerateRequest); i {
case 0:
return &v.state
case 1:
@@ -6438,7 +6708,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsCopyResponse); i {
+ switch v := v.(*VolumeEcShardsGenerateResponse); i {
case 0:
return &v.state
case 1:
@@ -6450,7 +6720,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsDeleteRequest); i {
+ switch v := v.(*VolumeEcShardsRebuildRequest); i {
case 0:
return &v.state
case 1:
@@ -6462,7 +6732,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsDeleteResponse); i {
+ switch v := v.(*VolumeEcShardsRebuildResponse); i {
case 0:
return &v.state
case 1:
@@ -6474,7 +6744,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsMountRequest); i {
+ switch v := v.(*VolumeEcShardsCopyRequest); i {
case 0:
return &v.state
case 1:
@@ -6486,7 +6756,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsMountResponse); i {
+ switch v := v.(*VolumeEcShardsCopyResponse); i {
case 0:
return &v.state
case 1:
@@ -6498,7 +6768,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsUnmountRequest); i {
+ switch v := v.(*VolumeEcShardsDeleteRequest); i {
case 0:
return &v.state
case 1:
@@ -6510,7 +6780,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsUnmountResponse); i {
+ switch v := v.(*VolumeEcShardsDeleteResponse); i {
case 0:
return &v.state
case 1:
@@ -6522,7 +6792,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardReadRequest); i {
+ switch v := v.(*VolumeEcShardsMountRequest); i {
case 0:
return &v.state
case 1:
@@ -6534,7 +6804,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardReadResponse); i {
+ switch v := v.(*VolumeEcShardsMountResponse); i {
case 0:
return &v.state
case 1:
@@ -6546,7 +6816,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcBlobDeleteRequest); i {
+ switch v := v.(*VolumeEcShardsUnmountRequest); i {
case 0:
return &v.state
case 1:
@@ -6558,7 +6828,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcBlobDeleteResponse); i {
+ switch v := v.(*VolumeEcShardsUnmountResponse); i {
case 0:
return &v.state
case 1:
@@ -6570,7 +6840,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsToVolumeRequest); i {
+ switch v := v.(*VolumeEcShardReadRequest); i {
case 0:
return &v.state
case 1:
@@ -6582,7 +6852,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeEcShardsToVolumeResponse); i {
+ switch v := v.(*VolumeEcShardReadResponse); i {
case 0:
return &v.state
case 1:
@@ -6594,7 +6864,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ReadVolumeFileStatusRequest); i {
+ switch v := v.(*VolumeEcBlobDeleteRequest); i {
case 0:
return &v.state
case 1:
@@ -6606,7 +6876,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ReadVolumeFileStatusResponse); i {
+ switch v := v.(*VolumeEcBlobDeleteResponse); i {
case 0:
return &v.state
case 1:
@@ -6618,7 +6888,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*DiskStatus); i {
+ switch v := v.(*VolumeEcShardsToVolumeRequest); i {
case 0:
return &v.state
case 1:
@@ -6630,7 +6900,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*MemStatus); i {
+ switch v := v.(*VolumeEcShardsToVolumeResponse); i {
case 0:
return &v.state
case 1:
@@ -6642,7 +6912,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*RemoteFile); i {
+ switch v := v.(*ReadVolumeFileStatusRequest); i {
case 0:
return &v.state
case 1:
@@ -6654,7 +6924,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeInfo); i {
+ switch v := v.(*ReadVolumeFileStatusResponse); i {
case 0:
return &v.state
case 1:
@@ -6666,7 +6936,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTierMoveDatToRemoteRequest); i {
+ switch v := v.(*DiskStatus); i {
case 0:
return &v.state
case 1:
@@ -6678,7 +6948,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTierMoveDatToRemoteResponse); i {
+ switch v := v.(*MemStatus); i {
case 0:
return &v.state
case 1:
@@ -6690,7 +6960,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTierMoveDatFromRemoteRequest); i {
+ switch v := v.(*RemoteFile); i {
case 0:
return &v.state
case 1:
@@ -6702,7 +6972,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeTierMoveDatFromRemoteResponse); i {
+ switch v := v.(*VolumeInfo); i {
case 0:
return &v.state
case 1:
@@ -6714,7 +6984,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeServerStatusRequest); i {
+ switch v := v.(*VolumeTierMoveDatToRemoteRequest); i {
case 0:
return &v.state
case 1:
@@ -6726,7 +6996,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeServerStatusResponse); i {
+ switch v := v.(*VolumeTierMoveDatToRemoteResponse); i {
case 0:
return &v.state
case 1:
@@ -6738,7 +7008,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeServerLeaveRequest); i {
+ switch v := v.(*VolumeTierMoveDatFromRemoteRequest); i {
case 0:
return &v.state
case 1:
@@ -6750,7 +7020,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeServerLeaveResponse); i {
+ switch v := v.(*VolumeTierMoveDatFromRemoteResponse); i {
case 0:
return &v.state
case 1:
@@ -6762,7 +7032,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*QueryRequest); i {
+ switch v := v.(*VolumeServerStatusRequest); i {
case 0:
return &v.state
case 1:
@@ -6774,7 +7044,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*QueriedStripe); i {
+ switch v := v.(*VolumeServerStatusResponse); i {
case 0:
return &v.state
case 1:
@@ -6786,7 +7056,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeNeedleStatusRequest); i {
+ switch v := v.(*VolumeServerLeaveRequest); i {
case 0:
return &v.state
case 1:
@@ -6798,7 +7068,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VolumeNeedleStatusResponse); i {
+ switch v := v.(*VolumeServerLeaveResponse); i {
case 0:
return &v.state
case 1:
@@ -6810,7 +7080,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*QueryRequest_Filter); i {
+ switch v := v.(*QueryRequest); i {
case 0:
return &v.state
case 1:
@@ -6822,7 +7092,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*QueryRequest_InputSerialization); i {
+ switch v := v.(*QueriedStripe); i {
case 0:
return &v.state
case 1:
@@ -6834,7 +7104,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*QueryRequest_OutputSerialization); i {
+ switch v := v.(*VolumeNeedleStatusRequest); i {
case 0:
return &v.state
case 1:
@@ -6846,7 +7116,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*QueryRequest_InputSerialization_CSVInput); i {
+ switch v := v.(*VolumeNeedleStatusResponse); i {
case 0:
return &v.state
case 1:
@@ -6858,7 +7128,7 @@ func file_volume_server_proto_init() {
}
}
file_volume_server_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*QueryRequest_InputSerialization_JSONInput); i {
+ switch v := v.(*QueryRequest_Filter); i {
case 0:
return &v.state
case 1:
@@ -6870,7 +7140,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_ParquetInput); i {
+ switch v := v.(*QueryRequest_InputSerialization); i {
case 0:
return &v.state
case 1:
@@ -6882,7 +7152,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_CSVOutput); i {
+ switch v := v.(*QueryRequest_OutputSerialization); i {
case 0:
return &v.state
case 1:
@@ -6894,6 +7164,54 @@ 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 {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_volume_server_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*QueryRequest_InputSerialization_JSONInput); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_volume_server_proto_msgTypes[87].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[88].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[89].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*QueryRequest_OutputSerialization_JSONOutput); i {
case 0:
return &v.state
@@ -6912,7 +7230,7 @@ func file_volume_server_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_volume_server_proto_rawDesc,
NumEnums: 0,
- NumMessages: 86,
+ NumMessages: 90,
NumExtensions: 0,
NumServices: 1,
},
@@ -6959,6 +7277,8 @@ type VolumeServerClient interface {
VolumeCopy(ctx context.Context, in *VolumeCopyRequest, opts ...grpc.CallOption) (*VolumeCopyResponse, error)
ReadVolumeFileStatus(ctx context.Context, in *ReadVolumeFileStatusRequest, opts ...grpc.CallOption) (*ReadVolumeFileStatusResponse, error)
CopyFile(ctx context.Context, in *CopyFileRequest, opts ...grpc.CallOption) (VolumeServer_CopyFileClient, error)
+ ReadNeedleBlob(ctx context.Context, in *ReadNeedleBlobRequest, opts ...grpc.CallOption) (*ReadNeedleBlobResponse, error)
+ WriteNeedleBlob(ctx context.Context, in *WriteNeedleBlobRequest, opts ...grpc.CallOption) (*WriteNeedleBlobResponse, error)
VolumeTailSender(ctx context.Context, in *VolumeTailSenderRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTailSenderClient, error)
VolumeTailReceiver(ctx context.Context, in *VolumeTailReceiverRequest, opts ...grpc.CallOption) (*VolumeTailReceiverResponse, error)
// erasure coding
@@ -7206,6 +7526,24 @@ func (x *volumeServerCopyFileClient) Recv() (*CopyFileResponse, error) {
return m, nil
}
+func (c *volumeServerClient) ReadNeedleBlob(ctx context.Context, in *ReadNeedleBlobRequest, opts ...grpc.CallOption) (*ReadNeedleBlobResponse, error) {
+ out := new(ReadNeedleBlobResponse)
+ err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/ReadNeedleBlob", in, out, opts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
+func (c *volumeServerClient) WriteNeedleBlob(ctx context.Context, in *WriteNeedleBlobRequest, opts ...grpc.CallOption) (*WriteNeedleBlobResponse, error) {
+ out := new(WriteNeedleBlobResponse)
+ err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/WriteNeedleBlob", in, out, opts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
func (c *volumeServerClient) VolumeTailSender(ctx context.Context, in *VolumeTailSenderRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTailSenderClient, error) {
stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[2], "/volume_server_pb.VolumeServer/VolumeTailSender", opts...)
if err != nil {
@@ -7497,6 +7835,8 @@ type VolumeServerServer interface {
VolumeCopy(context.Context, *VolumeCopyRequest) (*VolumeCopyResponse, error)
ReadVolumeFileStatus(context.Context, *ReadVolumeFileStatusRequest) (*ReadVolumeFileStatusResponse, error)
CopyFile(*CopyFileRequest, VolumeServer_CopyFileServer) error
+ ReadNeedleBlob(context.Context, *ReadNeedleBlobRequest) (*ReadNeedleBlobResponse, error)
+ WriteNeedleBlob(context.Context, *WriteNeedleBlobRequest) (*WriteNeedleBlobResponse, error)
VolumeTailSender(*VolumeTailSenderRequest, VolumeServer_VolumeTailSenderServer) error
VolumeTailReceiver(context.Context, *VolumeTailReceiverRequest) (*VolumeTailReceiverResponse, error)
// erasure coding
@@ -7580,6 +7920,12 @@ func (*UnimplementedVolumeServerServer) ReadVolumeFileStatus(context.Context, *R
func (*UnimplementedVolumeServerServer) CopyFile(*CopyFileRequest, VolumeServer_CopyFileServer) error {
return status.Errorf(codes.Unimplemented, "method CopyFile not implemented")
}
+func (*UnimplementedVolumeServerServer) ReadNeedleBlob(context.Context, *ReadNeedleBlobRequest) (*ReadNeedleBlobResponse, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method ReadNeedleBlob not implemented")
+}
+func (*UnimplementedVolumeServerServer) WriteNeedleBlob(context.Context, *WriteNeedleBlobRequest) (*WriteNeedleBlobResponse, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method WriteNeedleBlob not implemented")
+}
func (*UnimplementedVolumeServerServer) VolumeTailSender(*VolumeTailSenderRequest, VolumeServer_VolumeTailSenderServer) error {
return status.Errorf(codes.Unimplemented, "method VolumeTailSender not implemented")
}
@@ -7984,6 +8330,42 @@ func (x *volumeServerCopyFileServer) Send(m *CopyFileResponse) error {
return x.ServerStream.SendMsg(m)
}
+func _VolumeServer_ReadNeedleBlob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(ReadNeedleBlobRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(VolumeServerServer).ReadNeedleBlob(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: "/volume_server_pb.VolumeServer/ReadNeedleBlob",
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(VolumeServerServer).ReadNeedleBlob(ctx, req.(*ReadNeedleBlobRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
+func _VolumeServer_WriteNeedleBlob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(WriteNeedleBlobRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(VolumeServerServer).WriteNeedleBlob(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: "/volume_server_pb.VolumeServer/WriteNeedleBlob",
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(VolumeServerServer).WriteNeedleBlob(ctx, req.(*WriteNeedleBlobRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
func _VolumeServer_VolumeTailSender_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(VolumeTailSenderRequest)
if err := stream.RecvMsg(m); err != nil {
@@ -8378,6 +8760,14 @@ var _VolumeServer_serviceDesc = grpc.ServiceDesc{
Handler: _VolumeServer_ReadVolumeFileStatus_Handler,
},
{
+ MethodName: "ReadNeedleBlob",
+ Handler: _VolumeServer_ReadNeedleBlob_Handler,
+ },
+ {
+ MethodName: "WriteNeedleBlob",
+ Handler: _VolumeServer_WriteNeedleBlob_Handler,
+ },
+ {
MethodName: "VolumeTailReceiver",
Handler: _VolumeServer_VolumeTailReceiver_Handler,
},
diff --git a/weed/replication/repl_util/replication_utli.go b/weed/replication/repl_util/replication_util.go
index 3514c6977..519a9a201 100644
--- a/weed/replication/repl_util/replication_utli.go
+++ b/weed/replication/repl_util/replication_util.go
@@ -20,7 +20,7 @@ func CopyFromChunkViews(chunkViews []*filer.ChunkView, filerSource *source.Filer
var shouldRetry bool
for _, fileUrl := range fileUrls {
- shouldRetry, err = util.FastReadUrlAsStream(fileUrl+"?readDeleted=true", nil, false, chunk.IsFullChunk(), chunk.Offset, int(chunk.Size), func(data []byte) {
+ shouldRetry, err = util.ReadUrlAsStream(fileUrl, nil, false, chunk.IsFullChunk(), chunk.Offset, int(chunk.Size), func(data []byte) {
writeErr = writeFunc(data)
})
if err != nil {
diff --git a/weed/replication/replicator.go b/weed/replication/replicator.go
index 7688029e6..d7e609c68 100644
--- a/weed/replication/replicator.go
+++ b/weed/replication/replicator.go
@@ -42,7 +42,7 @@ func (r *Replicator) Replicate(ctx context.Context, key string, message *filer_p
return nil
}
var dateKey string
- if r.sink.GetName() == "local_incremental" {
+ if r.sink.IsIncremental() {
var mTime int64
if message.NewEntry != nil {
mTime = message.NewEntry.Attributes.Mtime
diff --git a/weed/replication/sink/azuresink/azure_sink.go b/weed/replication/sink/azuresink/azure_sink.go
index df70be64b..d13a1049b 100644
--- a/weed/replication/sink/azuresink/azure_sink.go
+++ b/weed/replication/sink/azuresink/azure_sink.go
@@ -18,10 +18,11 @@ import (
)
type AzureSink struct {
- containerURL azblob.ContainerURL
- container string
- dir string
- filerSource *source.FilerSource
+ containerURL azblob.ContainerURL
+ container string
+ dir string
+ filerSource *source.FilerSource
+ isIncremental bool
}
func init() {
@@ -36,7 +37,12 @@ func (g *AzureSink) GetSinkToDirectory() string {
return g.dir
}
+func (g *AzureSink) IsIncremental() bool {
+ return g.isIncremental
+}
+
func (g *AzureSink) Initialize(configuration util.Configuration, prefix string) error {
+ g.isIncremental = configuration.GetBool(prefix + "is_incremental")
return g.initialize(
configuration.GetString(prefix+"account_name"),
configuration.GetString(prefix+"account_key"),
diff --git a/weed/replication/sink/b2sink/b2_sink.go b/weed/replication/sink/b2sink/b2_sink.go
index 24f0ecbbc..90a0bb2e8 100644
--- a/weed/replication/sink/b2sink/b2_sink.go
+++ b/weed/replication/sink/b2sink/b2_sink.go
@@ -14,10 +14,11 @@ import (
)
type B2Sink struct {
- client *b2.Client
- bucket string
- dir string
- filerSource *source.FilerSource
+ client *b2.Client
+ bucket string
+ dir string
+ filerSource *source.FilerSource
+ isIncremental bool
}
func init() {
@@ -32,7 +33,12 @@ func (g *B2Sink) GetSinkToDirectory() string {
return g.dir
}
+func (g *B2Sink) IsIncremental() bool {
+ return g.isIncremental
+}
+
func (g *B2Sink) Initialize(configuration util.Configuration, prefix string) error {
+ g.isIncremental = configuration.GetBool(prefix + "is_incremental")
return g.initialize(
configuration.GetString(prefix+"b2_account_id"),
configuration.GetString(prefix+"b2_master_application_key"),
diff --git a/weed/replication/sink/filersink/filer_sink.go b/weed/replication/sink/filersink/filer_sink.go
index 509f75116..d7c5fccc3 100644
--- a/weed/replication/sink/filersink/filer_sink.go
+++ b/weed/replication/sink/filersink/filer_sink.go
@@ -30,6 +30,7 @@ type FilerSink struct {
grpcDialOption grpc.DialOption
address string
writeChunkByFiler bool
+ isIncremental bool
}
func init() {
@@ -44,7 +45,12 @@ func (fs *FilerSink) GetSinkToDirectory() string {
return fs.dir
}
+func (fs *FilerSink) IsIncremental() bool {
+ return fs.isIncremental
+}
+
func (fs *FilerSink) Initialize(configuration util.Configuration, prefix string) error {
+ fs.isIncremental = configuration.GetBool(prefix + "is_incremental")
return fs.DoInitialize(
"",
configuration.GetString(prefix+"grpcAddress"),
diff --git a/weed/replication/sink/gcssink/gcs_sink.go b/weed/replication/sink/gcssink/gcs_sink.go
index badabc32c..5cf5b7317 100644
--- a/weed/replication/sink/gcssink/gcs_sink.go
+++ b/weed/replication/sink/gcssink/gcs_sink.go
@@ -18,10 +18,11 @@ import (
)
type GcsSink struct {
- client *storage.Client
- bucket string
- dir string
- filerSource *source.FilerSource
+ client *storage.Client
+ bucket string
+ dir string
+ filerSource *source.FilerSource
+ isIncremental bool
}
func init() {
@@ -36,7 +37,12 @@ func (g *GcsSink) GetSinkToDirectory() string {
return g.dir
}
+func (g *GcsSink) IsIncremental() bool {
+ return g.isIncremental
+}
+
func (g *GcsSink) Initialize(configuration util.Configuration, prefix string) error {
+ g.isIncremental = configuration.GetBool(prefix + "is_incremental")
return g.initialize(
configuration.GetString(prefix+"google_application_credentials"),
configuration.GetString(prefix+"bucket"),
diff --git a/weed/replication/sink/localsink/local_sink.go b/weed/replication/sink/localsink/local_sink.go
index 21c625c3f..2b9b3e69a 100644
--- a/weed/replication/sink/localsink/local_sink.go
+++ b/weed/replication/sink/localsink/local_sink.go
@@ -50,6 +50,10 @@ func (localsink *LocalSink) GetSinkToDirectory() string {
return localsink.Dir
}
+func (localsink *LocalSink) IsIncremental() bool {
+ return true
+}
+
func (localsink *LocalSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool, signatures []int32) error {
if localsink.isMultiPartEntry(key) {
return nil
@@ -74,13 +78,13 @@ func (localsink *LocalSink) CreateEntry(key string, entry *filer_pb.Entry, signa
if _, err := os.Stat(dir); os.IsNotExist(err) {
glog.V(4).Infof("Create Direcotry key: %s", dir)
- if err = os.MkdirAll(dir, 0); err != nil {
+ if err = os.MkdirAll(dir, 0755); err != nil {
return err
}
}
writeFunc := func(data []byte) error {
- writeErr := ioutil.WriteFile(key, data, 0)
+ writeErr := ioutil.WriteFile(key, data, 0755)
return writeErr
}
diff --git a/weed/replication/sink/replication_sink.go b/weed/replication/sink/replication_sink.go
index cfc6e0a4d..4ffd09462 100644
--- a/weed/replication/sink/replication_sink.go
+++ b/weed/replication/sink/replication_sink.go
@@ -14,6 +14,7 @@ type ReplicationSink interface {
UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool, signatures []int32) (foundExistingEntry bool, err error)
GetSinkToDirectory() string
SetSourceFiler(s *source.FilerSource)
+ IsIncremental() bool
}
var (
diff --git a/weed/replication/sink/s3sink/s3_sink.go b/weed/replication/sink/s3sink/s3_sink.go
index 58432ee6b..9a36573e3 100644
--- a/weed/replication/sink/s3sink/s3_sink.go
+++ b/weed/replication/sink/s3sink/s3_sink.go
@@ -21,12 +21,13 @@ import (
)
type S3Sink struct {
- conn s3iface.S3API
- region string
- bucket string
- dir string
- endpoint string
- filerSource *source.FilerSource
+ conn s3iface.S3API
+ region string
+ bucket string
+ dir string
+ endpoint string
+ filerSource *source.FilerSource
+ isIncremental bool
}
func init() {
@@ -41,11 +42,17 @@ func (s3sink *S3Sink) GetSinkToDirectory() string {
return s3sink.dir
}
+func (s3sink *S3Sink) IsIncremental() bool {
+ return s3sink.isIncremental
+}
+
func (s3sink *S3Sink) Initialize(configuration util.Configuration, prefix string) error {
glog.V(0).Infof("sink.s3.region: %v", configuration.GetString(prefix+"region"))
glog.V(0).Infof("sink.s3.bucket: %v", configuration.GetString(prefix+"bucket"))
glog.V(0).Infof("sink.s3.directory: %v", configuration.GetString(prefix+"directory"))
glog.V(0).Infof("sink.s3.endpoint: %v", configuration.GetString(prefix+"endpoint"))
+ glog.V(0).Infof("sink.s3.is_incremental: %v", configuration.GetString(prefix+"is_incremental"))
+ s3sink.isIncremental = configuration.GetBool(prefix + "is_incremental")
return s3sink.initialize(
configuration.GetString(prefix+"aws_access_key_id"),
configuration.GetString(prefix+"aws_secret_access_key"),
@@ -67,8 +74,9 @@ func (s3sink *S3Sink) initialize(awsAccessKeyId, awsSecretAccessKey, region, buc
s3sink.endpoint = endpoint
config := &aws.Config{
- Region: aws.String(s3sink.region),
- Endpoint: aws.String(s3sink.endpoint),
+ Region: aws.String(s3sink.region),
+ Endpoint: aws.String(s3sink.endpoint),
+ S3ForcePathStyle: aws.Bool(true),
}
if awsAccessKeyId != "" && awsSecretAccessKey != "" {
config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, awsSecretAccessKey, "")
@@ -104,7 +112,7 @@ func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry, signatures
uploadId, err := s3sink.createMultipartUpload(key, entry)
if err != nil {
- return err
+ return fmt.Errorf("createMultipartUpload: %v", err)
}
totalSize := filer.FileSize(entry)
@@ -120,6 +128,7 @@ func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry, signatures
defer wg.Done()
if part, uploadErr := s3sink.uploadPart(key, uploadId, partId, chunk); uploadErr != nil {
err = uploadErr
+ glog.Errorf("uploadPart: %v", uploadErr)
} else {
parts[index] = part
}
@@ -129,7 +138,7 @@ func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry, signatures
if err != nil {
s3sink.abortMultipartUpload(key, uploadId)
- return err
+ return fmt.Errorf("uploadPart: %v", err)
}
return s3sink.completeMultipartUpload(context.Background(), key, uploadId, parts)
diff --git a/weed/replication/sink/s3sink/s3_write.go b/weed/replication/sink/s3sink/s3_write.go
index b172ea2c3..3dde52616 100644
--- a/weed/replication/sink/s3sink/s3_write.go
+++ b/weed/replication/sink/s3sink/s3_write.go
@@ -24,7 +24,7 @@ func (s3sink *S3Sink) deleteObject(key string) error {
result, err := s3sink.conn.DeleteObject(input)
if err == nil {
- glog.V(0).Infof("[%s] delete %s: %v", s3sink.bucket, key, result)
+ glog.V(2).Infof("[%s] delete %s: %v", s3sink.bucket, key, result)
} else {
glog.Errorf("[%s] delete %s: %v", s3sink.bucket, key, err)
}
@@ -43,7 +43,7 @@ func (s3sink *S3Sink) createMultipartUpload(key string, entry *filer_pb.Entry) (
result, err := s3sink.conn.CreateMultipartUpload(input)
if err == nil {
- glog.V(0).Infof("[%s] createMultipartUpload %s: %v", s3sink.bucket, key, result)
+ glog.V(2).Infof("[%s] createMultipartUpload %s: %v", s3sink.bucket, key, result)
} else {
glog.Errorf("[%s] createMultipartUpload %s: %v", s3sink.bucket, key, err)
return "", err
@@ -94,12 +94,13 @@ func (s3sink *S3Sink) completeMultipartUpload(ctx context.Context, key, uploadId
result, err := s3sink.conn.CompleteMultipartUpload(input)
if err == nil {
- glog.V(0).Infof("[%s] completeMultipartUpload %s: %v", s3sink.bucket, key, result)
+ glog.V(2).Infof("[%s] completeMultipartUpload %s: %v", s3sink.bucket, key, result)
} else {
glog.Errorf("[%s] completeMultipartUpload %s: %v", s3sink.bucket, key, err)
+ return fmt.Errorf("[%s] completeMultipartUpload %s: %v", s3sink.bucket, key, err)
}
- return err
+ return nil
}
// To upload a part
@@ -122,7 +123,7 @@ func (s3sink *S3Sink) uploadPart(key, uploadId string, partId int, chunk *filer.
result, err := s3sink.conn.UploadPart(input)
if err == nil {
- glog.V(0).Infof("[%s] uploadPart %s %d upload: %v", s3sink.bucket, key, partId, result)
+ glog.V(2).Infof("[%s] uploadPart %s %d upload: %v", s3sink.bucket, key, partId, result)
} else {
glog.Errorf("[%s] uploadPart %s %d upload: %v", s3sink.bucket, key, partId, err)
}
@@ -163,7 +164,7 @@ func (s3sink *S3Sink) buildReadSeeker(chunk *filer.ChunkView) (io.ReadSeeker, er
}
buf := make([]byte, chunk.Size)
for _, fileUrl := range fileUrls {
- _, err = util.ReadUrl(fileUrl+"?readDeleted=true", nil, false, false, chunk.Offset, int(chunk.Size), buf)
+ _, err = util.ReadUrl(fileUrl, chunk.CipherKey, chunk.IsGzipped, false, chunk.Offset, int(chunk.Size), buf)
if err != nil {
glog.V(1).Infof("read from %s: %v", fileUrl, err)
} else {
diff --git a/weed/replication/source/filer_source.go b/weed/replication/source/filer_source.go
index 3982360b0..e2e3575dc 100644
--- a/weed/replication/source/filer_source.go
+++ b/weed/replication/source/filer_source.go
@@ -83,8 +83,12 @@ func (fs *FilerSource) LookupFileId(part string) (fileUrls []string, err error)
return nil, fmt.Errorf("LookupFileId locate volume id %s: %v", vid, err)
}
- for _, loc := range locations.Locations {
- fileUrls = append(fileUrls, fmt.Sprintf("http://%s/%s", loc.Url, part))
+ if !fs.proxyByFiler {
+ for _, loc := range locations.Locations {
+ fileUrls = append(fileUrls, fmt.Sprintf("http://%s/%s?readDeleted=true", loc.Url, part))
+ }
+ } else {
+ fileUrls = append(fileUrls, fmt.Sprintf("http://%s/?proxyChunkId=%s", fs.address, part))
}
return
diff --git a/weed/s3api/filer_util.go b/weed/s3api/filer_util.go
index 3626ece98..1803332a3 100644
--- a/weed/s3api/filer_util.go
+++ b/weed/s3api/filer_util.go
@@ -31,6 +31,10 @@ func (s3a *S3ApiServer) list(parentDirectoryPath, prefix, startFrom string, incl
return nil
}, startFrom, inclusive, limit)
+ if len(entries) == 0 {
+ isLast = true
+ }
+
return
}
@@ -75,6 +79,12 @@ func (s3a *S3ApiServer) exists(parentDirectoryPath string, entryName string, isD
}
+func (s3a *S3ApiServer) touch(parentDirectoryPath string, entryName string, entry *filer_pb.Entry) (err error) {
+
+ return filer_pb.Touch(s3a, parentDirectoryPath, entryName, entry)
+
+}
+
func (s3a *S3ApiServer) getEntry(parentDirectoryPath, entryName string) (entry *filer_pb.Entry, err error) {
fullPath := util.NewFullPath(parentDirectoryPath, entryName)
return filer_pb.GetEntry(s3a, fullPath)
diff --git a/weed/s3api/s3api_bucket_handlers.go b/weed/s3api/s3api_bucket_handlers.go
index 338f82668..48e8cb047 100644
--- a/weed/s3api/s3api_bucket_handlers.go
+++ b/weed/s3api/s3api_bucket_handlers.go
@@ -51,7 +51,7 @@ func (s3a *S3ApiServer) ListBucketsHandler(w http.ResponseWriter, r *http.Reques
var buckets []*s3.Bucket
for _, entry := range entries {
if entry.IsDirectory {
- if identity != nil && !identity.canDo(s3_constants.ACTION_ADMIN, entry.Name) {
+ if identity != nil && !identity.canDo(s3_constants.ACTION_LIST, entry.Name) {
continue
}
buckets = append(buckets, &s3.Bucket{
diff --git a/weed/s3api/s3api_object_copy_handlers.go b/weed/s3api/s3api_object_copy_handlers.go
index ca578e7e5..84a85fd78 100644
--- a/weed/s3api/s3api_object_copy_handlers.go
+++ b/weed/s3api/s3api_object_copy_handlers.go
@@ -4,6 +4,7 @@ import (
"fmt"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/s3api/s3err"
+ weed_server "github.com/chrislusf/seaweedfs/weed/server"
"net/http"
"net/url"
"strconv"
@@ -25,6 +26,26 @@ func (s3a *S3ApiServer) CopyObjectHandler(w http.ResponseWriter, r *http.Request
}
srcBucket, srcObject := pathToBucketAndObject(cpSrcPath)
+
+ if (srcBucket == dstBucket && srcObject == dstObject || cpSrcPath == "") && isReplace(r) {
+ fullPath := util.FullPath(fmt.Sprintf("%s/%s%s", s3a.option.BucketsPath, dstBucket, dstObject))
+ dir, name := fullPath.DirAndName()
+ entry, err := s3a.getEntry(dir, name)
+ if err != nil {
+ writeErrorResponse(w, s3err.ErrInvalidCopySource, r.URL)
+ }
+ entry.Extended = weed_server.SaveAmzMetaData(r, entry.Extended, isReplace(r))
+ err = s3a.touch(dir, name, entry)
+ if err != nil {
+ writeErrorResponse(w, s3err.ErrInvalidCopySource, r.URL)
+ }
+ writeSuccessResponseXML(w, encodeResponse(CopyObjectResult{
+ ETag: fmt.Sprintf("%x", entry.Attributes.Md5),
+ LastModified: time.Now().UTC(),
+ }))
+ return
+ }
+
// If source object is empty or bucket is empty, reply back invalid copy source.
if srcObject == "" || srcBucket == "" {
writeErrorResponse(w, s3err.ErrInvalidCopySource, r.URL)
@@ -32,7 +53,7 @@ func (s3a *S3ApiServer) CopyObjectHandler(w http.ResponseWriter, r *http.Request
}
if srcBucket == dstBucket && srcObject == dstObject {
- writeErrorResponse(w, s3err.ErrInvalidCopySource, r.URL)
+ writeErrorResponse(w, s3err.ErrInvalidCopyDest, r.URL)
return
}
@@ -147,3 +168,7 @@ func (s3a *S3ApiServer) CopyObjectPartHandler(w http.ResponseWriter, r *http.Req
writeSuccessResponseXML(w, encodeResponse(response))
}
+
+func isReplace(r *http.Request) bool {
+ return r.Header.Get("X-Amz-Metadata-Directive") == "REPLACE"
+}
diff --git a/weed/s3api/s3api_object_handlers.go b/weed/s3api/s3api_object_handlers.go
index 19d85c495..b3cfd9ec7 100644
--- a/weed/s3api/s3api_object_handlers.go
+++ b/weed/s3api/s3api_object_handlers.go
@@ -5,9 +5,11 @@ import (
"encoding/json"
"encoding/xml"
"fmt"
+ "github.com/chrislusf/seaweedfs/weed/filer"
"io"
"io/ioutil"
"net/http"
+ "net/url"
"sort"
"strings"
@@ -69,7 +71,7 @@ func (s3a *S3ApiServer) PutObjectHandler(w http.ResponseWriter, r *http.Request)
return
}
} else {
- uploadUrl := fmt.Sprintf("http://%s%s/%s%s", s3a.option.Filer, s3a.option.BucketsPath, bucket, object)
+ uploadUrl := fmt.Sprintf("http://%s%s/%s%s", s3a.option.Filer, s3a.option.BucketsPath, bucket, urlPathEscape(object))
etag, errCode := s3a.putToFiler(r, uploadUrl, dataReader)
@@ -84,6 +86,14 @@ func (s3a *S3ApiServer) PutObjectHandler(w http.ResponseWriter, r *http.Request)
writeSuccessResponseEmpty(w)
}
+func urlPathEscape(object string) string {
+ var escapedParts []string
+ for _, part := range strings.Split(object, "/") {
+ escapedParts = append(escapedParts, url.PathEscape(part))
+ }
+ return strings.Join(escapedParts, "/")
+}
+
func (s3a *S3ApiServer) GetObjectHandler(w http.ResponseWriter, r *http.Request) {
bucket, object := getBucketAndObject(r)
@@ -94,7 +104,7 @@ func (s3a *S3ApiServer) GetObjectHandler(w http.ResponseWriter, r *http.Request)
}
destUrl := fmt.Sprintf("http://%s%s/%s%s",
- s3a.option.Filer, s3a.option.BucketsPath, bucket, object)
+ s3a.option.Filer, s3a.option.BucketsPath, bucket, urlPathEscape(object))
s3a.proxyToFiler(w, r, destUrl, passThroughResponse)
@@ -105,7 +115,7 @@ func (s3a *S3ApiServer) HeadObjectHandler(w http.ResponseWriter, r *http.Request
bucket, object := getBucketAndObject(r)
destUrl := fmt.Sprintf("http://%s%s/%s%s",
- s3a.option.Filer, s3a.option.BucketsPath, bucket, object)
+ s3a.option.Filer, s3a.option.BucketsPath, bucket, urlPathEscape(object))
s3a.proxyToFiler(w, r, destUrl, passThroughResponse)
@@ -116,7 +126,7 @@ func (s3a *S3ApiServer) DeleteObjectHandler(w http.ResponseWriter, r *http.Reque
bucket, object := getBucketAndObject(r)
destUrl := fmt.Sprintf("http://%s%s/%s%s?recursive=true",
- s3a.option.Filer, s3a.option.BucketsPath, bucket, object)
+ s3a.option.Filer, s3a.option.BucketsPath, bucket, urlPathEscape(object))
s3a.proxyToFiler(w, r, destUrl, func(proxyResponse *http.Response, w http.ResponseWriter) {
for k, v := range proxyResponse.Header {
@@ -196,6 +206,8 @@ func (s3a *S3ApiServer) DeleteMultipleObjectsHandler(w http.ResponseWriter, r *h
if err == nil {
directoriesWithDeletion[parentDirectoryPath]++
deletedObjects = append(deletedObjects, object)
+ } else if strings.Contains(err.Error(), filer.MsgFailDelNonEmptyFolder) {
+ deletedObjects = append(deletedObjects, object)
} else {
delete(directoriesWithDeletion, parentDirectoryPath)
deleteErrors = append(deleteErrors, DeleteError{
@@ -299,7 +311,7 @@ func (s3a *S3ApiServer) proxyToFiler(w http.ResponseWriter, r *http.Request, des
}
defer util.CloseResponse(resp)
- if (resp.ContentLength == -1 || resp.StatusCode == 404) && !strings.HasSuffix(destUrl, "/") {
+ if resp.ContentLength == -1 || resp.StatusCode == 404 {
if r.Method != "DELETE" {
writeErrorResponse(w, s3err.ErrNoSuchKey, r.URL)
return
@@ -314,7 +326,11 @@ func passThroughResponse(proxyResponse *http.Response, w http.ResponseWriter) {
for k, v := range proxyResponse.Header {
w.Header()[k] = v
}
- w.WriteHeader(proxyResponse.StatusCode)
+ if proxyResponse.Header.Get("Content-Range") != "" && proxyResponse.StatusCode == 200 {
+ w.WriteHeader(http.StatusPartialContent)
+ } else {
+ w.WriteHeader(proxyResponse.StatusCode)
+ }
io.Copy(w, proxyResponse.Body)
}
diff --git a/weed/s3api/s3api_object_handlers_postpolicy.go b/weed/s3api/s3api_object_handlers_postpolicy.go
index 044e732db..035302ae6 100644
--- a/weed/s3api/s3api_object_handlers_postpolicy.go
+++ b/weed/s3api/s3api_object_handlers_postpolicy.go
@@ -110,7 +110,7 @@ func (s3a *S3ApiServer) PostPolicyBucketHandler(w http.ResponseWriter, r *http.R
}
}
- uploadUrl := fmt.Sprintf("http://%s%s/%s/%s", s3a.option.Filer, s3a.option.BucketsPath, bucket, object)
+ uploadUrl := fmt.Sprintf("http://%s%s/%s%s", s3a.option.Filer, s3a.option.BucketsPath, bucket, urlPathEscape(object))
etag, errCode := s3a.putToFiler(r, uploadUrl, fileBody)
diff --git a/weed/s3api/s3api_objects_list_handlers.go b/weed/s3api/s3api_objects_list_handlers.go
index 2d36c6ec9..739cdd8f9 100644
--- a/weed/s3api/s3api_objects_list_handlers.go
+++ b/weed/s3api/s3api_objects_list_handlers.go
@@ -206,7 +206,6 @@ func (s3a *S3ApiServer) doListFilerEntries(client filer_pb.SeaweedFilerClient, d
isTruncated = isTruncated || subIsTruncated
maxKeys -= subCounter
nextMarker = subDir + "/" + subNextMarker
- counter += subCounter
// finished processing this sub directory
marker = subDir
}
diff --git a/weed/security/tls.go b/weed/security/tls.go
index b4bf84e2d..7d3ffcdca 100644
--- a/weed/security/tls.go
+++ b/weed/security/tls.go
@@ -1,10 +1,16 @@
package security
import (
+ "context"
"crypto/tls"
"crypto/x509"
"github.com/chrislusf/seaweedfs/weed/util"
+ grpc_auth "github.com/grpc-ecosystem/go-grpc-middleware/auth"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/peer"
+ "google.golang.org/grpc/status"
"io/ioutil"
+ "strings"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
@@ -12,21 +18,29 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
)
-func LoadServerTLS(config *util.ViperProxy, component string) grpc.ServerOption {
+type Authenticator struct {
+ AllowedWildcardDomain string
+ AllowedCommonNames map[string]bool
+}
+
+func LoadServerTLS(config *util.ViperProxy, component string) (grpc.ServerOption, grpc.ServerOption) {
if config == nil {
- return nil
+ return nil, nil
}
// load cert/key, ca cert
cert, err := tls.LoadX509KeyPair(config.GetString(component+".cert"), config.GetString(component+".key"))
if err != nil {
- glog.V(1).Infof("load cert/key error: %v", err)
- return nil
+ glog.V(1).Infof("load cert: %s / key: %s error: %v",
+ config.GetString(component+".cert"),
+ config.GetString(component+".key"),
+ err)
+ return nil, nil
}
caCert, err := ioutil.ReadFile(config.GetString("grpc.ca"))
if err != nil {
- glog.V(1).Infof("read ca cert file error: %v", err)
- return nil
+ glog.V(1).Infof("read ca cert file %s error: %v", config.GetString("grpc.ca"), err)
+ return nil, nil
}
caCertPool := x509.NewCertPool()
caCertPool.AppendCertsFromPEM(caCert)
@@ -36,7 +50,20 @@ func LoadServerTLS(config *util.ViperProxy, component string) grpc.ServerOption
ClientAuth: tls.RequireAndVerifyClientCert,
})
- return grpc.Creds(ta)
+ allowedCommonNames := config.GetString(component + ".allowed_commonNames")
+ allowedWildcardDomain := config.GetString("grpc.allowed_wildcard_domain")
+ if allowedCommonNames != "" || allowedWildcardDomain != "" {
+ allowedCommonNamesMap := make(map[string]bool)
+ for _, s := range strings.Split(allowedCommonNames, ",") {
+ allowedCommonNamesMap[s] = true
+ }
+ auther := Authenticator{
+ AllowedCommonNames: allowedCommonNamesMap,
+ AllowedWildcardDomain: allowedWildcardDomain,
+ }
+ return grpc.Creds(ta), grpc.UnaryInterceptor(grpc_auth.UnaryServerInterceptor(auther.Authenticate))
+ }
+ return grpc.Creds(ta), nil
}
func LoadClientTLS(config *util.ViperProxy, component string) grpc.DialOption {
@@ -70,3 +97,28 @@ func LoadClientTLS(config *util.ViperProxy, component string) grpc.DialOption {
})
return grpc.WithTransportCredentials(ta)
}
+
+func (a Authenticator) Authenticate(ctx context.Context) (newCtx context.Context, err error) {
+ p, ok := peer.FromContext(ctx)
+ if !ok {
+ return ctx, status.Error(codes.Unauthenticated, "no peer found")
+ }
+
+ tlsAuth, ok := p.AuthInfo.(credentials.TLSInfo)
+ if !ok {
+ return ctx, status.Error(codes.Unauthenticated, "unexpected peer transport credentials")
+ }
+ if len(tlsAuth.State.VerifiedChains) == 0 || len(tlsAuth.State.VerifiedChains[0]) == 0 {
+ return ctx, status.Error(codes.Unauthenticated, "could not verify peer certificate")
+ }
+
+ commonName := tlsAuth.State.VerifiedChains[0][0].Subject.CommonName
+ if a.AllowedWildcardDomain != "" && strings.HasSuffix(commonName, a.AllowedWildcardDomain) {
+ return ctx, nil
+ }
+ if _, ok := a.AllowedCommonNames[commonName]; ok {
+ return ctx, nil
+ }
+
+ return ctx, status.Errorf(codes.Unauthenticated, "invalid subject common name: %s", commonName)
+}
diff --git a/weed/sequence/snowflake_sequencer.go b/weed/sequence/snowflake_sequencer.go
new file mode 100644
index 000000000..300449fa0
--- /dev/null
+++ b/weed/sequence/snowflake_sequencer.go
@@ -0,0 +1,46 @@
+package sequence
+
+import (
+ "fmt"
+ "hash/fnv"
+
+ "github.com/bwmarrin/snowflake"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+)
+
+// a simple snowflake Sequencer
+type SnowflakeSequencer struct {
+ node *snowflake.Node
+}
+
+func NewSnowflakeSequencer(nodeid string) (*SnowflakeSequencer, error) {
+ nodeid_hash := hash(nodeid) & 0x3ff
+ glog.V(0).Infof("use snowflake seq id generator, nodeid:%s hex_of_nodeid: %x", nodeid, nodeid_hash)
+ node, err := snowflake.NewNode(int64(nodeid_hash))
+ if err != nil {
+ fmt.Println(err)
+ return nil, err
+ }
+
+ sequencer := &SnowflakeSequencer{node: node}
+ return sequencer, nil
+}
+
+func hash(s string) uint32 {
+ h := fnv.New32a()
+ h.Write([]byte(s))
+ return h.Sum32()
+}
+
+func (m *SnowflakeSequencer) NextFileId(count uint64) uint64 {
+ return uint64(m.node.Generate().Int64())
+}
+
+// ignore setmax as we are snowflake
+func (m *SnowflakeSequencer) SetMax(seenValue uint64) {
+}
+
+// return a new id as no Peek is stored
+func (m *SnowflakeSequencer) Peek() uint64 {
+ return uint64(m.node.Generate().Int64())
+}
diff --git a/weed/server/common.go b/weed/server/common.go
index 9001a3b33..5c5f1b8eb 100644
--- a/weed/server/common.go
+++ b/weed/server/common.go
@@ -234,12 +234,12 @@ func adjustHeaderContentDisposition(w http.ResponseWriter, r *http.Request, file
}
}
-func processRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64, mimeType string, writeFn func(writer io.Writer, offset int64, size int64, httpStatusCode int) error) {
+func processRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64, mimeType string, writeFn func(writer io.Writer, offset int64, size int64) error) {
rangeReq := r.Header.Get("Range")
if rangeReq == "" {
w.Header().Set("Content-Length", strconv.FormatInt(totalSize, 10))
- if err := writeFn(w, 0, totalSize, 0); err != nil {
+ if err := writeFn(w, 0, totalSize); err != nil {
http.Error(w, err.Error(), http.StatusInternalServerError)
return
}
@@ -279,7 +279,7 @@ func processRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
w.Header().Set("Content-Length", strconv.FormatInt(ra.length, 10))
w.Header().Set("Content-Range", ra.contentRange(totalSize))
- err = writeFn(w, ra.start, ra.length, http.StatusPartialContent)
+ err = writeFn(w, ra.start, ra.length)
if err != nil {
http.Error(w, err.Error(), http.StatusInternalServerError)
return
@@ -307,7 +307,7 @@ func processRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
pw.CloseWithError(e)
return
}
- if e = writeFn(part, ra.start, ra.length, 0); e != nil {
+ if e = writeFn(part, ra.start, ra.length); e != nil {
pw.CloseWithError(e)
return
}
diff --git a/weed/server/filer_grpc_server_rename.go b/weed/server/filer_grpc_server_rename.go
index 5b68b64de..c1e5bc789 100644
--- a/weed/server/filer_grpc_server_rename.go
+++ b/weed/server/filer_grpc_server_rename.go
@@ -33,8 +33,7 @@ func (fs *FilerServer) AtomicRenameEntry(ctx context.Context, req *filer_pb.Atom
return nil, fmt.Errorf("%s/%s not found: %v", req.OldDirectory, req.OldName, err)
}
- var events MoveEvents
- moveErr := fs.moveEntry(ctx, oldParent, oldEntry, newParent, req.NewName, &events)
+ moveErr := fs.moveEntry(ctx, oldParent, oldEntry, newParent, req.NewName)
if moveErr != nil {
fs.filer.RollbackTransaction(ctx)
return nil, fmt.Errorf("%s/%s move error: %v", req.OldDirectory, req.OldName, moveErr)
@@ -48,11 +47,11 @@ func (fs *FilerServer) AtomicRenameEntry(ctx context.Context, req *filer_pb.Atom
return &filer_pb.AtomicRenameEntryResponse{}, nil
}
-func (fs *FilerServer) moveEntry(ctx context.Context, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, events *MoveEvents) error {
+func (fs *FilerServer) moveEntry(ctx context.Context, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string) error {
- if err := fs.moveSelfEntry(ctx, oldParent, entry, newParent, newName, events, func() error {
+ if err := fs.moveSelfEntry(ctx, oldParent, entry, newParent, newName, func() error {
if entry.IsDirectory() {
- if err := fs.moveFolderSubEntries(ctx, oldParent, entry, newParent, newName, events); err != nil {
+ if err := fs.moveFolderSubEntries(ctx, oldParent, entry, newParent, newName); err != nil {
return err
}
}
@@ -64,7 +63,7 @@ func (fs *FilerServer) moveEntry(ctx context.Context, oldParent util.FullPath, e
return nil
}
-func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, events *MoveEvents) error {
+func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string) error {
currentDirPath := oldParent.Child(entry.Name())
newDirPath := newParent.Child(newName)
@@ -85,7 +84,7 @@ func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, oldParent util.
for _, item := range entries {
lastFileName = item.Name()
// println("processing", lastFileName)
- err := fs.moveEntry(ctx, currentDirPath, item, newDirPath, item.Name(), events)
+ err := fs.moveEntry(ctx, currentDirPath, item, newDirPath, item.Name())
if err != nil {
return err
}
@@ -97,8 +96,7 @@ func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, oldParent util.
return nil
}
-func (fs *FilerServer) moveSelfEntry(ctx context.Context, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, events *MoveEvents,
- moveFolderSubEntries func() error) error {
+func (fs *FilerServer) moveSelfEntry(ctx context.Context, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, moveFolderSubEntries func() error) error {
oldPath, newPath := oldParent.Child(entry.Name()), newParent.Child(newName)
@@ -122,8 +120,6 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, oldParent util.FullPat
return createErr
}
- events.newEntries = append(events.newEntries, newEntry)
-
if moveFolderSubEntries != nil {
if moveChildrenErr := moveFolderSubEntries(); moveChildrenErr != nil {
return moveChildrenErr
@@ -136,13 +132,6 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, oldParent util.FullPat
return deleteErr
}
- events.oldEntries = append(events.oldEntries, entry)
-
return nil
}
-
-type MoveEvents struct {
- oldEntries []*filer.Entry
- newEntries []*filer.Entry
-}
diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go
index 22474a5e2..2734223ea 100644
--- a/weed/server/filer_server.go
+++ b/weed/server/filer_server.go
@@ -45,22 +45,23 @@ import (
)
type FilerOption struct {
- Masters []string
- Collection string
- DefaultReplication string
- DisableDirListing bool
- MaxMB int
- DirListingLimit int
- DataCenter string
- Rack string
- DefaultLevelDbDir string
- DisableHttp bool
- Host string
- Port uint32
- recursiveDelete bool
- Cipher bool
- SaveToFilerLimit int
- Filers []string
+ Masters []string
+ Collection string
+ DefaultReplication string
+ DisableDirListing bool
+ MaxMB int
+ DirListingLimit int
+ DataCenter string
+ Rack string
+ DefaultLevelDbDir string
+ DisableHttp bool
+ Host string
+ Port uint32
+ recursiveDelete bool
+ Cipher bool
+ SaveToFilerLimit int64
+ Filers []string
+ ConcurrentUploadLimit int64
}
type FilerServer struct {
@@ -79,14 +80,18 @@ type FilerServer struct {
brokers map[string]map[string]bool
brokersLock sync.Mutex
+
+ inFlightDataSize int64
+ inFlightDataLimitCond *sync.Cond
}
func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption) (fs *FilerServer, err error) {
fs = &FilerServer{
- option: option,
- grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.filer"),
- brokers: make(map[string]map[string]bool),
+ option: option,
+ grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.filer"),
+ brokers: make(map[string]map[string]bool),
+ inFlightDataLimitCond: sync.NewCond(new(sync.Mutex)),
}
fs.listenersCond = sync.NewCond(&fs.listenersLock)
@@ -153,7 +158,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
func (fs *FilerServer) checkWithMaster() {
for _, master := range fs.option.Masters {
- _, err := pb.ParseFilerGrpcAddress(master)
+ _, err := pb.ParseServerToGrpcAddress(master)
if err != nil {
glog.Fatalf("invalid master address %s: %v", master, err)
}
diff --git a/weed/server/filer_server_handlers.go b/weed/server/filer_server_handlers.go
index 3bc0c5d0d..ed6bbb6f6 100644
--- a/weed/server/filer_server_handlers.go
+++ b/weed/server/filer_server_handlers.go
@@ -4,6 +4,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/util"
"net/http"
"strings"
+ "sync/atomic"
"time"
"github.com/chrislusf/seaweedfs/weed/stats"
@@ -47,18 +48,34 @@ func (fs *FilerServer) filerHandler(w http.ResponseWriter, r *http.Request) {
fs.DeleteHandler(w, r)
}
stats.FilerRequestHistogram.WithLabelValues("delete").Observe(time.Since(start).Seconds())
- case "PUT":
- stats.FilerRequestCounter.WithLabelValues("put").Inc()
- if _, ok := r.URL.Query()["tagging"]; ok {
- fs.PutTaggingHandler(w, r)
- } else {
- fs.PostHandler(w, r)
+ case "POST", "PUT":
+
+ // wait until in flight data is less than the limit
+ contentLength := getContentLength(r)
+ fs.inFlightDataLimitCond.L.Lock()
+ for atomic.LoadInt64(&fs.inFlightDataSize) > fs.option.ConcurrentUploadLimit {
+ fs.inFlightDataLimitCond.Wait()
+ }
+ atomic.AddInt64(&fs.inFlightDataSize, contentLength)
+ fs.inFlightDataLimitCond.L.Unlock()
+ defer func() {
+ atomic.AddInt64(&fs.inFlightDataSize, -contentLength)
+ fs.inFlightDataLimitCond.Signal()
+ }()
+
+ if r.Method == "PUT" {
+ stats.FilerRequestCounter.WithLabelValues("put").Inc()
+ if _, ok := r.URL.Query()["tagging"]; ok {
+ fs.PutTaggingHandler(w, r)
+ } else {
+ fs.PostHandler(w, r, contentLength)
+ }
+ stats.FilerRequestHistogram.WithLabelValues("put").Observe(time.Since(start).Seconds())
+ } else { // method == "POST"
+ stats.FilerRequestCounter.WithLabelValues("post").Inc()
+ fs.PostHandler(w, r, contentLength)
+ stats.FilerRequestHistogram.WithLabelValues("post").Observe(time.Since(start).Seconds())
}
- stats.FilerRequestHistogram.WithLabelValues("put").Observe(time.Since(start).Seconds())
- case "POST":
- stats.FilerRequestCounter.WithLabelValues("post").Inc()
- fs.PostHandler(w, r)
- stats.FilerRequestHistogram.WithLabelValues("post").Observe(time.Since(start).Seconds())
case "OPTIONS":
stats.FilerRequestCounter.WithLabelValues("options").Inc()
OptionsHandler(w, r, false)
diff --git a/weed/server/filer_server_handlers_read.go b/weed/server/filer_server_handlers_read.go
index 160ee9d97..f90b070a2 100644
--- a/weed/server/filer_server_handlers_read.go
+++ b/weed/server/filer_server_handlers_read.go
@@ -131,6 +131,9 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request,
if r.Method == "HEAD" {
w.Header().Set("Content-Length", strconv.FormatInt(totalSize, 10))
+ processRangeRequest(r, w, totalSize, mimeType, func(writer io.Writer, offset int64, size int64) error {
+ return filer.StreamContent(fs.filer.MasterClient, writer, entry.Chunks, offset, size, true)
+ })
return
}
@@ -150,15 +153,15 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request,
}
}
- processRangeRequest(r, w, totalSize, mimeType, func(writer io.Writer, offset int64, size int64, httpStatusCode int) error {
- if httpStatusCode != 0 {
- w.WriteHeader(httpStatusCode)
- }
+ processRangeRequest(r, w, totalSize, mimeType, func(writer io.Writer, offset int64, size int64) error {
if offset+size <= int64(len(entry.Content)) {
_, err := writer.Write(entry.Content[offset : offset+size])
+ if err != nil {
+ glog.Errorf("failed to write entry content: %v", err)
+ }
return err
}
- return filer.StreamContent(fs.filer.MasterClient, writer, entry.Chunks, offset, size)
+ return filer.StreamContent(fs.filer.MasterClient, writer, entry.Chunks, offset, size, false)
})
}
diff --git a/weed/server/filer_server_handlers_write.go b/weed/server/filer_server_handlers_write.go
index 0ce3e4a58..95eba9d3d 100644
--- a/weed/server/filer_server_handlers_write.go
+++ b/weed/server/filer_server_handlers_write.go
@@ -52,7 +52,7 @@ func (fs *FilerServer) assignNewFileInfo(so *operation.StorageOption) (fileId, u
return
}
-func (fs *FilerServer) PostHandler(w http.ResponseWriter, r *http.Request) {
+func (fs *FilerServer) PostHandler(w http.ResponseWriter, r *http.Request, contentLength int64) {
ctx := context.Background()
@@ -66,7 +66,7 @@ func (fs *FilerServer) PostHandler(w http.ResponseWriter, r *http.Request) {
query.Get("rack"),
)
- fs.autoChunk(ctx, w, r, so)
+ fs.autoChunk(ctx, w, r, contentLength, so)
util.CloseRequest(r)
}
diff --git a/weed/server/filer_server_handlers_write_autochunk.go b/weed/server/filer_server_handlers_write_autochunk.go
index d3ce7e605..2808042c7 100644
--- a/weed/server/filer_server_handlers_write_autochunk.go
+++ b/weed/server/filer_server_handlers_write_autochunk.go
@@ -2,11 +2,8 @@ package weed_server
import (
"context"
- "crypto/md5"
"fmt"
- "hash"
"io"
- "io/ioutil"
"net/http"
"os"
"path"
@@ -19,13 +16,12 @@ import (
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
xhttp "github.com/chrislusf/seaweedfs/weed/s3api/http"
- "github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/stats"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/util"
)
-func (fs *FilerServer) autoChunk(ctx context.Context, w http.ResponseWriter, r *http.Request, so *operation.StorageOption) {
+func (fs *FilerServer) autoChunk(ctx context.Context, w http.ResponseWriter, r *http.Request, contentLength int64, so *operation.StorageOption) {
// autoChunking can be set at the command-line level or as a query param. Query param overrides command-line
query := r.URL.Query()
@@ -38,10 +34,10 @@ func (fs *FilerServer) autoChunk(ctx context.Context, w http.ResponseWriter, r *
chunkSize := 1024 * 1024 * maxMB
- stats.FilerRequestCounter.WithLabelValues("postAutoChunk").Inc()
+ stats.FilerRequestCounter.WithLabelValues("chunk").Inc()
start := time.Now()
defer func() {
- stats.FilerRequestHistogram.WithLabelValues("postAutoChunk").Observe(time.Since(start).Seconds())
+ stats.FilerRequestHistogram.WithLabelValues("chunk").Observe(time.Since(start).Seconds())
}()
var reply *FilerPostResult
@@ -51,14 +47,16 @@ func (fs *FilerServer) autoChunk(ctx context.Context, w http.ResponseWriter, r *
if r.Header.Get("Content-Type") == "" && strings.HasSuffix(r.URL.Path, "/") {
reply, err = fs.mkdir(ctx, w, r)
} else {
- reply, md5bytes, err = fs.doPostAutoChunk(ctx, w, r, chunkSize, so)
+ reply, md5bytes, err = fs.doPostAutoChunk(ctx, w, r, chunkSize, contentLength, so)
}
} else {
- reply, md5bytes, err = fs.doPutAutoChunk(ctx, w, r, chunkSize, so)
+ reply, md5bytes, err = fs.doPutAutoChunk(ctx, w, r, chunkSize, contentLength, so)
}
if err != nil {
if strings.HasPrefix(err.Error(), "read input:") {
writeJsonError(w, r, 499, err)
+ } else if strings.HasSuffix(err.Error(), "is a file") {
+ writeJsonError(w, r, http.StatusConflict, err)
} else {
writeJsonError(w, r, http.StatusInternalServerError, err)
}
@@ -70,7 +68,7 @@ func (fs *FilerServer) autoChunk(ctx context.Context, w http.ResponseWriter, r *
}
}
-func (fs *FilerServer) doPostAutoChunk(ctx context.Context, w http.ResponseWriter, r *http.Request, chunkSize int32, so *operation.StorageOption) (filerResult *FilerPostResult, md5bytes []byte, replyerr error) {
+func (fs *FilerServer) doPostAutoChunk(ctx context.Context, w http.ResponseWriter, r *http.Request, chunkSize int32, contentLength int64, so *operation.StorageOption) (filerResult *FilerPostResult, md5bytes []byte, replyerr error) {
multipartReader, multipartReaderErr := r.MultipartReader()
if multipartReaderErr != nil {
@@ -91,7 +89,7 @@ func (fs *FilerServer) doPostAutoChunk(ctx context.Context, w http.ResponseWrite
contentType = ""
}
- fileChunks, md5Hash, chunkOffset, err, smallContent := fs.uploadReaderToChunks(w, r, part1, chunkSize, fileName, contentType, so)
+ fileChunks, md5Hash, chunkOffset, err, smallContent := fs.uploadReaderToChunks(w, r, part1, chunkSize, fileName, contentType, contentLength, so)
if err != nil {
return nil, nil, err
}
@@ -102,7 +100,7 @@ func (fs *FilerServer) doPostAutoChunk(ctx context.Context, w http.ResponseWrite
return
}
-func (fs *FilerServer) doPutAutoChunk(ctx context.Context, w http.ResponseWriter, r *http.Request, chunkSize int32, so *operation.StorageOption) (filerResult *FilerPostResult, md5bytes []byte, replyerr error) {
+func (fs *FilerServer) doPutAutoChunk(ctx context.Context, w http.ResponseWriter, r *http.Request, chunkSize int32, contentLength int64, so *operation.StorageOption) (filerResult *FilerPostResult, md5bytes []byte, replyerr error) {
fileName := path.Base(r.URL.Path)
contentType := r.Header.Get("Content-Type")
@@ -110,7 +108,7 @@ func (fs *FilerServer) doPutAutoChunk(ctx context.Context, w http.ResponseWriter
contentType = ""
}
- fileChunks, md5Hash, chunkOffset, err, smallContent := fs.uploadReaderToChunks(w, r, r.Body, chunkSize, fileName, contentType, so)
+ fileChunks, md5Hash, chunkOffset, err, smallContent := fs.uploadReaderToChunks(w, r, r.Body, chunkSize, fileName, contentType, contentLength, so)
if err != nil {
return nil, nil, err
}
@@ -212,7 +210,7 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
entry.Extended = make(map[string][]byte)
}
- fs.saveAmzMetaData(r, entry)
+ SaveAmzMetaData(r, entry.Extended, false)
for k, v := range r.Header {
if len(v) > 0 && strings.HasPrefix(k, needle.PairNamePrefix) {
@@ -229,89 +227,6 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
return filerResult, replyerr
}
-func (fs *FilerServer) uploadReaderToChunks(w http.ResponseWriter, r *http.Request, reader io.Reader, chunkSize int32, fileName, contentType string, so *operation.StorageOption) ([]*filer_pb.FileChunk, hash.Hash, int64, error, []byte) {
- var fileChunks []*filer_pb.FileChunk
-
- md5Hash := md5.New()
- var partReader = ioutil.NopCloser(io.TeeReader(reader, md5Hash))
-
- chunkOffset := int64(0)
- var smallContent []byte
-
- for {
- limitedReader := io.LimitReader(partReader, int64(chunkSize))
-
- data, err := ioutil.ReadAll(limitedReader)
- if err != nil {
- return nil, nil, 0, err, nil
- }
- if chunkOffset == 0 && !isAppend(r) {
- if len(data) < fs.option.SaveToFilerLimit || strings.HasPrefix(r.URL.Path, filer.DirectoryEtcRoot) && len(data) < 4*1024 {
- smallContent = data
- chunkOffset += int64(len(data))
- break
- }
- }
- dataReader := util.NewBytesReader(data)
-
- // retry to assign a different file id
- var fileId, urlLocation string
- var auth security.EncodedJwt
- var assignErr, uploadErr error
- var uploadResult *operation.UploadResult
- for i := 0; i < 3; i++ {
- // assign one file id for one chunk
- fileId, urlLocation, auth, assignErr = fs.assignNewFileInfo(so)
- if assignErr != nil {
- return nil, nil, 0, assignErr, nil
- }
-
- // upload the chunk to the volume server
- uploadResult, uploadErr, _ = fs.doUpload(urlLocation, w, r, dataReader, fileName, contentType, nil, auth)
- if uploadErr != nil {
- time.Sleep(251 * time.Millisecond)
- continue
- }
- break
- }
- if uploadErr != nil {
- return nil, nil, 0, uploadErr, nil
- }
-
- // if last chunk exhausted the reader exactly at the border
- if uploadResult.Size == 0 {
- break
- }
-
- // Save to chunk manifest structure
- fileChunks = append(fileChunks, uploadResult.ToPbFileChunk(fileId, chunkOffset))
-
- glog.V(4).Infof("uploaded %s chunk %d to %s [%d,%d)", fileName, len(fileChunks), fileId, chunkOffset, chunkOffset+int64(uploadResult.Size))
-
- // reset variables for the next chunk
- chunkOffset = chunkOffset + int64(uploadResult.Size)
-
- // if last chunk was not at full chunk size, but already exhausted the reader
- if int64(uploadResult.Size) < int64(chunkSize) {
- break
- }
- }
-
- return fileChunks, md5Hash, chunkOffset, nil, smallContent
-}
-
-func (fs *FilerServer) doUpload(urlLocation string, w http.ResponseWriter, r *http.Request, limitedReader io.Reader, fileName string, contentType string, pairMap map[string]string, auth security.EncodedJwt) (*operation.UploadResult, error, []byte) {
-
- stats.FilerRequestCounter.WithLabelValues("postAutoChunkUpload").Inc()
- start := time.Now()
- defer func() {
- stats.FilerRequestHistogram.WithLabelValues("postAutoChunkUpload").Observe(time.Since(start).Seconds())
- }()
-
- uploadResult, err, data := operation.Upload(urlLocation, fileName, fs.option.Cipher, limitedReader, false, contentType, pairMap, auth)
- return uploadResult, err, data
-}
-
func (fs *FilerServer) saveAsChunk(so *operation.StorageOption) filer.SaveDataAsChunkFunctionType {
return func(reader io.Reader, name string, offset int64) (*filer_pb.FileChunk, string, string, error) {
@@ -380,17 +295,24 @@ func (fs *FilerServer) mkdir(ctx context.Context, w http.ResponseWriter, r *http
return filerResult, replyerr
}
-func (fs *FilerServer) saveAmzMetaData(r *http.Request, entry *filer.Entry) {
+func SaveAmzMetaData(r *http.Request, existing map[string][]byte, isReplace bool) (metadata map[string][]byte) {
+
+ metadata = make(map[string][]byte)
+ if !isReplace {
+ for k, v := range existing {
+ metadata[k] = v
+ }
+ }
if sc := r.Header.Get(xhttp.AmzStorageClass); sc != "" {
- entry.Extended[xhttp.AmzStorageClass] = []byte(sc)
+ metadata[xhttp.AmzStorageClass] = []byte(sc)
}
if tags := r.Header.Get(xhttp.AmzObjectTagging); tags != "" {
for _, v := range strings.Split(tags, "&") {
tag := strings.Split(v, "=")
if len(tag) == 2 {
- entry.Extended[xhttp.AmzObjectTagging+"-"+tag[0]] = []byte(tag[1])
+ metadata[xhttp.AmzObjectTagging+"-"+tag[0]] = []byte(tag[1])
}
}
}
@@ -398,8 +320,11 @@ func (fs *FilerServer) saveAmzMetaData(r *http.Request, entry *filer.Entry) {
for header, values := range r.Header {
if strings.HasPrefix(header, xhttp.AmzUserMetaPrefix) {
for _, value := range values {
- entry.Extended[header] = []byte(value)
+ metadata[header] = []byte(value)
}
}
}
+
+ return
+
}
diff --git a/weed/server/filer_server_handlers_write_upload.go b/weed/server/filer_server_handlers_write_upload.go
new file mode 100644
index 000000000..81b2ce1b0
--- /dev/null
+++ b/weed/server/filer_server_handlers_write_upload.go
@@ -0,0 +1,175 @@
+package weed_server
+
+import (
+ "crypto/md5"
+ "hash"
+ "io"
+ "io/ioutil"
+ "net/http"
+ "runtime"
+ "strings"
+ "sync"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/filer"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+ "github.com/chrislusf/seaweedfs/weed/security"
+ "github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+var (
+ limitedUploadProcessor = util.NewLimitedOutOfOrderProcessor(int32(runtime.NumCPU()))
+)
+
+func (fs *FilerServer) uploadReaderToChunks(w http.ResponseWriter, r *http.Request, reader io.Reader, chunkSize int32, fileName, contentType string, contentLength int64, so *operation.StorageOption) (fileChunks []*filer_pb.FileChunk, md5Hash hash.Hash, dataSize int64, err error, smallContent []byte) {
+
+ md5Hash = md5.New()
+ var partReader = ioutil.NopCloser(io.TeeReader(reader, md5Hash))
+
+ // save small content directly
+ if !isAppend(r) && ((0 < contentLength && contentLength < fs.option.SaveToFilerLimit) || strings.HasPrefix(r.URL.Path, filer.DirectoryEtcRoot) && contentLength < 4*1024) {
+ smallContent, err = ioutil.ReadAll(partReader)
+ dataSize = int64(len(smallContent))
+ return
+ }
+
+ resultsChan := make(chan *ChunkCreationResult, 2)
+
+ var waitForAllData sync.WaitGroup
+ waitForAllData.Add(1)
+ go func() {
+ // process upload results
+ defer waitForAllData.Done()
+ for result := range resultsChan {
+ if result.err != nil {
+ err = result.err
+ continue
+ }
+
+ // Save to chunk manifest structure
+ fileChunks = append(fileChunks, result.chunk)
+ }
+ }()
+
+ var lock sync.Mutex
+ readOffset := int64(0)
+ var wg sync.WaitGroup
+
+ for err == nil {
+
+ wg.Add(1)
+ request := func() {
+ defer wg.Done()
+
+ var localOffset int64
+ // read from the input
+ lock.Lock()
+ localOffset = readOffset
+ limitedReader := io.LimitReader(partReader, int64(chunkSize))
+ data, readErr := ioutil.ReadAll(limitedReader)
+ readOffset += int64(len(data))
+ lock.Unlock()
+ // handle read errors
+ if readErr != nil {
+ if err == nil {
+ err = readErr
+ }
+ if readErr != io.EOF {
+ resultsChan <- &ChunkCreationResult{
+ err: readErr,
+ }
+ }
+ return
+ }
+ if len(data) == 0 {
+ readErr = io.EOF
+ if err == nil {
+ err = readErr
+ }
+ return
+ }
+
+ // upload
+ dataReader := util.NewBytesReader(data)
+ fileId, uploadResult, uploadErr := fs.doCreateChunk(w, r, so, dataReader, fileName, contentType)
+ if uploadErr != nil {
+ if err == nil {
+ err = uploadErr
+ }
+ resultsChan <- &ChunkCreationResult{
+ err: uploadErr,
+ }
+ return
+ }
+
+ glog.V(4).Infof("uploaded %s to %s [%d,%d)", fileName, fileId, localOffset, localOffset+int64(uploadResult.Size))
+
+ // send back uploaded file chunk
+ resultsChan <- &ChunkCreationResult{
+ chunk: uploadResult.ToPbFileChunk(fileId, localOffset),
+ }
+
+ }
+ limitedUploadProcessor.Execute(request)
+ }
+
+ go func() {
+ wg.Wait()
+ close(resultsChan)
+ }()
+
+ waitForAllData.Wait()
+
+ if err == io.EOF {
+ err = nil
+ }
+
+ return fileChunks, md5Hash, readOffset, err, nil
+}
+
+type ChunkCreationResult struct {
+ chunk *filer_pb.FileChunk
+ err error
+}
+
+func (fs *FilerServer) doCreateChunk(w http.ResponseWriter, r *http.Request, so *operation.StorageOption, dataReader *util.BytesReader, fileName string, contentType string) (string, *operation.UploadResult, error) {
+ // retry to assign a different file id
+ var fileId, urlLocation string
+ var auth security.EncodedJwt
+ var assignErr, uploadErr error
+ var uploadResult *operation.UploadResult
+ for i := 0; i < 3; i++ {
+ // assign one file id for one chunk
+ fileId, urlLocation, auth, assignErr = fs.assignNewFileInfo(so)
+ if assignErr != nil {
+ return "", nil, assignErr
+ }
+
+ // upload the chunk to the volume server
+ uploadResult, uploadErr, _ = fs.doUpload(urlLocation, w, r, dataReader, fileName, contentType, nil, auth)
+ if uploadErr != nil {
+ time.Sleep(251 * time.Millisecond)
+ continue
+ }
+ break
+ }
+ return fileId, uploadResult, uploadErr
+}
+
+func (fs *FilerServer) doUpload(urlLocation string, w http.ResponseWriter, r *http.Request, limitedReader io.Reader, fileName string, contentType string, pairMap map[string]string, auth security.EncodedJwt) (*operation.UploadResult, error, []byte) {
+
+ stats.FilerRequestCounter.WithLabelValues("chunkUpload").Inc()
+ start := time.Now()
+ defer func() {
+ stats.FilerRequestHistogram.WithLabelValues("chunkUpload").Observe(time.Since(start).Seconds())
+ }()
+
+ uploadResult, err, data := operation.Upload(urlLocation, fileName, fs.option.Cipher, limitedReader, false, contentType, pairMap, auth)
+ if uploadResult != nil && uploadResult.RetryCount > 0 {
+ stats.FilerRequestCounter.WithLabelValues("chunkUploadRetry").Add(float64(uploadResult.RetryCount))
+ }
+ return uploadResult, err, data
+}
diff --git a/weed/server/master_grpc_server.go b/weed/server/master_grpc_server.go
index 0f0b7f101..3e6d9bb9e 100644
--- a/weed/server/master_grpc_server.go
+++ b/weed/server/master_grpc_server.go
@@ -80,10 +80,14 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
dn.AdjustMaxVolumeCounts(heartbeat.MaxVolumeCounts)
glog.V(4).Infof("master received heartbeat %s", heartbeat.String())
+ var dataCenter string
+ if dc := dn.GetDataCenter(); dc != nil {
+ dataCenter = string(dc.Id())
+ }
message := &master_pb.VolumeLocation{
Url: dn.Url(),
PublicUrl: dn.PublicUrl,
- DataCenter: string(dn.GetDataCenter().Id()),
+ DataCenter: dataCenter,
}
if len(heartbeat.NewVolumes) > 0 || len(heartbeat.DeletedVolumes) > 0 {
// process delta volume ids if exists for fast volume id updates
diff --git a/weed/server/master_grpc_server_volume.go b/weed/server/master_grpc_server_volume.go
index 29aff5c0b..156afd4a1 100644
--- a/weed/server/master_grpc_server_volume.go
+++ b/weed/server/master_grpc_server_volume.go
@@ -77,7 +77,7 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
if !ms.Topo.HasWritableVolume(option) {
if ms.Topo.AvailableSpaceFor(option) <= 0 {
- return nil, fmt.Errorf("no free volumes left for "+option.String())
+ return nil, fmt.Errorf("no free volumes left for " + option.String())
}
ms.vgLock.Lock()
if !ms.Topo.HasWritableVolume(option) {
@@ -122,11 +122,8 @@ func (ms *MasterServer) Statistics(ctx context.Context, req *master_pb.Statistic
volumeLayout := ms.Topo.GetVolumeLayout(req.Collection, replicaPlacement, ttl, types.ToDiskType(req.DiskType))
stats := volumeLayout.Stats()
-
- totalSize := ms.Topo.GetDiskUsages().GetMaxVolumeCount() * int64(ms.option.VolumeSizeLimitMB) * 1024 * 1024
-
resp := &master_pb.StatisticsResponse{
- TotalSize: uint64(totalSize),
+ TotalSize: stats.TotalSize,
UsedSize: stats.UsedSize,
FileCount: stats.FileCount,
}
diff --git a/weed/server/master_server.go b/weed/server/master_server.go
index 9404081b4..e2b2df18d 100644
--- a/weed/server/master_server.go
+++ b/weed/server/master_server.go
@@ -277,6 +277,13 @@ func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer
glog.Error(err)
seq = nil
}
+ case "snowflake":
+ var err error
+ seq, err = sequence.NewSnowflakeSequencer(fmt.Sprintf("%s:%d", option.Host, option.Port))
+ if err != nil {
+ glog.Error(err)
+ seq = nil
+ }
default:
seq = sequence.NewMemorySequencer()
}
diff --git a/weed/server/volume_grpc_read_write.go b/weed/server/volume_grpc_read_write.go
new file mode 100644
index 000000000..988e9e145
--- /dev/null
+++ b/weed/server/volume_grpc_read_write.go
@@ -0,0 +1,38 @@
+package weed_server
+
+import (
+ "context"
+ "fmt"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+func (vs *VolumeServer) ReadNeedleBlob(ctx context.Context, req *volume_server_pb.ReadNeedleBlobRequest) (resp *volume_server_pb.ReadNeedleBlobResponse, err error) {
+ resp = &volume_server_pb.ReadNeedleBlobResponse{}
+ v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
+ if v == nil {
+ return nil, fmt.Errorf("not found volume id %d", req.VolumeId)
+ }
+
+ resp.NeedleBlob, err = v.ReadNeedleBlob(req.Offset, types.Size(req.Size))
+ if err != nil {
+ return nil, fmt.Errorf("read needle blob offset %d size %d: %v", req.Offset, req.Size, err)
+ }
+
+ return resp, nil
+}
+
+func (vs *VolumeServer) WriteNeedleBlob(ctx context.Context, req *volume_server_pb.WriteNeedleBlobRequest) (resp *volume_server_pb.WriteNeedleBlobResponse, err error) {
+ resp = &volume_server_pb.WriteNeedleBlobResponse{}
+ v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
+ if v == nil {
+ return nil, fmt.Errorf("not found volume id %d", req.VolumeId)
+ }
+
+ if err = v.WriteNeedleBlob(types.NeedleId(req.NeedleId), req.NeedleBlob, 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/server/volume_server.go b/weed/server/volume_server.go
index e496b1ce2..e11d607a4 100644
--- a/weed/server/volume_server.go
+++ b/weed/server/volume_server.go
@@ -4,6 +4,7 @@ import (
"fmt"
"github.com/chrislusf/seaweedfs/weed/storage/types"
"net/http"
+ "sync"
"google.golang.org/grpc"
@@ -34,6 +35,10 @@ type VolumeServer struct {
fileSizeLimitBytes int64
isHeartbeating bool
stopChan chan bool
+
+ inFlightDataSize int64
+ inFlightDataLimitCond *sync.Cond
+ concurrentUploadLimit int64
}
func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
@@ -48,6 +53,7 @@ func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
readRedirect bool,
compactionMBPerSecond int,
fileSizeLimitMB int,
+ concurrentUploadLimit int64,
) *VolumeServer {
v := util.GetViper()
@@ -72,6 +78,8 @@ func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
fileSizeLimitBytes: int64(fileSizeLimitMB) * 1024 * 1024,
isHeartbeating: true,
stopChan: make(chan bool),
+ inFlightDataLimitCond: sync.NewCond(new(sync.Mutex)),
+ concurrentUploadLimit: concurrentUploadLimit,
}
vs.SeedMasterNodes = masterNodes
diff --git a/weed/server/volume_server_handlers.go b/weed/server/volume_server_handlers.go
index 7852c950a..4527add44 100644
--- a/weed/server/volume_server_handlers.go
+++ b/weed/server/volume_server_handlers.go
@@ -2,7 +2,9 @@ package weed_server
import (
"net/http"
+ "strconv"
"strings"
+ "sync/atomic"
"github.com/chrislusf/seaweedfs/weed/util"
@@ -40,8 +42,24 @@ func (vs *VolumeServer) privateStoreHandler(w http.ResponseWriter, r *http.Reque
stats.DeleteRequest()
vs.guard.WhiteList(vs.DeleteHandler)(w, r)
case "PUT", "POST":
+
+ // wait until in flight data is less than the limit
+ contentLength := getContentLength(r)
+ vs.inFlightDataLimitCond.L.Lock()
+ for atomic.LoadInt64(&vs.inFlightDataSize) > vs.concurrentUploadLimit {
+ vs.inFlightDataLimitCond.Wait()
+ }
+ atomic.AddInt64(&vs.inFlightDataSize, contentLength)
+ vs.inFlightDataLimitCond.L.Unlock()
+ defer func() {
+ atomic.AddInt64(&vs.inFlightDataSize, -contentLength)
+ vs.inFlightDataLimitCond.Signal()
+ }()
+
+ // processs uploads
stats.WriteRequest()
vs.guard.WhiteList(vs.PostHandler)(w, r)
+
case "OPTIONS":
stats.ReadRequest()
w.Header().Add("Access-Control-Allow-Methods", "PUT, POST, GET, DELETE, OPTIONS")
@@ -49,6 +67,18 @@ func (vs *VolumeServer) privateStoreHandler(w http.ResponseWriter, r *http.Reque
}
}
+func getContentLength(r *http.Request) int64 {
+ contentLength := r.Header.Get("Content-Length")
+ if contentLength != "" {
+ length, err := strconv.ParseInt(contentLength, 10, 64)
+ if err != nil {
+ return 0
+ }
+ return length
+ }
+ return 0
+}
+
func (vs *VolumeServer) publicReadOnlyHandler(w http.ResponseWriter, r *http.Request) {
w.Header().Set("Server", "SeaweedFS Volume "+util.VERSION)
if r.Header.Get("Origin") != "" {
diff --git a/weed/server/volume_server_handlers_read.go b/weed/server/volume_server_handlers_read.go
index 2db46ac9b..3e977cfd4 100644
--- a/weed/server/volume_server_handlers_read.go
+++ b/weed/server/volume_server_handlers_read.go
@@ -27,7 +27,7 @@ var fileNameEscaper = strings.NewReplacer(`\`, `\\`, `"`, `\"`)
func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request) {
- // println(r.Method + " " + r.URL.Path)
+ glog.V(9).Info(r.Method + " " + r.URL.Path + " " + r.Header.Get("Range"))
stats.VolumeServerRequestCounter.WithLabelValues("get").Inc()
start := time.Now()
@@ -261,13 +261,10 @@ func writeResponseContent(filename, mimeType string, rs io.ReadSeeker, w http.Re
return nil
}
- processRangeRequest(r, w, totalSize, mimeType, func(writer io.Writer, offset int64, size int64, httpStatusCode int) error {
+ processRangeRequest(r, w, totalSize, mimeType, func(writer io.Writer, offset int64, size int64) error {
if _, e = rs.Seek(offset, 0); e != nil {
return e
}
- if httpStatusCode != 0 {
- w.WriteHeader(httpStatusCode)
- }
_, e = io.CopyN(writer, rs, size)
return e
})
diff --git a/weed/server/volume_server_tcp_handlers_write.go b/weed/server/volume_server_tcp_handlers_write.go
new file mode 100644
index 000000000..a009611da
--- /dev/null
+++ b/weed/server/volume_server_tcp_handlers_write.go
@@ -0,0 +1,137 @@
+package weed_server
+
+import (
+ "bufio"
+ "fmt"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/util"
+ "io"
+ "net"
+ "strings"
+)
+
+func (vs *VolumeServer) HandleTcpConnection(c net.Conn) {
+ defer c.Close()
+
+ glog.V(0).Infof("Serving writes from %s", c.RemoteAddr().String())
+
+ bufReader := bufio.NewReaderSize(c, 1024*1024)
+ bufWriter := bufio.NewWriterSize(c, 1024*1024)
+
+ for {
+ cmd, err := bufReader.ReadString('\n')
+ if err != nil {
+ if err != io.EOF {
+ glog.Errorf("read command from %s: %v", c.RemoteAddr().String(), err)
+ }
+ return
+ }
+ cmd = cmd[:len(cmd)-1]
+ switch cmd[0] {
+ case '+':
+ fileId := cmd[1:]
+ err = vs.handleTcpPut(fileId, bufReader)
+ if err == nil {
+ bufWriter.Write([]byte("+OK\n"))
+ } else {
+ bufWriter.Write([]byte("-ERR " + string(err.Error()) + "\n"))
+ }
+ case '-':
+ fileId := cmd[1:]
+ err = vs.handleTcpDelete(fileId)
+ if err == nil {
+ bufWriter.Write([]byte("+OK\n"))
+ } else {
+ bufWriter.Write([]byte("-ERR " + string(err.Error()) + "\n"))
+ }
+ case '?':
+ fileId := cmd[1:]
+ err = vs.handleTcpGet(fileId, bufWriter)
+ case '!':
+ bufWriter.Flush()
+ }
+
+ }
+
+}
+
+func (vs *VolumeServer) handleTcpGet(fileId string, writer *bufio.Writer) (err error) {
+
+ volumeId, n, err2 := vs.parseFileId(fileId)
+ if err2 != nil {
+ return err2
+ }
+
+ volume := vs.store.GetVolume(volumeId)
+ if volume == nil {
+ return fmt.Errorf("volume %d not found", volumeId)
+ }
+
+ err = volume.StreamRead(n, writer)
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+func (vs *VolumeServer) handleTcpPut(fileId string, bufReader *bufio.Reader) (err error) {
+
+ volumeId, n, err2 := vs.parseFileId(fileId)
+ if err2 != nil {
+ return err2
+ }
+
+ volume := vs.store.GetVolume(volumeId)
+ if volume == nil {
+ return fmt.Errorf("volume %d not found", volumeId)
+ }
+
+ sizeBuf := make([]byte, 4)
+ if _, err = bufReader.Read(sizeBuf); err != nil {
+ return err
+ }
+ dataSize := util.BytesToUint32(sizeBuf)
+
+ err = volume.StreamWrite(n, bufReader, dataSize)
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+func (vs *VolumeServer) handleTcpDelete(fileId string) (err error) {
+
+ volumeId, n, err2 := vs.parseFileId(fileId)
+ if err2 != nil {
+ return err2
+ }
+
+ _, err = vs.store.DeleteVolumeNeedle(volumeId, n)
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+func (vs *VolumeServer) parseFileId(fileId string) (needle.VolumeId, *needle.Needle, error) {
+
+ commaIndex := strings.LastIndex(fileId, ",")
+ if commaIndex <= 0 {
+ return 0, nil, fmt.Errorf("unknown fileId %s", fileId)
+ }
+
+ vid, fid := fileId[0:commaIndex], fileId[commaIndex+1:]
+
+ volumeId, ve := needle.NewVolumeId(vid)
+ if ve != nil {
+ return 0, nil, fmt.Errorf("unknown volume id in fileId %s", fileId)
+ }
+
+ n := new(needle.Needle)
+ n.ParsePath(fid)
+ return volumeId, n, nil
+}
diff --git a/weed/shell/command_ec_encode.go b/weed/shell/command_ec_encode.go
index edacf22c6..634cb11e2 100644
--- a/weed/shell/command_ec_encode.go
+++ b/weed/shell/command_ec_encode.go
@@ -274,7 +274,7 @@ func collectVolumeIdsForEcEncode(commandEnv *CommandEnv, selectedCollection stri
quietSeconds := int64(quietPeriod / time.Second)
nowUnixSeconds := time.Now().Unix()
- fmt.Printf("ec encode volumes quiet for: %d seconds\n", quietSeconds)
+ fmt.Printf("collect volumes quiet for: %d seconds\n", quietSeconds)
vidMap := make(map[uint32]bool)
eachDataNode(topologyInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
diff --git a/weed/shell/command_fs_cat.go b/weed/shell/command_fs_cat.go
index 3c5e13663..df43d93dc 100644
--- a/weed/shell/command_fs_cat.go
+++ b/weed/shell/command_fs_cat.go
@@ -52,7 +52,7 @@ func (c *commandFsCat) Do(args []string, commandEnv *CommandEnv, writer io.Write
return err
}
- return filer.StreamContent(commandEnv.MasterClient, writer, respLookupEntry.Entry.Chunks, 0, math.MaxInt64)
+ return filer.StreamContent(commandEnv.MasterClient, writer, respLookupEntry.Entry.Chunks, 0, math.MaxInt64, false)
})
diff --git a/weed/shell/command_fs_mv.go b/weed/shell/command_fs_mv.go
index 0a7eed02d..2448c8f61 100644
--- a/weed/shell/command_fs_mv.go
+++ b/weed/shell/command_fs_mv.go
@@ -36,6 +36,10 @@ func (c *commandFsMv) Help() string {
func (c *commandFsMv) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+ if len(args) != 2 {
+ return fmt.Errorf("need to have 2 arguments")
+ }
+
sourcePath, err := commandEnv.parseUrl(args[0])
if err != nil {
return err
diff --git a/weed/shell/command_s3_clean_uploads.go b/weed/shell/command_s3_clean_uploads.go
new file mode 100644
index 000000000..5f674d7b6
--- /dev/null
+++ b/weed/shell/command_s3_clean_uploads.go
@@ -0,0 +1,92 @@
+package shell
+
+import (
+ "flag"
+ "fmt"
+ "github.com/chrislusf/seaweedfs/weed/util"
+ "io"
+ "math"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+)
+
+func init() {
+ Commands = append(Commands, &commandS3CleanUploads{})
+}
+
+type commandS3CleanUploads struct {
+}
+
+func (c *commandS3CleanUploads) Name() string {
+ return "s3.clean.uploads"
+}
+
+func (c *commandS3CleanUploads) Help() string {
+ return `clean up stale multipart uploads
+
+ Example:
+ s3.clean.uploads -replication 001
+
+`
+}
+
+func (c *commandS3CleanUploads) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ bucketCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ uploadedTimeAgo := bucketCommand.Duration("timeAgo", 24*time.Hour, "created time before now. \"1.5h\" or \"2h45m\". Valid time units are \"m\", \"h\"")
+ if err = bucketCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ var filerBucketsPath string
+ filerBucketsPath, err = readFilerBucketsPath(commandEnv)
+ if err != nil {
+ return fmt.Errorf("read buckets: %v", err)
+ }
+
+ var buckets []string
+ err = filer_pb.List(commandEnv, filerBucketsPath, "", func(entry *filer_pb.Entry, isLast bool) error {
+ buckets = append(buckets, entry.Name)
+ return nil
+ }, "", false, math.MaxUint32)
+ if err != nil {
+ return fmt.Errorf("list buckets under %v: %v", filerBucketsPath, err)
+ }
+
+ for _, bucket := range buckets {
+ c.cleanupUploads(commandEnv, writer, filerBucketsPath, bucket, *uploadedTimeAgo)
+ }
+
+ return err
+
+}
+
+func (c *commandS3CleanUploads) cleanupUploads(commandEnv *CommandEnv, writer io.Writer, filerBucketsPath string, bucket string, timeAgo time.Duration) error {
+ uploadsDir := filerBucketsPath + "/" + bucket + "/.uploads"
+ var staleUploads []string
+ now := time.Now()
+ err := filer_pb.List(commandEnv, uploadsDir, "", func(entry *filer_pb.Entry, isLast bool) error {
+ ctime := time.Unix(entry.Attributes.Crtime, 0)
+ if ctime.Add(timeAgo).Before(now) {
+ staleUploads = append(staleUploads, entry.Name)
+ }
+ return nil
+ }, "", false, math.MaxUint32)
+ if err != nil {
+ return fmt.Errorf("list uploads under %v: %v", uploadsDir, err)
+ }
+
+ for _, staleUpload := range staleUploads {
+ deleteUrl := fmt.Sprintf("http://%s:%d%s/%s?recursive=true&ignoreRecursiveError=true", commandEnv.option.FilerHost, commandEnv.option.FilerPort, uploadsDir, staleUpload)
+ fmt.Fprintf(writer, "purge %s\n", deleteUrl)
+
+ err = util.Delete(deleteUrl, "")
+ if err != nil {
+ return fmt.Errorf("purge %s/%s: %v", uploadsDir, staleUpload, err)
+ }
+ }
+
+ return nil
+
+}
diff --git a/weed/shell/command_volume_balance.go b/weed/shell/command_volume_balance.go
index e0c41f310..ad7da0e44 100644
--- a/weed/shell/command_volume_balance.go
+++ b/weed/shell/command_volume_balance.go
@@ -393,9 +393,8 @@ func adjustAfterMove(v *master_pb.VolumeInformationMessage, volumeReplicas map[u
if replica.location.dataNode.Id == fullNode.info.Id &&
replica.location.rack == fullNode.rack &&
replica.location.dc == fullNode.dc {
- replica.location.dc = emptyNode.dc
- replica.location.rack = emptyNode.rack
- replica.location.dataNode = emptyNode.info
+ loc := newLocation(emptyNode.dc, emptyNode.rack, emptyNode.info)
+ replica.location = &loc
return
}
}
diff --git a/weed/shell/command_volume_balance_test.go b/weed/shell/command_volume_balance_test.go
index 696bc7fac..b77811f51 100644
--- a/weed/shell/command_volume_balance_test.go
+++ b/weed/shell/command_volume_balance_test.go
@@ -169,3 +169,15 @@ func TestIsGoodMove(t *testing.T) {
}
}
+
+func TestBalance(t *testing.T) {
+ topologyInfo := parseOutput(topoData)
+ volumeServers := collectVolumeServersByDc(topologyInfo, "")
+ volumeReplicas, _ := collectVolumeReplicaLocations(topologyInfo)
+ diskTypes := collectVolumeDiskTypes(topologyInfo)
+
+ if err := balanceVolumeServers(nil, diskTypes, volumeReplicas, volumeServers, 30*1024*1024*1024, "ALL_COLLECTIONS", false); err != nil {
+ t.Errorf("balance: %v", err)
+ }
+
+}
diff --git a/weed/shell/command_volume_check_disk.go b/weed/shell/command_volume_check_disk.go
new file mode 100644
index 000000000..5a0d46869
--- /dev/null
+++ b/weed/shell/command_volume_check_disk.go
@@ -0,0 +1,257 @@
+package shell
+
+import (
+ "bytes"
+ "context"
+ "flag"
+ "fmt"
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ "io"
+ "math"
+ "sort"
+)
+
+func init() {
+ Commands = append(Commands, &commandVolumeCheckDisk{})
+}
+
+type commandVolumeCheckDisk struct {
+ env *CommandEnv
+}
+
+func (c *commandVolumeCheckDisk) Name() string {
+ return "volume.check.disk"
+}
+
+func (c *commandVolumeCheckDisk) Help() string {
+ return `check all replicated volumes to find and fix inconsistencies
+
+ How it works:
+
+ find all volumes that are replicated
+ for each volume id, if there are more than 2 replicas, find one pair with the largest 2 in file count.
+ for the pair volume A and B
+ append entries in A and not in B to B
+ append entries in B and not in A to A
+
+`
+}
+
+func (c *commandVolumeCheckDisk) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ if err = commandEnv.confirmIsLocked(); err != nil {
+ return
+ }
+
+ fsckCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ slowMode := fsckCommand.Bool("slow", false, "slow mode checks all replicas even file counts are the same")
+ verbose := fsckCommand.Bool("v", false, "verbose mode")
+ applyChanges := fsckCommand.Bool("force", false, "apply the fix")
+ nonRepairThreshold := fsckCommand.Float64("nonRepairThreshold", 0.3, "repair when missing keys is not more than this limit")
+ if err = fsckCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ c.env = commandEnv
+
+ // collect topology information
+ topologyInfo, _, err := collectTopologyInfo(commandEnv)
+ if err != nil {
+ return err
+ }
+ volumeReplicas, _ := collectVolumeReplicaLocations(topologyInfo)
+
+ // pick 1 pairs of volume replica
+ fileCount := func(replica *VolumeReplica) uint64 {
+ return replica.info.FileCount - replica.info.DeleteCount
+ }
+ aDB, bDB := needle_map.NewMemDb(), needle_map.NewMemDb()
+ defer aDB.Close()
+ defer bDB.Close()
+
+ for _, replicas := range volumeReplicas {
+ sort.Slice(replicas, func(i, j int) bool {
+ return fileCount(replicas[i]) > fileCount(replicas[j])
+ })
+ for len(replicas) >= 2 {
+ a, b := replicas[0], replicas[1]
+ if !*slowMode {
+ if fileCount(a) == fileCount(b) {
+ replicas = replicas[1:]
+ continue
+ }
+ }
+ if a.info.ReadOnly || b.info.ReadOnly {
+ fmt.Fprintf(writer, "skipping readonly volume %d on %s and %s\n", a.info.Id, a.location.dataNode.Id, b.location.dataNode.Id)
+ continue
+ }
+
+ // reset index db
+ aDB.Close()
+ bDB.Close()
+ aDB, bDB = needle_map.NewMemDb(), needle_map.NewMemDb()
+
+ // read index db
+ if err := c.readIndexDatabase(aDB, a.info.Collection, a.info.Id, a.location.dataNode.Id, *verbose, writer); err != nil {
+ return err
+ }
+ if err := c.readIndexDatabase(bDB, b.info.Collection, b.info.Id, b.location.dataNode.Id, *verbose, writer); err != nil {
+ return err
+ }
+
+ // find and make up the differnces
+ if err := c.doVolumeCheckDisk(aDB, bDB, a, b, *verbose, writer, *applyChanges, *nonRepairThreshold); err != nil {
+ return err
+ }
+ if err := c.doVolumeCheckDisk(bDB, aDB, b, a, *verbose, writer, *applyChanges, *nonRepairThreshold); err != nil {
+ return err
+ }
+ replicas = replicas[1:]
+ }
+ }
+
+ return nil
+}
+
+func (c *commandVolumeCheckDisk) doVolumeCheckDisk(subtrahend, minuend *needle_map.MemDb, source, target *VolumeReplica, verbose bool, writer io.Writer, applyChanges bool, nonRepairThreshold float64) error {
+
+ // find missing keys
+ // hash join, can be more efficient
+ var missingNeedles []needle_map.NeedleValue
+ var counter int
+ subtrahend.AscendingVisit(func(value needle_map.NeedleValue) error {
+ counter++
+ if _, found := minuend.Get(value.Key); !found {
+ missingNeedles = append(missingNeedles, value)
+ }
+ return nil
+ })
+
+ fmt.Fprintf(writer, "volume %d %s has %d entries, %s missed %d entries\n", source.info.Id, source.location.dataNode.Id, counter, target.location.dataNode.Id, len(missingNeedles))
+
+ if counter == 0 || len(missingNeedles) == 0 {
+ return nil
+ }
+
+ missingNeedlesFraction := float64(len(missingNeedles)) / float64(counter)
+ if missingNeedlesFraction > nonRepairThreshold {
+ return fmt.Errorf(
+ "failed to start repair volume %d, percentage of missing keys is greater than the threshold: %.2f > %.2f",
+ source.info.Id, missingNeedlesFraction, nonRepairThreshold)
+ }
+
+ for _, needleValue := range missingNeedles {
+
+ needleBlob, err := c.readSourceNeedleBlob(source.location.dataNode.Id, source.info.Id, needleValue)
+ if err != nil {
+ return err
+ }
+
+ if !applyChanges {
+ continue
+ }
+
+ if verbose {
+ fmt.Fprintf(writer, "read %d,%x %s => %s \n", source.info.Id, needleValue.Key, source.location.dataNode.Id, target.location.dataNode.Id)
+ }
+
+ if err := c.writeNeedleBlobToTarget(target.location.dataNode.Id, source.info.Id, needleValue, needleBlob); err != nil {
+ return err
+ }
+
+ }
+
+ return nil
+}
+
+func (c *commandVolumeCheckDisk) readSourceNeedleBlob(sourceVolumeServer string, volumeId uint32, needleValue needle_map.NeedleValue) (needleBlob []byte, err error) {
+
+ err = operation.WithVolumeServerClient(sourceVolumeServer, c.env.option.GrpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
+ resp, err := client.ReadNeedleBlob(context.Background(), &volume_server_pb.ReadNeedleBlobRequest{
+ VolumeId: volumeId,
+ NeedleId: uint64(needleValue.Key),
+ Offset: needleValue.Offset.ToActualOffset(),
+ Size: int32(needleValue.Size),
+ })
+ if err != nil {
+ return err
+ }
+ needleBlob = resp.NeedleBlob
+ return nil
+ })
+ return
+}
+
+func (c *commandVolumeCheckDisk) writeNeedleBlobToTarget(targetVolumeServer string, volumeId uint32, needleValue needle_map.NeedleValue, needleBlob []byte) error {
+
+ return operation.WithVolumeServerClient(targetVolumeServer, c.env.option.GrpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
+ _, err := client.WriteNeedleBlob(context.Background(), &volume_server_pb.WriteNeedleBlobRequest{
+ VolumeId: volumeId,
+ NeedleId: uint64(needleValue.Key),
+ Size: int32(needleValue.Size),
+ NeedleBlob: needleBlob,
+ })
+ return err
+ })
+
+}
+
+func (c *commandVolumeCheckDisk) readIndexDatabase(db *needle_map.MemDb, collection string, volumeId uint32, volumeServer string, verbose bool, writer io.Writer) error {
+
+ var buf bytes.Buffer
+ if err := c.copyVolumeIndexFile(collection, volumeId, volumeServer, &buf, verbose, writer); err != nil {
+ return err
+ }
+
+ if verbose {
+ fmt.Fprintf(writer, "load collection %s volume %d index size %d from %s ...\n", collection, volumeId, buf.Len(), volumeServer)
+ }
+
+ return db.LoadFromReaderAt(bytes.NewReader(buf.Bytes()))
+
+}
+
+func (c *commandVolumeCheckDisk) copyVolumeIndexFile(collection string, volumeId uint32, volumeServer string, buf *bytes.Buffer, verbose bool, writer io.Writer) error {
+
+ return operation.WithVolumeServerClient(volumeServer, c.env.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+
+ ext := ".idx"
+
+ copyFileClient, err := volumeServerClient.CopyFile(context.Background(), &volume_server_pb.CopyFileRequest{
+ VolumeId: volumeId,
+ Ext: ".idx",
+ CompactionRevision: math.MaxUint32,
+ StopOffset: math.MaxInt64,
+ Collection: collection,
+ IsEcVolume: false,
+ IgnoreSourceFileNotFound: false,
+ })
+ if err != nil {
+ return fmt.Errorf("failed to start copying volume %d%s: %v", volumeId, ext, err)
+ }
+
+ err = writeToBuffer(copyFileClient, buf)
+ if err != nil {
+ return fmt.Errorf("failed to copy %d%s from %s: %v", volumeId, ext, volumeServer, err)
+ }
+
+ return nil
+
+ })
+}
+
+func writeToBuffer(client volume_server_pb.VolumeServer_CopyFileClient, buf *bytes.Buffer) error {
+ for {
+ resp, receiveErr := client.Recv()
+ if receiveErr == io.EOF {
+ break
+ }
+ if receiveErr != nil {
+ return fmt.Errorf("receiving: %v", receiveErr)
+ }
+ buf.Write(resp.FileContent)
+ }
+ return nil
+}
diff --git a/weed/shell/command_volume_list_test.go b/weed/shell/command_volume_list_test.go
new file mode 100644
index 000000000..72c76f242
--- /dev/null
+++ b/weed/shell/command_volume_list_test.go
@@ -0,0 +1,893 @@
+package shell
+
+import (
+ "github.com/chrislusf/seaweedfs/weed/storage/types"
+ "github.com/golang/protobuf/proto"
+ "github.com/stretchr/testify/assert"
+ "strconv"
+ "strings"
+ "testing"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+)
+
+func TestParsing(t *testing.T) {
+ topo := parseOutput(topoData)
+
+ assert.Equal(t, 5, len(topo.DataCenterInfos))
+
+}
+
+func parseOutput(output string) *master_pb.TopologyInfo {
+ lines := strings.Split(output, "\n")
+ var topo *master_pb.TopologyInfo
+ var dc *master_pb.DataCenterInfo
+ var rack *master_pb.RackInfo
+ var dn *master_pb.DataNodeInfo
+ var disk *master_pb.DiskInfo
+ for _, line := range lines {
+ line = strings.TrimSpace(line)
+ parts := strings.Split(line, " ")
+ switch parts[0] {
+ case "Topology":
+ if topo == nil {
+ topo = &master_pb.TopologyInfo{}
+ }
+ case "DataCenter":
+ if dc == nil {
+ dc = &master_pb.DataCenterInfo{
+ Id: parts[1],
+ }
+ topo.DataCenterInfos = append(topo.DataCenterInfos, dc)
+ } else {
+ dc = nil
+ }
+ case "Rack":
+ if rack == nil {
+ rack = &master_pb.RackInfo{
+ Id: parts[1],
+ }
+ dc.RackInfos = append(dc.RackInfos, rack)
+ } else {
+ rack = nil
+ }
+ case "DataNode":
+ if dn == nil {
+ dn = &master_pb.DataNodeInfo{
+ Id: parts[1],
+ DiskInfos: make(map[string]*master_pb.DiskInfo),
+ }
+ rack.DataNodeInfos = append(rack.DataNodeInfos, dn)
+ } else {
+ dn = nil
+ }
+ case "Disk":
+ if disk == nil {
+ diskType := parts[1][:strings.Index(parts[1], "(")]
+ maxVolumeCountStr := parts[1][strings.Index(parts[1], "/")+1:]
+ maxVolumeCount, _ := strconv.Atoi(maxVolumeCountStr)
+ disk = &master_pb.DiskInfo{
+ Type: diskType,
+ MaxVolumeCount: uint64(maxVolumeCount),
+ }
+ dn.DiskInfos[types.ToDiskType(diskType).String()] = disk
+ } else {
+ disk = nil
+ }
+ case "volume":
+ volumeLine := line[len("volume "):]
+ volume := &master_pb.VolumeInformationMessage{}
+ proto.UnmarshalText(volumeLine, volume)
+ disk.VolumeInfos = append(disk.VolumeInfos, volume)
+ }
+ }
+
+ return topo
+}
+
+const topoData = `
+Topology volumeSizeLimit:1024 MB hdd(volume:760/7280 active:760 free:6520 remote:0)
+ DataCenter dc1 hdd(volume:0/0 active:0 free:0 remote:0)
+ Rack DefaultRack hdd(volume:0/0 active:0 free:0 remote:0)
+ Rack DefaultRack total size:0 file_count:0
+ DataCenter dc1 total size:0 file_count:0
+ DataCenter dc2 hdd(volume:86/430 active:86 free:344 remote:0)
+ Rack rack1 hdd(volume:50/240 active:50 free:190 remote:0)
+ DataNode 192.168.1.4:8080 hdd(volume:50/240 active:50 free:190 remote:0)
+ Disk hdd(volume:50/240 active:50 free:190 remote:0)
+ volume id:15 size:1115965064 collection:"collection0" file_count:83 replica_placement:100 version:3 modified_at_second:1609923671
+ volume id:21 size:1097631536 collection:"collection0" file_count:82 delete_count:7 deleted_byte_count:68975485 replica_placement:100 version:3 modified_at_second:1609929578
+ volume id:22 size:1086828272 collection:"collection0" file_count:75 replica_placement:100 version:3 modified_at_second:1609930001
+ volume id:23 size:1076380216 collection:"collection0" file_count:68 replica_placement:100 version:3 modified_at_second:1609930434
+ volume id:24 size:1074139776 collection:"collection0" file_count:90 replica_placement:100 version:3 modified_at_second:1609930909
+ volume id:25 size:690757512 collection:"collection0" file_count:38 replica_placement:100 version:3 modified_at_second:1611144216
+ volume id:27 size:298886792 file_count:1608 replica_placement:100 version:3 modified_at_second:1615632482
+ volume id:28 size:308919192 file_count:1591 delete_count:1 deleted_byte_count:125280 replica_placement:100 version:3 modified_at_second:1615631762
+ volume id:29 size:281582680 file_count:1537 replica_placement:100 version:3 modified_at_second:1615629422
+ volume id:30 size:289466144 file_count:1566 delete_count:1 deleted_byte_count:124972 replica_placement:100 version:3 modified_at_second:1615632422
+ volume id:31 size:273363256 file_count:1498 replica_placement:100 version:3 modified_at_second:1615631642
+ volume id:33 size:1130226400 collection:"collection1" file_count:7322 delete_count:172 deleted_byte_count:45199399 replica_placement:100 version:3 modified_at_second:1615618789
+ volume id:38 size:1075545744 collection:"collection1" file_count:13324 delete_count:100 deleted_byte_count:25223906 replica_placement:100 version:3 modified_at_second:1615569830
+ volume id:51 size:1076796120 collection:"collection1" file_count:10550 delete_count:39 deleted_byte_count:12723654 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615547786
+ volume id:52 size:1083529728 collection:"collection1" file_count:10128 delete_count:32 deleted_byte_count:10608391 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615599195
+ volume id:54 size:1045022344 collection:"collection1" file_count:9408 delete_count:30 deleted_byte_count:15132106 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630812
+ volume id:63 size:956941112 collection:"collection1" file_count:8271 delete_count:32 deleted_byte_count:15876189 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632036
+ volume id:69 size:869213648 collection:"collection1" file_count:7293 delete_count:102 deleted_byte_count:30643207 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630534
+ volume id:74 size:957046128 collection:"collection1" file_count:6982 delete_count:258 deleted_byte_count:73054259 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631460
+ volume id:80 size:827912928 collection:"collection1" file_count:6914 delete_count:17 deleted_byte_count:5689635 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631157
+ volume id:84 size:873121856 collection:"collection1" file_count:8200 delete_count:13 deleted_byte_count:3131676 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631161
+ volume id:85 size:1023869320 collection:"collection1" file_count:7788 delete_count:234 deleted_byte_count:78037967 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631723
+ volume id:97 size:1053112992 collection:"collection1" file_count:6789 delete_count:50 deleted_byte_count:38894001 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631193
+ volume id:98 size:1077836440 collection:"collection1" file_count:7605 delete_count:202 deleted_byte_count:73180379 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615523691
+ volume id:105 size:1073996824 collection:"collection1" file_count:6872 delete_count:20 deleted_byte_count:14482293 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615499757
+ volume id:106 size:1075458664 collection:"collection1" file_count:7182 delete_count:307 deleted_byte_count:69349053 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615598137
+ volume id:112 size:1076392512 collection:"collection1" file_count:8291 delete_count:156 deleted_byte_count:74120183 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615569823
+ volume id:116 size:1074489504 collection:"collection1" file_count:9981 delete_count:174 deleted_byte_count:53998777 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615611565
+ volume id:119 size:1075940104 collection:"collection1" file_count:9003 delete_count:12 deleted_byte_count:9128155 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615573878
+ volume id:128 size:1074874632 collection:"collection1" file_count:9821 delete_count:148 deleted_byte_count:43633334 replica_placement:100 version:3 modified_at_second:1615602670
+ volume id:133 size:1075952760 collection:"collection1" file_count:9538 delete_count:74 deleted_byte_count:19558008 replica_placement:100 version:3 modified_at_second:1615584779
+ volume id:136 size:1074433552 collection:"collection1" file_count:9593 delete_count:72 deleted_byte_count:26912512 replica_placement:100 version:3 modified_at_second:1615376036
+ volume id:138 size:1074465744 collection:"collection1" file_count:10120 delete_count:55 deleted_byte_count:15875438 replica_placement:100 version:3 modified_at_second:1615572231
+ volume id:140 size:1076203744 collection:"collection1" file_count:11219 delete_count:57 deleted_byte_count:19864498 replica_placement:100 version:3 modified_at_second:1615571947
+ volume id:144 size:1074549720 collection:"collection1" file_count:8780 delete_count:50 deleted_byte_count:52475146 replica_placement:100 version:3 modified_at_second:1615573451
+ volume id:161 size:1077397192 collection:"collection1" file_count:9988 delete_count:28 deleted_byte_count:12509164 replica_placement:100 version:3 modified_at_second:1615631452
+ volume id:173 size:1074154704 collection:"collection1" file_count:30884 delete_count:34 deleted_byte_count:2578509 replica_placement:100 version:3 modified_at_second:1615591904
+ volume id:174 size:1073824232 collection:"collection1" file_count:30689 delete_count:36 deleted_byte_count:2160116 replica_placement:100 version:3 modified_at_second:1615598914
+ volume id:197 size:1075423240 collection:"collection1" file_count:16473 delete_count:15 deleted_byte_count:12552442 replica_placement:100 version:3 modified_at_second:1615485254
+ volume id:219 size:1092298904 collection:"collection1" file_count:3193 delete_count:17 deleted_byte_count:2047576 replica_placement:100 version:3 modified_at_second:1615579316
+ volume id:263 size:1077167352 collection:"collection2" file_count:20227 delete_count:4 deleted_byte_count:97887 replica_placement:100 version:3 modified_at_second:1614871567
+ volume id:272 size:1076146040 collection:"collection2" file_count:21034 delete_count:2 deleted_byte_count:216564 replica_placement:100 version:3 modified_at_second:1614884139
+ volume id:291 size:1076256760 collection:"collection2" file_count:28301 delete_count:5 deleted_byte_count:116027 replica_placement:100 version:3 modified_at_second:1614904924
+ volume id:299 size:1075147824 collection:"collection2" file_count:22927 delete_count:4 deleted_byte_count:345569 replica_placement:100 version:3 modified_at_second:1614918454
+ volume id:301 size:1074655600 collection:"collection2" file_count:22543 delete_count:6 deleted_byte_count:136968 replica_placement:100 version:3 modified_at_second:1614918378
+ volume id:302 size:1077559792 collection:"collection2" file_count:23124 delete_count:7 deleted_byte_count:293111 replica_placement:100 version:3 modified_at_second:1614925500
+ volume id:339 size:1078402392 collection:"collection2" file_count:22309 replica_placement:100 version:3 modified_at_second:1614969996
+ volume id:345 size:1074560760 collection:"collection2" file_count:22117 delete_count:2 deleted_byte_count:373286 replica_placement:100 version:3 modified_at_second:1614977458
+ volume id:355 size:1075239792 collection:"collection2" file_count:22244 delete_count:1 deleted_byte_count:23282 replica_placement:100 version:3 modified_at_second:1614992157
+ volume id:373 size:1080928000 collection:"collection2" file_count:22617 delete_count:4 deleted_byte_count:91849 replica_placement:100 version:3 modified_at_second:1615016877
+ Disk hdd total size:48630015544 file_count:537880 deleted_file:2580 deleted_bytes:929560253
+ DataNode 192.168.1.4:8080 total size:48630015544 file_count:537880 deleted_file:2580 deleted_bytes:929560253
+ Rack rack1 total size:48630015544 file_count:537880 deleted_file:2580 deleted_bytes:929560253
+ Rack rack2 hdd(volume:36/190 active:36 free:154 remote:0)
+ DataNode 192.168.1.2:8080 hdd(volume:36/190 active:36 free:154 remote:0)
+ Disk hdd(volume:36/190 active:36 free:154 remote:0)
+ volume id:2 size:289228560 file_count:1640 delete_count:4 deleted_byte_count:480564 replica_placement:100 version:3 compact_revision:6 modified_at_second:1615630622
+ volume id:3 size:308743136 file_count:1638 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632242
+ volume id:4 size:285986968 file_count:1641 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632302
+ volume id:6 size:302411024 file_count:1604 delete_count:2 deleted_byte_count:274587 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631402
+ volume id:7 size:1924728 collection:"collection4" file_count:15 replica_placement:100 version:3 modified_at_second:1609331040
+ volume id:9 size:77337416 collection:"collection3" file_count:58 replica_placement:100 version:3 ttl:772 modified_at_second:1615513762
+ volume id:10 size:1212784656 collection:"collection0" file_count:58 replica_placement:100 version:3 modified_at_second:1609814550
+ volume id:12 size:1110923848 collection:"collection0" file_count:45 replica_placement:100 version:3 modified_at_second:1609819732
+ volume id:13 size:1184910656 collection:"collection0" file_count:47 replica_placement:100 version:3 modified_at_second:1609827837
+ volume id:14 size:1107475720 collection:"collection0" file_count:80 delete_count:3 deleted_byte_count:6870 replica_placement:100 version:3 modified_at_second:1612956980
+ volume id:16 size:1113666104 collection:"collection0" file_count:73 delete_count:5 deleted_byte_count:6318 replica_placement:100 version:3 modified_at_second:1612957007
+ volume id:17 size:1095115800 collection:"collection0" file_count:83 delete_count:3 deleted_byte_count:7099 replica_placement:100 version:3 modified_at_second:1612957000
+ volume id:21 size:1097631664 collection:"collection0" file_count:82 delete_count:11 deleted_byte_count:68985100 replica_placement:100 version:3 modified_at_second:1612957007
+ volume id:56 size:1001897616 collection:"collection1" file_count:8762 delete_count:37 deleted_byte_count:65375405 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615632014
+ volume id:81 size:880693104 collection:"collection1" file_count:7481 delete_count:236 deleted_byte_count:80386421 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631396
+ volume id:104 size:1076383624 collection:"collection1" file_count:7663 delete_count:184 deleted_byte_count:100728071 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615602658
+ volume id:107 size:1073811840 collection:"collection1" file_count:7436 delete_count:168 deleted_byte_count:57747484 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615293569
+ volume id:113 size:1076709184 collection:"collection1" file_count:9355 delete_count:177 deleted_byte_count:59796765 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615569822
+ volume id:139 size:1074163936 collection:"collection1" file_count:9315 delete_count:42 deleted_byte_count:10630966 replica_placement:100 version:3 modified_at_second:1615571946
+ volume id:151 size:1098659752 collection:"collection1" file_count:10808 delete_count:24 deleted_byte_count:7088102 replica_placement:100 version:3 modified_at_second:1615586389
+ volume id:155 size:1075140688 collection:"collection1" file_count:10882 delete_count:32 deleted_byte_count:9076141 replica_placement:100 version:3 modified_at_second:1615606614
+ volume id:167 size:1073958176 collection:"collection1" file_count:25229 delete_count:48 deleted_byte_count:25871565 replica_placement:100 version:3 modified_at_second:1615602669
+ volume id:177 size:1074120216 collection:"collection1" file_count:22293 delete_count:16 deleted_byte_count:3803952 replica_placement:100 version:3 modified_at_second:1615516892
+ volume id:179 size:1074313920 collection:"collection1" file_count:21829 delete_count:24 deleted_byte_count:45552859 replica_placement:100 version:3 modified_at_second:1615580308
+ volume id:182 size:1076131280 collection:"collection1" file_count:31987 delete_count:21 deleted_byte_count:1452346 replica_placement:100 version:3 modified_at_second:1615568922
+ volume id:215 size:1068268216 collection:"collection1" file_count:2813 delete_count:10 deleted_byte_count:5676795 replica_placement:100 version:3 modified_at_second:1615586386
+ volume id:217 size:1075381872 collection:"collection1" file_count:3331 delete_count:14 deleted_byte_count:2009141 replica_placement:100 version:3 modified_at_second:1615401638
+ volume id:283 size:1080178944 collection:"collection2" file_count:19462 delete_count:7 deleted_byte_count:660407 replica_placement:100 version:3 modified_at_second:1614896626
+ volume id:303 size:1075944504 collection:"collection2" file_count:22541 delete_count:2 deleted_byte_count:13617 replica_placement:100 version:3 modified_at_second:1614925431
+ volume id:309 size:1075178624 collection:"collection2" file_count:22692 delete_count:3 deleted_byte_count:171124 replica_placement:100 version:3 modified_at_second:1614931409
+ volume id:323 size:1074608200 collection:"collection2" file_count:21605 delete_count:4 deleted_byte_count:172090 replica_placement:100 version:3 modified_at_second:1614950526
+ volume id:344 size:1075035448 collection:"collection2" file_count:21765 delete_count:1 deleted_byte_count:24623 replica_placement:100 version:3 modified_at_second:1614977465
+ volume id:347 size:1075145496 collection:"collection2" file_count:22178 delete_count:1 deleted_byte_count:79392 replica_placement:100 version:3 modified_at_second:1614984727
+ volume id:357 size:1074276208 collection:"collection2" file_count:23137 delete_count:4 deleted_byte_count:188487 replica_placement:100 version:3 modified_at_second:1614998792
+ volume id:380 size:1010760456 collection:"collection2" file_count:14921 delete_count:6 deleted_byte_count:65678 replica_placement:100 version:3 modified_at_second:1615632322
+ volume id:381 size:939292792 collection:"collection2" file_count:14619 delete_count:2 deleted_byte_count:5119 replica_placement:100 version:3 modified_at_second:1615632324
+ Disk hdd total size:33468194376 file_count:369168 deleted_file:1091 deleted_bytes:546337088
+ DataNode 192.168.1.2:8080 total size:33468194376 file_count:369168 deleted_file:1091 deleted_bytes:546337088
+ Rack rack2 total size:33468194376 file_count:369168 deleted_file:1091 deleted_bytes:546337088
+ DataCenter dc2 total size:82098209920 file_count:907048 deleted_file:3671 deleted_bytes:1475897341
+ DataCenter dc3 hdd(volume:108/1850 active:108 free:1742 remote:0)
+ Rack rack3 hdd(volume:108/1850 active:108 free:1742 remote:0)
+ DataNode 192.168.1.6:8080 hdd(volume:108/1850 active:108 free:1742 remote:0)
+ Disk hdd(volume:108/1850 active:108 free:1742 remote:0)
+ volume id:1 size:284685936 file_count:1557 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632062
+ volume id:32 size:281390512 file_count:1496 delete_count:6 deleted_byte_count:546403 replica_placement:100 version:3 modified_at_second:1615632362
+ volume id:47 size:444599784 collection:"collection1" file_count:709 delete_count:19 deleted_byte_count:11913451 replica_placement:100 version:3 modified_at_second:1615632397
+ volume id:49 size:1078775288 collection:"collection1" file_count:9636 delete_count:22 deleted_byte_count:5625976 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630446
+ volume id:68 size:898630584 collection:"collection1" file_count:6934 delete_count:95 deleted_byte_count:27460707 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632284
+ volume id:88 size:1073767976 collection:"collection1" file_count:14995 delete_count:206 deleted_byte_count:81222360 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615629897
+ volume id:202 size:1077533160 collection:"collection1" file_count:2847 delete_count:67 deleted_byte_count:65172985 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615588497
+ volume id:203 size:1027316272 collection:"collection1" file_count:3040 delete_count:11 deleted_byte_count:3993230 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631728
+ volume id:205 size:1078485304 collection:"collection1" file_count:2869 delete_count:43 deleted_byte_count:18290259 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615579314
+ volume id:206 size:1082045848 collection:"collection1" file_count:2979 delete_count:225 deleted_byte_count:88220074 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615564274
+ volume id:209 size:1074083592 collection:"collection1" file_count:3238 delete_count:4 deleted_byte_count:1494244 replica_placement:100 version:3 modified_at_second:1615419954
+ volume id:211 size:1080610712 collection:"collection1" file_count:3247 delete_count:7 deleted_byte_count:1891456 replica_placement:100 version:3 modified_at_second:1615269124
+ volume id:212 size:1078293360 collection:"collection1" file_count:3106 delete_count:6 deleted_byte_count:2085755 replica_placement:100 version:3 modified_at_second:1615586387
+ volume id:213 size:1093587976 collection:"collection1" file_count:3681 delete_count:12 deleted_byte_count:3138791 replica_placement:100 version:3 modified_at_second:1615586387
+ volume id:214 size:1074486992 collection:"collection1" file_count:3217 delete_count:10 deleted_byte_count:6392871 replica_placement:100 version:3 modified_at_second:1615586383
+ volume id:216 size:1080073496 collection:"collection1" file_count:3316 delete_count:4 deleted_byte_count:179819 replica_placement:100 version:3 modified_at_second:1615586387
+ volume id:222 size:1106623104 collection:"collection1" file_count:3273 delete_count:11 deleted_byte_count:2114627 replica_placement:100 version:3 modified_at_second:1615586243
+ volume id:223 size:1075233064 collection:"collection1" file_count:2966 delete_count:9 deleted_byte_count:744001 replica_placement:100 version:3 modified_at_second:1615586244
+ volume id:227 size:1106699896 collection:"collection1" file_count:2827 delete_count:20 deleted_byte_count:5496790 replica_placement:100 version:3 modified_at_second:1615609989
+ volume id:229 size:1109855312 collection:"collection1" file_count:2857 delete_count:22 deleted_byte_count:2839883 replica_placement:100 version:3 modified_at_second:1615609988
+ volume id:230 size:1080722984 collection:"collection1" file_count:2898 delete_count:15 deleted_byte_count:3929261 replica_placement:100 version:3 modified_at_second:1615610537
+ volume id:231 size:1112917696 collection:"collection1" file_count:3151 delete_count:20 deleted_byte_count:2989828 replica_placement:100 version:3 modified_at_second:1615611350
+ volume id:233 size:1080526464 collection:"collection1" file_count:3136 delete_count:61 deleted_byte_count:17991717 replica_placement:100 version:3 modified_at_second:1615611352
+ volume id:234 size:1073835280 collection:"collection1" file_count:2965 delete_count:41 deleted_byte_count:4960354 replica_placement:100 version:3 modified_at_second:1615611351
+ volume id:235 size:1075586104 collection:"collection1" file_count:2767 delete_count:33 deleted_byte_count:3216540 replica_placement:100 version:3 modified_at_second:1615611354
+ volume id:237 size:375722792 collection:"collection1" file_count:736 delete_count:16 deleted_byte_count:4464870 replica_placement:100 version:3 modified_at_second:1615631727
+ volume id:239 size:426569024 collection:"collection1" file_count:693 delete_count:19 deleted_byte_count:13020783 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630838
+ volume id:241 size:380217424 collection:"collection1" file_count:633 delete_count:6 deleted_byte_count:1715768 replica_placement:100 version:3 modified_at_second:1615632006
+ volume id:244 size:1080295352 collection:"collection2" file_count:10812 delete_count:1 deleted_byte_count:795 replica_placement:100 version:3 modified_at_second:1614852171
+ volume id:245 size:1074597056 collection:"collection2" file_count:10371 delete_count:3 deleted_byte_count:209701 replica_placement:100 version:3 modified_at_second:1614852093
+ volume id:246 size:1075998648 collection:"collection2" file_count:10365 delete_count:1 deleted_byte_count:13112 replica_placement:100 version:3 modified_at_second:1614852105
+ volume id:248 size:1084301184 collection:"collection2" file_count:11217 delete_count:4 deleted_byte_count:746488 replica_placement:100 version:3 modified_at_second:1614856285
+ volume id:249 size:1074819136 collection:"collection2" file_count:10763 delete_count:2 deleted_byte_count:271699 replica_placement:100 version:3 modified_at_second:1614856230
+ volume id:251 size:1075684488 collection:"collection2" file_count:10847 replica_placement:100 version:3 modified_at_second:1614856270
+ volume id:252 size:1075065208 collection:"collection2" file_count:14622 delete_count:2 deleted_byte_count:5228 replica_placement:100 version:3 modified_at_second:1614861196
+ volume id:253 size:1087328816 collection:"collection2" file_count:14920 delete_count:3 deleted_byte_count:522994 replica_placement:100 version:3 modified_at_second:1614861255
+ volume id:255 size:1079581640 collection:"collection2" file_count:14877 delete_count:3 deleted_byte_count:101223 replica_placement:100 version:3 modified_at_second:1614861233
+ volume id:256 size:1074283592 collection:"collection2" file_count:14157 delete_count:1 deleted_byte_count:18156 replica_placement:100 version:3 modified_at_second:1614861100
+ volume id:258 size:1075527216 collection:"collection2" file_count:18421 delete_count:4 deleted_byte_count:267833 replica_placement:100 version:3 modified_at_second:1614866420
+ volume id:259 size:1075507776 collection:"collection2" file_count:18079 delete_count:2 deleted_byte_count:71992 replica_placement:100 version:3 modified_at_second:1614866381
+ volume id:264 size:1081624192 collection:"collection2" file_count:21151 replica_placement:100 version:3 modified_at_second:1614871629
+ volume id:265 size:1076401104 collection:"collection2" file_count:19932 delete_count:2 deleted_byte_count:160823 replica_placement:100 version:3 modified_at_second:1615629130
+ volume id:266 size:1075617464 collection:"collection2" file_count:20075 delete_count:1 deleted_byte_count:1039 replica_placement:100 version:3 modified_at_second:1614871526
+ volume id:267 size:1075699544 collection:"collection2" file_count:21039 delete_count:3 deleted_byte_count:59956 replica_placement:100 version:3 modified_at_second:1614877294
+ volume id:268 size:1074490592 collection:"collection2" file_count:21698 delete_count:1 deleted_byte_count:33968 replica_placement:100 version:3 modified_at_second:1614877434
+ volume id:269 size:1077552872 collection:"collection2" file_count:21875 delete_count:4 deleted_byte_count:347272 replica_placement:100 version:3 modified_at_second:1614877481
+ volume id:270 size:1076876568 collection:"collection2" file_count:22057 delete_count:1 deleted_byte_count:43916 replica_placement:100 version:3 modified_at_second:1614877469
+ volume id:275 size:1078349024 collection:"collection2" file_count:20808 delete_count:1 deleted_byte_count:1118 replica_placement:100 version:3 modified_at_second:1614884147
+ volume id:277 size:1074956288 collection:"collection2" file_count:19260 delete_count:2 deleted_byte_count:172356 replica_placement:100 version:3 modified_at_second:1614889988
+ volume id:278 size:1078798640 collection:"collection2" file_count:20597 delete_count:5 deleted_byte_count:400060 replica_placement:100 version:3 modified_at_second:1614890292
+ volume id:279 size:1077325040 collection:"collection2" file_count:19671 delete_count:6 deleted_byte_count:379116 replica_placement:100 version:3 modified_at_second:1614890229
+ volume id:280 size:1077432216 collection:"collection2" file_count:20286 delete_count:1 deleted_byte_count:879 replica_placement:100 version:3 modified_at_second:1614890262
+ volume id:281 size:1077581096 collection:"collection2" file_count:20206 delete_count:3 deleted_byte_count:143964 replica_placement:100 version:3 modified_at_second:1614890237
+ volume id:284 size:1074533384 collection:"collection2" file_count:22196 delete_count:4 deleted_byte_count:154683 replica_placement:100 version:3 modified_at_second:1614897231
+ volume id:285 size:1082128688 collection:"collection2" file_count:21804 delete_count:1 deleted_byte_count:1064 replica_placement:100 version:3 modified_at_second:1614897165
+ volume id:289 size:1075284256 collection:"collection2" file_count:29342 delete_count:5 deleted_byte_count:100454 replica_placement:100 version:3 modified_at_second:1614904977
+ volume id:290 size:1074723792 collection:"collection2" file_count:28340 delete_count:4 deleted_byte_count:199064 replica_placement:100 version:3 modified_at_second:1614904924
+ volume id:291 size:1076256768 collection:"collection2" file_count:28301 delete_count:5 deleted_byte_count:116027 replica_placement:100 version:3 modified_at_second:1614904924
+ volume id:293 size:1075409792 collection:"collection2" file_count:26063 delete_count:4 deleted_byte_count:183834 replica_placement:100 version:3 modified_at_second:1614912235
+ volume id:294 size:1075444048 collection:"collection2" file_count:26076 delete_count:4 deleted_byte_count:194914 replica_placement:100 version:3 modified_at_second:1614912220
+ volume id:296 size:1077824032 collection:"collection2" file_count:26741 delete_count:4 deleted_byte_count:199906 replica_placement:100 version:3 modified_at_second:1614912301
+ volume id:297 size:1080229136 collection:"collection2" file_count:23409 delete_count:5 deleted_byte_count:46268 replica_placement:100 version:3 modified_at_second:1614918481
+ volume id:298 size:1075410136 collection:"collection2" file_count:23222 delete_count:2 deleted_byte_count:46110 replica_placement:100 version:3 modified_at_second:1614918474
+ volume id:299 size:1075147936 collection:"collection2" file_count:22927 delete_count:4 deleted_byte_count:345569 replica_placement:100 version:3 modified_at_second:1614918455
+ volume id:300 size:1076212392 collection:"collection2" file_count:22892 delete_count:2 deleted_byte_count:61320 replica_placement:100 version:3 modified_at_second:1614918464
+ volume id:301 size:1074655600 collection:"collection2" file_count:22543 delete_count:6 deleted_byte_count:136968 replica_placement:100 version:3 modified_at_second:1614918378
+ volume id:303 size:1075944480 collection:"collection2" file_count:22541 delete_count:2 deleted_byte_count:13617 replica_placement:100 version:3 modified_at_second:1614925431
+ volume id:306 size:1074764016 collection:"collection2" file_count:22939 replica_placement:100 version:3 modified_at_second:1614925462
+ volume id:307 size:1076568000 collection:"collection2" file_count:23377 delete_count:2 deleted_byte_count:25453 replica_placement:100 version:3 modified_at_second:1614931448
+ volume id:308 size:1074022392 collection:"collection2" file_count:23086 delete_count:2 deleted_byte_count:2127 replica_placement:100 version:3 modified_at_second:1614931401
+ volume id:309 size:1075178664 collection:"collection2" file_count:22692 delete_count:3 deleted_byte_count:171124 replica_placement:100 version:3 modified_at_second:1614931409
+ volume id:310 size:1074761528 collection:"collection2" file_count:21441 delete_count:3 deleted_byte_count:13934 replica_placement:100 version:3 modified_at_second:1614931077
+ volume id:314 size:1074670840 collection:"collection2" file_count:20964 delete_count:4 deleted_byte_count:304291 replica_placement:100 version:3 modified_at_second:1614937441
+ volume id:315 size:1084153544 collection:"collection2" file_count:23638 delete_count:2 deleted_byte_count:53956 replica_placement:100 version:3 modified_at_second:1614937885
+ volume id:317 size:1076215096 collection:"collection2" file_count:23572 delete_count:2 deleted_byte_count:1441356 replica_placement:100 version:3 modified_at_second:1614943965
+ volume id:318 size:1075965168 collection:"collection2" file_count:22459 delete_count:2 deleted_byte_count:37778 replica_placement:100 version:3 modified_at_second:1614943862
+ volume id:319 size:1073952880 collection:"collection2" file_count:22286 delete_count:2 deleted_byte_count:43421 replica_placement:100 version:3 modified_at_second:1614943810
+ volume id:320 size:1082437792 collection:"collection2" file_count:21544 delete_count:3 deleted_byte_count:16712 replica_placement:100 version:3 modified_at_second:1614943599
+ volume id:321 size:1081477904 collection:"collection2" file_count:23531 delete_count:5 deleted_byte_count:262564 replica_placement:100 version:3 modified_at_second:1614943982
+ volume id:324 size:1075606680 collection:"collection2" file_count:20799 delete_count:1 deleted_byte_count:251210 replica_placement:100 version:3 modified_at_second:1614950310
+ volume id:325 size:1080701144 collection:"collection2" file_count:21735 replica_placement:100 version:3 modified_at_second:1614950525
+ volume id:330 size:1080825832 collection:"collection2" file_count:22464 delete_count:2 deleted_byte_count:15771 replica_placement:100 version:3 modified_at_second:1614956477
+ volume id:332 size:1075569928 collection:"collection2" file_count:22097 delete_count:3 deleted_byte_count:98273 replica_placement:100 version:3 modified_at_second:1614962869
+ volume id:334 size:1075607880 collection:"collection2" file_count:22546 delete_count:6 deleted_byte_count:101538 replica_placement:100 version:3 modified_at_second:1614962978
+ volume id:336 size:1087853056 collection:"collection2" file_count:22801 delete_count:2 deleted_byte_count:26394 replica_placement:100 version:3 modified_at_second:1614963005
+ volume id:337 size:1075646784 collection:"collection2" file_count:21934 delete_count:1 deleted_byte_count:3397 replica_placement:100 version:3 modified_at_second:1614969937
+ volume id:338 size:1076118304 collection:"collection2" file_count:21680 replica_placement:100 version:3 modified_at_second:1614969850
+ volume id:340 size:1079462184 collection:"collection2" file_count:22319 delete_count:4 deleted_byte_count:93620 replica_placement:100 version:3 modified_at_second:1614969978
+ volume id:341 size:1074448400 collection:"collection2" file_count:21590 delete_count:5 deleted_byte_count:160085 replica_placement:100 version:3 modified_at_second:1614969858
+ volume id:342 size:1080186424 collection:"collection2" file_count:22405 delete_count:4 deleted_byte_count:64819 replica_placement:100 version:3 modified_at_second:1614977521
+ volume id:344 size:1075035416 collection:"collection2" file_count:21765 delete_count:1 deleted_byte_count:24623 replica_placement:100 version:3 modified_at_second:1614977465
+ volume id:345 size:1074560760 collection:"collection2" file_count:22117 delete_count:2 deleted_byte_count:373286 replica_placement:100 version:3 modified_at_second:1614977457
+ volume id:346 size:1076464112 collection:"collection2" file_count:22320 delete_count:4 deleted_byte_count:798258 replica_placement:100 version:3 modified_at_second:1615631322
+ volume id:348 size:1080623640 collection:"collection2" file_count:21667 delete_count:1 deleted_byte_count:2443 replica_placement:100 version:3 modified_at_second:1614984606
+ volume id:350 size:1074756688 collection:"collection2" file_count:21990 delete_count:3 deleted_byte_count:233881 replica_placement:100 version:3 modified_at_second:1614984682
+ volume id:351 size:1078795112 collection:"collection2" file_count:23660 delete_count:3 deleted_byte_count:102141 replica_placement:100 version:3 modified_at_second:1614984816
+ volume id:352 size:1077145936 collection:"collection2" file_count:22066 delete_count:1 deleted_byte_count:1018 replica_placement:100 version:3 modified_at_second:1614992130
+ volume id:353 size:1074897496 collection:"collection2" file_count:21266 delete_count:2 deleted_byte_count:3105374 replica_placement:100 version:3 modified_at_second:1614991951
+ volume id:355 size:1075239728 collection:"collection2" file_count:22244 delete_count:1 deleted_byte_count:23282 replica_placement:100 version:3 modified_at_second:1614992157
+ volume id:356 size:1083305048 collection:"collection2" file_count:21552 delete_count:4 deleted_byte_count:14472 replica_placement:100 version:3 modified_at_second:1614992028
+ volume id:358 size:1085152368 collection:"collection2" file_count:23756 delete_count:3 deleted_byte_count:44531 replica_placement:100 version:3 modified_at_second:1614998824
+ volume id:360 size:1075532456 collection:"collection2" file_count:22574 delete_count:3 deleted_byte_count:1774776 replica_placement:100 version:3 modified_at_second:1614998770
+ volume id:361 size:1075362744 collection:"collection2" file_count:22272 delete_count:1 deleted_byte_count:3497 replica_placement:100 version:3 modified_at_second:1614998668
+ volume id:375 size:1076140568 collection:"collection2" file_count:21880 delete_count:2 deleted_byte_count:51103 replica_placement:100 version:3 modified_at_second:1615016787
+ volume id:376 size:1074845944 collection:"collection2" file_count:22908 delete_count:4 deleted_byte_count:432305 replica_placement:100 version:3 modified_at_second:1615016916
+ volume id:377 size:957284144 collection:"collection2" file_count:14923 delete_count:1 deleted_byte_count:1797 replica_placement:100 version:3 modified_at_second:1615632323
+ volume id:378 size:959273936 collection:"collection2" file_count:15027 delete_count:4 deleted_byte_count:231414 replica_placement:100 version:3 modified_at_second:1615632323
+ volume id:381 size:939261032 collection:"collection2" file_count:14615 delete_count:5 deleted_byte_count:1192272 replica_placement:100 version:3 modified_at_second:1615632324
+ Disk hdd total size:111617646696 file_count:1762773 deleted_file:1221 deleted_bytes:398484585
+ DataNode 192.168.1.6:8080 total size:111617646696 file_count:1762773 deleted_file:1221 deleted_bytes:398484585
+ Rack rack3 total size:111617646696 file_count:1762773 deleted_file:1221 deleted_bytes:398484585
+ DataCenter dc3 total size:111617646696 file_count:1762773 deleted_file:1221 deleted_bytes:398484585
+ DataCenter dc4 hdd(volume:267/2000 active:267 free:1733 remote:0)
+ Rack DefaultRack hdd(volume:267/2000 active:267 free:1733 remote:0)
+ DataNode 192.168.1.1:8080 hdd(volume:267/2000 active:267 free:1733 remote:0)
+ Disk hdd(volume:267/2000 active:267 free:1733 remote:0)
+ volume id:1 size:284693256 file_count:1558 delete_count:2 deleted_byte_count:4818 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632062
+ volume id:2 size:289228560 file_count:1640 delete_count:4 deleted_byte_count:464508 replica_placement:100 version:3 compact_revision:6 modified_at_second:1615630622
+ volume id:3 size:308741952 file_count:1637 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632242
+ volume id:4 size:285986968 file_count:1640 delete_count:1 deleted_byte_count:145095 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632302
+ volume id:5 size:293806008 file_count:1669 delete_count:2 deleted_byte_count:274334 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631342
+ volume id:6 size:302411024 file_count:1604 delete_count:2 deleted_byte_count:274587 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631402
+ volume id:7 size:1924728 collection:"collection4" file_count:15 replica_placement:100 version:3 modified_at_second:1609331040
+ volume id:9 size:77337416 collection:"collection3" file_count:58 replica_placement:100 version:3 ttl:772 modified_at_second:1615513762
+ volume id:10 size:1212784656 collection:"collection0" file_count:58 replica_placement:100 version:3 modified_at_second:1609814543
+ volume id:11 size:1109224552 collection:"collection0" file_count:44 replica_placement:100 version:3 modified_at_second:1609815123
+ volume id:12 size:1110923848 collection:"collection0" file_count:45 replica_placement:100 version:3 modified_at_second:1609819726
+ volume id:13 size:1184910656 collection:"collection0" file_count:47 replica_placement:100 version:3 modified_at_second:1609827832
+ volume id:14 size:1107475720 collection:"collection0" file_count:80 delete_count:3 deleted_byte_count:6870 replica_placement:100 version:3 modified_at_second:1612956983
+ volume id:15 size:1115965160 collection:"collection0" file_count:83 delete_count:3 deleted_byte_count:4956 replica_placement:100 version:3 modified_at_second:1612957001
+ volume id:16 size:1113666048 collection:"collection0" file_count:73 delete_count:5 deleted_byte_count:6318 replica_placement:100 version:3 modified_at_second:1612957007
+ volume id:17 size:1095115800 collection:"collection0" file_count:83 delete_count:3 deleted_byte_count:7099 replica_placement:100 version:3 modified_at_second:1612957000
+ volume id:18 size:1096678688 collection:"collection0" file_count:88 delete_count:4 deleted_byte_count:8633 replica_placement:100 version:3 modified_at_second:1612957000
+ volume id:19 size:1096923792 collection:"collection0" file_count:100 delete_count:10 deleted_byte_count:75779917 replica_placement:100 version:3 compact_revision:4 modified_at_second:1612957011
+ volume id:20 size:1074760432 collection:"collection0" file_count:82 delete_count:5 deleted_byte_count:12156 replica_placement:100 version:3 compact_revision:2 modified_at_second:1612957011
+ volume id:22 size:1086828368 collection:"collection0" file_count:75 delete_count:3 deleted_byte_count:5551 replica_placement:100 version:3 modified_at_second:1612957007
+ volume id:23 size:1076380280 collection:"collection0" file_count:68 delete_count:2 deleted_byte_count:2910 replica_placement:100 version:3 modified_at_second:1612957011
+ volume id:24 size:1074139808 collection:"collection0" file_count:90 delete_count:1 deleted_byte_count:1977 replica_placement:100 version:3 modified_at_second:1612957011
+ volume id:25 size:690757544 collection:"collection0" file_count:38 delete_count:1 deleted_byte_count:1944 replica_placement:100 version:3 modified_at_second:1612956995
+ volume id:26 size:532657632 collection:"collection0" file_count:100 delete_count:4 deleted_byte_count:9081 replica_placement:100 version:3 modified_at_second:1614170023
+ volume id:34 size:1077111136 collection:"collection1" file_count:9781 delete_count:110 deleted_byte_count:20894827 replica_placement:100 version:3 modified_at_second:1615619366
+ volume id:35 size:1075241656 collection:"collection1" file_count:10523 delete_count:96 deleted_byte_count:46618989 replica_placement:100 version:3 modified_at_second:1615618790
+ volume id:36 size:1075118360 collection:"collection1" file_count:10342 delete_count:116 deleted_byte_count:25493106 replica_placement:100 version:3 modified_at_second:1615606148
+ volume id:37 size:1075895584 collection:"collection1" file_count:12013 delete_count:98 deleted_byte_count:50747932 replica_placement:100 version:3 modified_at_second:1615594777
+ volume id:39 size:1076606536 collection:"collection1" file_count:12612 delete_count:78 deleted_byte_count:17462730 replica_placement:100 version:3 modified_at_second:1615611959
+ volume id:40 size:1075358552 collection:"collection1" file_count:12597 delete_count:62 deleted_byte_count:11657901 replica_placement:100 version:3 modified_at_second:1615612994
+ volume id:41 size:1076283528 collection:"collection1" file_count:12088 delete_count:84 deleted_byte_count:19319268 replica_placement:100 version:3 modified_at_second:1615596736
+ volume id:42 size:1093948352 collection:"collection1" file_count:7889 delete_count:47 deleted_byte_count:5697275 replica_placement:100 version:3 modified_at_second:1615548908
+ volume id:43 size:1116445864 collection:"collection1" file_count:7358 delete_count:54 deleted_byte_count:9534379 replica_placement:100 version:3 modified_at_second:1615566170
+ volume id:44 size:1077582560 collection:"collection1" file_count:7295 delete_count:50 deleted_byte_count:12618414 replica_placement:100 version:3 modified_at_second:1615566170
+ volume id:45 size:1075254640 collection:"collection1" file_count:10772 delete_count:76 deleted_byte_count:22426345 replica_placement:100 version:3 modified_at_second:1615573499
+ volume id:46 size:1075286056 collection:"collection1" file_count:9947 delete_count:309 deleted_byte_count:105601163 replica_placement:100 version:3 modified_at_second:1615569826
+ volume id:48 size:1076778720 collection:"collection1" file_count:9850 delete_count:77 deleted_byte_count:16641907 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630690
+ volume id:50 size:1076688224 collection:"collection1" file_count:7921 delete_count:26 deleted_byte_count:5162032 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615610879
+ volume id:52 size:1083529704 collection:"collection1" file_count:10128 delete_count:32 deleted_byte_count:10608391 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615599195
+ volume id:53 size:1063089216 collection:"collection1" file_count:9832 delete_count:31 deleted_byte_count:9273066 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632156
+ volume id:55 size:1012890016 collection:"collection1" file_count:8651 delete_count:27 deleted_byte_count:9418841 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631452
+ volume id:57 size:839849792 collection:"collection1" file_count:7514 delete_count:24 deleted_byte_count:6228543 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631774
+ volume id:58 size:908064200 collection:"collection1" file_count:8128 delete_count:21 deleted_byte_count:6113731 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632342
+ volume id:59 size:988302272 collection:"collection1" file_count:8098 delete_count:20 deleted_byte_count:3947615 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632238
+ volume id:60 size:1010702480 collection:"collection1" file_count:8969 delete_count:79 deleted_byte_count:24782814 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615632439
+ volume id:61 size:975604488 collection:"collection1" file_count:8683 delete_count:20 deleted_byte_count:10276072 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615631176
+ volume id:62 size:873845936 collection:"collection1" file_count:7897 delete_count:23 deleted_byte_count:10920170 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631133
+ volume id:64 size:965638488 collection:"collection1" file_count:8218 delete_count:27 deleted_byte_count:6922489 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631031
+ volume id:65 size:823283552 collection:"collection1" file_count:7834 delete_count:29 deleted_byte_count:5950610 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632306
+ volume id:66 size:821343440 collection:"collection1" file_count:7383 delete_count:29 deleted_byte_count:12010343 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631968
+ volume id:67 size:878713872 collection:"collection1" file_count:7299 delete_count:117 deleted_byte_count:24857326 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632156
+ volume id:68 size:898630584 collection:"collection1" file_count:6934 delete_count:95 deleted_byte_count:27460707 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632284
+ volume id:70 size:886695472 collection:"collection1" file_count:7769 delete_count:164 deleted_byte_count:45162513 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632398
+ volume id:71 size:907608392 collection:"collection1" file_count:7658 delete_count:122 deleted_byte_count:27622941 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632307
+ volume id:72 size:903990720 collection:"collection1" file_count:6996 delete_count:240 deleted_byte_count:74147727 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630982
+ volume id:73 size:929047664 collection:"collection1" file_count:7038 delete_count:227 deleted_byte_count:65336664 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630707
+ volume id:74 size:957046128 collection:"collection1" file_count:6981 delete_count:259 deleted_byte_count:73080838 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631460
+ volume id:75 size:908044992 collection:"collection1" file_count:6911 delete_count:268 deleted_byte_count:73934373 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632430
+ volume id:76 size:985296744 collection:"collection1" file_count:6566 delete_count:61 deleted_byte_count:44464430 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632284
+ volume id:77 size:929398296 collection:"collection1" file_count:7427 delete_count:238 deleted_byte_count:59581579 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632013
+ volume id:78 size:1075671512 collection:"collection1" file_count:7540 delete_count:258 deleted_byte_count:71726846 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615582829
+ volume id:79 size:948225472 collection:"collection1" file_count:6997 delete_count:227 deleted_byte_count:60625763 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631326
+ volume id:82 size:1041661800 collection:"collection1" file_count:7043 delete_count:207 deleted_byte_count:52275724 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632430
+ volume id:83 size:936195856 collection:"collection1" file_count:7593 delete_count:13 deleted_byte_count:4633917 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632029
+ volume id:85 size:1023867520 collection:"collection1" file_count:7787 delete_count:240 deleted_byte_count:82091742 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631723
+ volume id:86 size:1009437488 collection:"collection1" file_count:8474 delete_count:236 deleted_byte_count:64543674 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630812
+ volume id:87 size:922276640 collection:"collection1" file_count:12902 delete_count:13 deleted_byte_count:3412959 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632438
+ volume id:89 size:1044401976 collection:"collection1" file_count:14943 delete_count:243 deleted_byte_count:58543159 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632208
+ volume id:90 size:891145784 collection:"collection1" file_count:14608 delete_count:10 deleted_byte_count:2564369 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615629390
+ volume id:91 size:936572832 collection:"collection1" file_count:14686 delete_count:11 deleted_byte_count:4717727 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631851
+ volume id:92 size:992440712 collection:"collection1" file_count:7061 delete_count:195 deleted_byte_count:60649573 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630566
+ volume id:93 size:1079603768 collection:"collection1" file_count:7878 delete_count:270 deleted_byte_count:74150048 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615556015
+ volume id:94 size:1030685824 collection:"collection1" file_count:7660 delete_count:207 deleted_byte_count:70150733 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631616
+ volume id:95 size:990879168 collection:"collection1" file_count:6620 delete_count:206 deleted_byte_count:60363604 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615631866
+ volume id:96 size:989296136 collection:"collection1" file_count:7544 delete_count:229 deleted_byte_count:59931853 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630778
+ volume id:97 size:1053112992 collection:"collection1" file_count:6789 delete_count:50 deleted_byte_count:38894001 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631194
+ volume id:99 size:1071718504 collection:"collection1" file_count:7470 delete_count:8 deleted_byte_count:9624950 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631175
+ volume id:100 size:1083617440 collection:"collection1" file_count:7018 delete_count:187 deleted_byte_count:61304236 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615505917
+ volume id:101 size:1077109520 collection:"collection1" file_count:7706 delete_count:226 deleted_byte_count:77864841 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630994
+ volume id:102 size:1074359920 collection:"collection1" file_count:7338 delete_count:7 deleted_byte_count:6499151 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615626683
+ volume id:103 size:1075863904 collection:"collection1" file_count:7184 delete_count:186 deleted_byte_count:58872238 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615628417
+ volume id:104 size:1076383768 collection:"collection1" file_count:7663 delete_count:184 deleted_byte_count:100578087 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615602661
+ volume id:105 size:1073996824 collection:"collection1" file_count:6873 delete_count:19 deleted_byte_count:14271533 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615499756
+ volume id:108 size:1074648024 collection:"collection1" file_count:7472 delete_count:194 deleted_byte_count:70864699 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615593232
+ volume id:109 size:1075254560 collection:"collection1" file_count:7556 delete_count:263 deleted_byte_count:55155265 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615502487
+ volume id:110 size:1076575744 collection:"collection1" file_count:6996 delete_count:163 deleted_byte_count:52954032 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615590786
+ volume id:111 size:1073826232 collection:"collection1" file_count:7355 delete_count:155 deleted_byte_count:50083578 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615593233
+ volume id:114 size:1074762784 collection:"collection1" file_count:8802 delete_count:156 deleted_byte_count:38470055 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615591826
+ volume id:115 size:1076192240 collection:"collection1" file_count:7690 delete_count:154 deleted_byte_count:32267193 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615285295
+ volume id:116 size:1074489504 collection:"collection1" file_count:9981 delete_count:174 deleted_byte_count:53998777 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615611567
+ volume id:117 size:1073917192 collection:"collection1" file_count:9520 delete_count:114 deleted_byte_count:21835126 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615573714
+ volume id:118 size:1074064400 collection:"collection1" file_count:8738 delete_count:15 deleted_byte_count:3460697 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615516265
+ volume id:119 size:1075940104 collection:"collection1" file_count:9003 delete_count:12 deleted_byte_count:9128155 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615573880
+ volume id:120 size:1076115928 collection:"collection1" file_count:9639 delete_count:118 deleted_byte_count:33357871 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615482567
+ volume id:121 size:1078803248 collection:"collection1" file_count:10113 delete_count:441 deleted_byte_count:94128627 replica_placement:100 version:3 modified_at_second:1615506629
+ volume id:122 size:1076235312 collection:"collection1" file_count:9106 delete_count:252 deleted_byte_count:93041272 replica_placement:100 version:3 modified_at_second:1615585913
+ volume id:123 size:1080491112 collection:"collection1" file_count:10623 delete_count:302 deleted_byte_count:83956998 replica_placement:100 version:3 modified_at_second:1615585916
+ volume id:124 size:1074519360 collection:"collection1" file_count:9457 delete_count:286 deleted_byte_count:74752459 replica_placement:100 version:3 modified_at_second:1615585916
+ volume id:125 size:1088687040 collection:"collection1" file_count:9518 delete_count:281 deleted_byte_count:76037905 replica_placement:100 version:3 modified_at_second:1615585913
+ volume id:126 size:1073867464 collection:"collection1" file_count:9320 delete_count:278 deleted_byte_count:94547424 replica_placement:100 version:3 modified_at_second:1615585912
+ volume id:127 size:1074907336 collection:"collection1" file_count:9900 delete_count:133 deleted_byte_count:48570820 replica_placement:100 version:3 modified_at_second:1615612991
+ volume id:129 size:1074704272 collection:"collection1" file_count:10012 delete_count:150 deleted_byte_count:64491721 replica_placement:100 version:3 modified_at_second:1615627566
+ volume id:130 size:1075000632 collection:"collection1" file_count:10633 delete_count:161 deleted_byte_count:34768201 replica_placement:100 version:3 modified_at_second:1615582330
+ volume id:131 size:1075279584 collection:"collection1" file_count:10075 delete_count:135 deleted_byte_count:29795712 replica_placement:100 version:3 modified_at_second:1615523898
+ volume id:132 size:1088539496 collection:"collection1" file_count:11051 delete_count:71 deleted_byte_count:17178322 replica_placement:100 version:3 modified_at_second:1615619584
+ volume id:133 size:1075952760 collection:"collection1" file_count:9538 delete_count:74 deleted_byte_count:19558008 replica_placement:100 version:3 modified_at_second:1615584780
+ volume id:134 size:1074367304 collection:"collection1" file_count:10662 delete_count:69 deleted_byte_count:25530139 replica_placement:100 version:3 modified_at_second:1615555876
+ volume id:135 size:1073906720 collection:"collection1" file_count:10446 delete_count:71 deleted_byte_count:28599975 replica_placement:100 version:3 modified_at_second:1615569816
+ volume id:137 size:1074309264 collection:"collection1" file_count:9633 delete_count:50 deleted_byte_count:27487972 replica_placement:100 version:3 modified_at_second:1615572231
+ volume id:139 size:1074163936 collection:"collection1" file_count:9314 delete_count:43 deleted_byte_count:10631353 replica_placement:100 version:3 modified_at_second:1615571946
+ volume id:141 size:1074619488 collection:"collection1" file_count:9840 delete_count:45 deleted_byte_count:40890181 replica_placement:100 version:3 modified_at_second:1615630994
+ volume id:142 size:1075732992 collection:"collection1" file_count:9009 delete_count:48 deleted_byte_count:9912854 replica_placement:100 version:3 modified_at_second:1615598914
+ volume id:143 size:1075011280 collection:"collection1" file_count:9608 delete_count:51 deleted_byte_count:37282460 replica_placement:100 version:3 modified_at_second:1615488586
+ volume id:145 size:1074394928 collection:"collection1" file_count:9255 delete_count:34 deleted_byte_count:38011392 replica_placement:100 version:3 modified_at_second:1615591825
+ volume id:146 size:1076337520 collection:"collection1" file_count:10492 delete_count:50 deleted_byte_count:17071505 replica_placement:100 version:3 modified_at_second:1615632005
+ volume id:147 size:1077130544 collection:"collection1" file_count:10451 delete_count:27 deleted_byte_count:8290907 replica_placement:100 version:3 modified_at_second:1615604117
+ volume id:148 size:1076066568 collection:"collection1" file_count:9547 delete_count:33 deleted_byte_count:7034089 replica_placement:100 version:3 modified_at_second:1615586393
+ volume id:149 size:1074989016 collection:"collection1" file_count:8352 delete_count:35 deleted_byte_count:7179742 replica_placement:100 version:3 modified_at_second:1615494496
+ volume id:150 size:1076290408 collection:"collection1" file_count:9328 delete_count:33 deleted_byte_count:43417791 replica_placement:100 version:3 modified_at_second:1615611569
+ volume id:151 size:1098659752 collection:"collection1" file_count:10805 delete_count:27 deleted_byte_count:7209106 replica_placement:100 version:3 modified_at_second:1615586390
+ volume id:152 size:1075941376 collection:"collection1" file_count:9951 delete_count:36 deleted_byte_count:25348335 replica_placement:100 version:3 modified_at_second:1615606614
+ volume id:153 size:1078539784 collection:"collection1" file_count:10924 delete_count:34 deleted_byte_count:12603081 replica_placement:100 version:3 modified_at_second:1615606614
+ volume id:154 size:1081244752 collection:"collection1" file_count:11002 delete_count:31 deleted_byte_count:8467560 replica_placement:100 version:3 modified_at_second:1615478471
+ volume id:156 size:1074975832 collection:"collection1" file_count:9535 delete_count:40 deleted_byte_count:11426621 replica_placement:100 version:3 modified_at_second:1615628342
+ volume id:157 size:1076758536 collection:"collection1" file_count:10012 delete_count:19 deleted_byte_count:11688737 replica_placement:100 version:3 modified_at_second:1615597782
+ volume id:158 size:1087251976 collection:"collection1" file_count:9972 delete_count:20 deleted_byte_count:10328429 replica_placement:100 version:3 modified_at_second:1615588879
+ volume id:159 size:1074132336 collection:"collection1" file_count:9382 delete_count:27 deleted_byte_count:11474574 replica_placement:100 version:3 modified_at_second:1615593593
+ volume id:160 size:1075680976 collection:"collection1" file_count:9772 delete_count:22 deleted_byte_count:4981968 replica_placement:100 version:3 modified_at_second:1615597782
+ volume id:161 size:1077397136 collection:"collection1" file_count:9988 delete_count:28 deleted_byte_count:12509164 replica_placement:100 version:3 modified_at_second:1615631452
+ volume id:162 size:1074286880 collection:"collection1" file_count:11220 delete_count:17 deleted_byte_count:1815547 replica_placement:100 version:3 modified_at_second:1615478127
+ volume id:163 size:1074457224 collection:"collection1" file_count:12524 delete_count:27 deleted_byte_count:6359619 replica_placement:100 version:3 modified_at_second:1615579313
+ volume id:164 size:1074261256 collection:"collection1" file_count:11922 delete_count:25 deleted_byte_count:2923288 replica_placement:100 version:3 modified_at_second:1615620085
+ volume id:165 size:1073891080 collection:"collection1" file_count:9152 delete_count:12 deleted_byte_count:19164659 replica_placement:100 version:3 modified_at_second:1615471910
+ volume id:166 size:1075637536 collection:"collection1" file_count:14211 delete_count:24 deleted_byte_count:20415490 replica_placement:100 version:3 modified_at_second:1615491021
+ volume id:167 size:1073958280 collection:"collection1" file_count:25231 delete_count:48 deleted_byte_count:26022344 replica_placement:100 version:3 modified_at_second:1615620014
+ volume id:168 size:1074718864 collection:"collection1" file_count:25702 delete_count:40 deleted_byte_count:4024775 replica_placement:100 version:3 modified_at_second:1615585664
+ volume id:169 size:1073863032 collection:"collection1" file_count:25248 delete_count:43 deleted_byte_count:3013817 replica_placement:100 version:3 modified_at_second:1615569832
+ volume id:170 size:1075747088 collection:"collection1" file_count:24596 delete_count:41 deleted_byte_count:3494711 replica_placement:100 version:3 modified_at_second:1615579207
+ volume id:171 size:1081881400 collection:"collection1" file_count:24215 delete_count:36 deleted_byte_count:3191335 replica_placement:100 version:3 modified_at_second:1615596486
+ volume id:172 size:1074787304 collection:"collection1" file_count:31236 delete_count:50 deleted_byte_count:3316482 replica_placement:100 version:3 modified_at_second:1615612385
+ volume id:174 size:1073824160 collection:"collection1" file_count:30689 delete_count:36 deleted_byte_count:2160116 replica_placement:100 version:3 modified_at_second:1615598914
+ volume id:175 size:1077742472 collection:"collection1" file_count:32353 delete_count:33 deleted_byte_count:1861403 replica_placement:100 version:3 modified_at_second:1615559516
+ volume id:176 size:1073854800 collection:"collection1" file_count:30582 delete_count:34 deleted_byte_count:7701976 replica_placement:100 version:3 modified_at_second:1615626169
+ volume id:178 size:1087560112 collection:"collection1" file_count:23482 delete_count:22 deleted_byte_count:18810492 replica_placement:100 version:3 modified_at_second:1615541369
+ volume id:179 size:1074313920 collection:"collection1" file_count:21829 delete_count:24 deleted_byte_count:45574435 replica_placement:100 version:3 modified_at_second:1615580308
+ volume id:180 size:1078438448 collection:"collection1" file_count:23614 delete_count:12 deleted_byte_count:4496474 replica_placement:100 version:3 modified_at_second:1614773243
+ volume id:181 size:1074571672 collection:"collection1" file_count:22898 delete_count:19 deleted_byte_count:6628413 replica_placement:100 version:3 modified_at_second:1614745117
+ volume id:183 size:1076361616 collection:"collection1" file_count:31293 delete_count:16 deleted_byte_count:468841 replica_placement:100 version:3 modified_at_second:1615572985
+ volume id:184 size:1074594216 collection:"collection1" file_count:31368 delete_count:22 deleted_byte_count:857453 replica_placement:100 version:3 modified_at_second:1615586578
+ volume id:185 size:1074099592 collection:"collection1" file_count:30612 delete_count:17 deleted_byte_count:2610847 replica_placement:100 version:3 modified_at_second:1615506835
+ volume id:186 size:1074220664 collection:"collection1" file_count:31450 delete_count:15 deleted_byte_count:391855 replica_placement:100 version:3 modified_at_second:1615614934
+ volume id:187 size:1074396112 collection:"collection1" file_count:31853 delete_count:17 deleted_byte_count:454283 replica_placement:100 version:3 modified_at_second:1615590491
+ volume id:188 size:1074732632 collection:"collection1" file_count:31867 delete_count:19 deleted_byte_count:393743 replica_placement:100 version:3 modified_at_second:1615487645
+ volume id:189 size:1074847824 collection:"collection1" file_count:31450 delete_count:16 deleted_byte_count:1040552 replica_placement:100 version:3 modified_at_second:1615335661
+ volume id:190 size:1074008968 collection:"collection1" file_count:31987 delete_count:11 deleted_byte_count:685125 replica_placement:100 version:3 modified_at_second:1615447162
+ volume id:191 size:1075492960 collection:"collection1" file_count:31301 delete_count:19 deleted_byte_count:708401 replica_placement:100 version:3 modified_at_second:1615357457
+ volume id:192 size:1075857384 collection:"collection1" file_count:31490 delete_count:25 deleted_byte_count:720617 replica_placement:100 version:3 modified_at_second:1615621632
+ volume id:193 size:1076616760 collection:"collection1" file_count:31907 delete_count:16 deleted_byte_count:464900 replica_placement:100 version:3 modified_at_second:1615507877
+ volume id:194 size:1073985792 collection:"collection1" file_count:31434 delete_count:18 deleted_byte_count:391432 replica_placement:100 version:3 modified_at_second:1615559502
+ volume id:195 size:1074158304 collection:"collection1" file_count:31453 delete_count:15 deleted_byte_count:718266 replica_placement:100 version:3 modified_at_second:1615559331
+ volume id:196 size:1074594640 collection:"collection1" file_count:31665 delete_count:18 deleted_byte_count:3468922 replica_placement:100 version:3 modified_at_second:1615501690
+ volume id:198 size:1075104624 collection:"collection1" file_count:16577 delete_count:18 deleted_byte_count:6583181 replica_placement:100 version:3 modified_at_second:1615623371
+ volume id:199 size:1078117688 collection:"collection1" file_count:16497 delete_count:14 deleted_byte_count:1514286 replica_placement:100 version:3 modified_at_second:1615585987
+ volume id:200 size:1075630464 collection:"collection1" file_count:16380 delete_count:18 deleted_byte_count:1103109 replica_placement:100 version:3 modified_at_second:1615485252
+ volume id:201 size:1091460440 collection:"collection1" file_count:16684 delete_count:26 deleted_byte_count:5590335 replica_placement:100 version:3 modified_at_second:1615585987
+ volume id:204 size:1079766904 collection:"collection1" file_count:3233 delete_count:255 deleted_byte_count:104707641 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615565702
+ volume id:207 size:1081939960 collection:"collection1" file_count:3010 delete_count:4 deleted_byte_count:692350 replica_placement:100 version:3 modified_at_second:1615269061
+ volume id:208 size:1077863624 collection:"collection1" file_count:3147 delete_count:6 deleted_byte_count:858726 replica_placement:100 version:3 modified_at_second:1615495515
+ volume id:209 size:1074083592 collection:"collection1" file_count:3238 delete_count:4 deleted_byte_count:1494244 replica_placement:100 version:3 modified_at_second:1615419954
+ volume id:210 size:1094311304 collection:"collection1" file_count:3468 delete_count:4 deleted_byte_count:466433 replica_placement:100 version:3 modified_at_second:1615495515
+ volume id:211 size:1080610712 collection:"collection1" file_count:3247 delete_count:7 deleted_byte_count:1891456 replica_placement:100 version:3 modified_at_second:1615269124
+ volume id:216 size:1080073496 collection:"collection1" file_count:3316 delete_count:4 deleted_byte_count:179819 replica_placement:100 version:3 modified_at_second:1615586387
+ volume id:218 size:1081263944 collection:"collection1" file_count:3433 delete_count:14 deleted_byte_count:3454237 replica_placement:100 version:3 modified_at_second:1615603637
+ volume id:220 size:1081928312 collection:"collection1" file_count:3166 delete_count:13 deleted_byte_count:4127709 replica_placement:100 version:3 modified_at_second:1615579317
+ volume id:221 size:1106545536 collection:"collection1" file_count:3153 delete_count:11 deleted_byte_count:1496835 replica_placement:100 version:3 modified_at_second:1615269138
+ volume id:224 size:1093691520 collection:"collection1" file_count:3463 delete_count:10 deleted_byte_count:1128328 replica_placement:100 version:3 modified_at_second:1615601870
+ volume id:225 size:1080698928 collection:"collection1" file_count:3115 delete_count:7 deleted_byte_count:18170416 replica_placement:100 version:3 modified_at_second:1615434685
+ volume id:226 size:1103504792 collection:"collection1" file_count:2965 delete_count:10 deleted_byte_count:2639254 replica_placement:100 version:3 modified_at_second:1615601870
+ volume id:227 size:1106699864 collection:"collection1" file_count:2827 delete_count:19 deleted_byte_count:5393310 replica_placement:100 version:3 modified_at_second:1615609989
+ volume id:228 size:1109784072 collection:"collection1" file_count:2504 delete_count:24 deleted_byte_count:5458950 replica_placement:100 version:3 modified_at_second:1615610489
+ volume id:229 size:1109855256 collection:"collection1" file_count:2857 delete_count:22 deleted_byte_count:2839883 replica_placement:100 version:3 modified_at_second:1615609989
+ volume id:231 size:1112917664 collection:"collection1" file_count:3151 delete_count:19 deleted_byte_count:2852517 replica_placement:100 version:3 modified_at_second:1615611350
+ volume id:232 size:1073901520 collection:"collection1" file_count:3004 delete_count:54 deleted_byte_count:10273081 replica_placement:100 version:3 modified_at_second:1615611352
+ volume id:233 size:1080526464 collection:"collection1" file_count:3136 delete_count:61 deleted_byte_count:17991717 replica_placement:100 version:3 modified_at_second:1615611354
+ volume id:236 size:1089476200 collection:"collection1" file_count:3231 delete_count:53 deleted_byte_count:11625921 replica_placement:100 version:3 modified_at_second:1615611351
+ volume id:238 size:354320000 collection:"collection1" file_count:701 delete_count:17 deleted_byte_count:5940420 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615632030
+ volume id:240 size:424791528 collection:"collection1" file_count:734 delete_count:12 deleted_byte_count:7353071 replica_placement:100 version:3 modified_at_second:1615631669
+ volume id:242 size:1075383304 collection:"collection2" file_count:10470 replica_placement:100 version:3 modified_at_second:1614852115
+ volume id:243 size:1088174560 collection:"collection2" file_count:11109 delete_count:1 deleted_byte_count:938 replica_placement:100 version:3 modified_at_second:1614852202
+ volume id:245 size:1074597056 collection:"collection2" file_count:10371 delete_count:3 deleted_byte_count:209701 replica_placement:100 version:3 modified_at_second:1614852093
+ volume id:247 size:1075859784 collection:"collection2" file_count:10443 delete_count:2 deleted_byte_count:564486 replica_placement:100 version:3 modified_at_second:1614856152
+ volume id:249 size:1074819168 collection:"collection2" file_count:10763 delete_count:2 deleted_byte_count:271699 replica_placement:100 version:3 modified_at_second:1614856231
+ volume id:250 size:1080572256 collection:"collection2" file_count:10220 replica_placement:100 version:3 modified_at_second:1614856129
+ volume id:251 size:1075684408 collection:"collection2" file_count:10847 replica_placement:100 version:3 modified_at_second:1614856270
+ volume id:254 size:1074830800 collection:"collection2" file_count:14140 delete_count:2 deleted_byte_count:105892 replica_placement:100 version:3 modified_at_second:1614861115
+ volume id:257 size:1082621664 collection:"collection2" file_count:18172 delete_count:2 deleted_byte_count:25125 replica_placement:100 version:3 modified_at_second:1614866395
+ volume id:260 size:1075105664 collection:"collection2" file_count:17316 delete_count:4 deleted_byte_count:2015310 replica_placement:100 version:3 modified_at_second:1614866226
+ volume id:261 size:1076628592 collection:"collection2" file_count:18355 delete_count:1 deleted_byte_count:1155 replica_placement:100 version:3 modified_at_second:1614866420
+ volume id:262 size:1078492464 collection:"collection2" file_count:20390 delete_count:3 deleted_byte_count:287601 replica_placement:100 version:3 modified_at_second:1614871601
+ volume id:263 size:1077167440 collection:"collection2" file_count:20227 delete_count:4 deleted_byte_count:97887 replica_placement:100 version:3 modified_at_second:1614871567
+ volume id:268 size:1074490592 collection:"collection2" file_count:21698 delete_count:1 deleted_byte_count:33968 replica_placement:100 version:3 modified_at_second:1614877435
+ volume id:269 size:1077552720 collection:"collection2" file_count:21875 delete_count:4 deleted_byte_count:347272 replica_placement:100 version:3 modified_at_second:1614877481
+ volume id:271 size:1076992648 collection:"collection2" file_count:22640 delete_count:1 deleted_byte_count:30645 replica_placement:100 version:3 modified_at_second:1614877504
+ volume id:273 size:1074873432 collection:"collection2" file_count:20511 delete_count:3 deleted_byte_count:46076 replica_placement:100 version:3 modified_at_second:1614884046
+ volume id:274 size:1075994128 collection:"collection2" file_count:20997 replica_placement:100 version:3 modified_at_second:1614884113
+ volume id:276 size:1076899888 collection:"collection2" file_count:20190 delete_count:1 deleted_byte_count:8798 replica_placement:100 version:3 modified_at_second:1614884003
+ volume id:277 size:1074956160 collection:"collection2" file_count:19260 delete_count:2 deleted_byte_count:172356 replica_placement:100 version:3 modified_at_second:1614889988
+ volume id:279 size:1077325096 collection:"collection2" file_count:19671 delete_count:6 deleted_byte_count:379116 replica_placement:100 version:3 modified_at_second:1614890230
+ volume id:282 size:1075232240 collection:"collection2" file_count:22659 delete_count:4 deleted_byte_count:67915 replica_placement:100 version:3 modified_at_second:1614897304
+ volume id:284 size:1074533384 collection:"collection2" file_count:22196 delete_count:4 deleted_byte_count:154683 replica_placement:100 version:3 modified_at_second:1614897231
+ volume id:285 size:1082128576 collection:"collection2" file_count:21804 delete_count:1 deleted_byte_count:1064 replica_placement:100 version:3 modified_at_second:1614897165
+ volume id:286 size:1077464824 collection:"collection2" file_count:23905 delete_count:6 deleted_byte_count:630577 replica_placement:100 version:3 modified_at_second:1614897401
+ volume id:287 size:1074590528 collection:"collection2" file_count:28163 delete_count:5 deleted_byte_count:35727 replica_placement:100 version:3 modified_at_second:1614904874
+ volume id:288 size:1075406800 collection:"collection2" file_count:27243 delete_count:2 deleted_byte_count:51519 replica_placement:100 version:3 modified_at_second:1614904738
+ volume id:292 size:1092010744 collection:"collection2" file_count:26781 delete_count:5 deleted_byte_count:508910 replica_placement:100 version:3 modified_at_second:1614912327
+ volume id:293 size:1075409776 collection:"collection2" file_count:26063 delete_count:4 deleted_byte_count:183834 replica_placement:100 version:3 modified_at_second:1614912235
+ volume id:294 size:1075443992 collection:"collection2" file_count:26076 delete_count:4 deleted_byte_count:194914 replica_placement:100 version:3 modified_at_second:1614912220
+ volume id:295 size:1074702376 collection:"collection2" file_count:24488 delete_count:3 deleted_byte_count:48555 replica_placement:100 version:3 modified_at_second:1614911929
+ volume id:300 size:1076212424 collection:"collection2" file_count:22892 delete_count:2 deleted_byte_count:61320 replica_placement:100 version:3 modified_at_second:1614918464
+ volume id:304 size:1081038888 collection:"collection2" file_count:24505 delete_count:2 deleted_byte_count:124447 replica_placement:100 version:3 modified_at_second:1614925567
+ volume id:305 size:1074185552 collection:"collection2" file_count:22074 delete_count:5 deleted_byte_count:20221 replica_placement:100 version:3 modified_at_second:1614925312
+ volume id:310 size:1074761520 collection:"collection2" file_count:21441 delete_count:3 deleted_byte_count:13934 replica_placement:100 version:3 modified_at_second:1614931077
+ volume id:311 size:1088248208 collection:"collection2" file_count:23553 delete_count:6 deleted_byte_count:191716 replica_placement:100 version:3 modified_at_second:1614931460
+ volume id:312 size:1075037808 collection:"collection2" file_count:22524 replica_placement:100 version:3 modified_at_second:1614937832
+ volume id:313 size:1074876016 collection:"collection2" file_count:22404 delete_count:4 deleted_byte_count:51728 replica_placement:100 version:3 modified_at_second:1614937755
+ volume id:314 size:1074670840 collection:"collection2" file_count:20964 delete_count:4 deleted_byte_count:304291 replica_placement:100 version:3 modified_at_second:1614937441
+ volume id:315 size:1084153456 collection:"collection2" file_count:23638 delete_count:2 deleted_byte_count:53956 replica_placement:100 version:3 modified_at_second:1614937884
+ volume id:316 size:1077720784 collection:"collection2" file_count:22605 delete_count:1 deleted_byte_count:8503 replica_placement:100 version:3 modified_at_second:1614937838
+ volume id:317 size:1076215040 collection:"collection2" file_count:23572 delete_count:2 deleted_byte_count:1441356 replica_placement:100 version:3 modified_at_second:1614943965
+ volume id:319 size:1073952744 collection:"collection2" file_count:22286 delete_count:2 deleted_byte_count:43421 replica_placement:100 version:3 modified_at_second:1614943810
+ volume id:320 size:1082437736 collection:"collection2" file_count:21544 delete_count:3 deleted_byte_count:16712 replica_placement:100 version:3 modified_at_second:1614943591
+ volume id:321 size:1081477960 collection:"collection2" file_count:23531 delete_count:5 deleted_byte_count:262564 replica_placement:100 version:3 modified_at_second:1614943982
+ volume id:322 size:1078471600 collection:"collection2" file_count:21905 delete_count:3 deleted_byte_count:145002 replica_placement:100 version:3 modified_at_second:1614950574
+ volume id:324 size:1075606712 collection:"collection2" file_count:20799 delete_count:1 deleted_byte_count:251210 replica_placement:100 version:3 modified_at_second:1614950310
+ volume id:326 size:1076059936 collection:"collection2" file_count:22564 delete_count:2 deleted_byte_count:192886 replica_placement:100 version:3 modified_at_second:1614950619
+ volume id:327 size:1076121224 collection:"collection2" file_count:22007 delete_count:3 deleted_byte_count:60358 replica_placement:100 version:3 modified_at_second:1614956487
+ volume id:328 size:1074767928 collection:"collection2" file_count:21720 delete_count:3 deleted_byte_count:56429 replica_placement:100 version:3 modified_at_second:1614956362
+ volume id:329 size:1076691776 collection:"collection2" file_count:22411 delete_count:5 deleted_byte_count:214092 replica_placement:100 version:3 modified_at_second:1614956485
+ volume id:331 size:1074957192 collection:"collection2" file_count:21230 delete_count:4 deleted_byte_count:62145 replica_placement:100 version:3 modified_at_second:1614956259
+ volume id:333 size:1074270192 collection:"collection2" file_count:21271 delete_count:2 deleted_byte_count:168122 replica_placement:100 version:3 modified_at_second:1614962697
+ volume id:335 size:1076235176 collection:"collection2" file_count:22391 delete_count:3 deleted_byte_count:8838 replica_placement:100 version:3 modified_at_second:1614962970
+ volume id:336 size:1087853032 collection:"collection2" file_count:22801 delete_count:2 deleted_byte_count:26394 replica_placement:100 version:3 modified_at_second:1614963003
+ volume id:338 size:1076118360 collection:"collection2" file_count:21680 replica_placement:100 version:3 modified_at_second:1614969850
+ volume id:342 size:1080186296 collection:"collection2" file_count:22405 delete_count:4 deleted_byte_count:64819 replica_placement:100 version:3 modified_at_second:1614977518
+ volume id:343 size:1075345184 collection:"collection2" file_count:21095 delete_count:2 deleted_byte_count:20581 replica_placement:100 version:3 modified_at_second:1614977148
+ volume id:349 size:1075957824 collection:"collection2" file_count:22395 delete_count:2 deleted_byte_count:61565 replica_placement:100 version:3 modified_at_second:1614984748
+ volume id:350 size:1074756688 collection:"collection2" file_count:21990 delete_count:3 deleted_byte_count:233881 replica_placement:100 version:3 modified_at_second:1614984682
+ volume id:354 size:1085213992 collection:"collection2" file_count:23150 delete_count:4 deleted_byte_count:82391 replica_placement:100 version:3 modified_at_second:1614992207
+ volume id:356 size:1083304992 collection:"collection2" file_count:21552 delete_count:4 deleted_byte_count:14472 replica_placement:100 version:3 modified_at_second:1614992027
+ volume id:358 size:1085152312 collection:"collection2" file_count:23756 delete_count:3 deleted_byte_count:44531 replica_placement:100 version:3 modified_at_second:1614998824
+ volume id:359 size:1074211240 collection:"collection2" file_count:22437 delete_count:2 deleted_byte_count:187953 replica_placement:100 version:3 modified_at_second:1614998711
+ volume id:362 size:1074074120 collection:"collection2" file_count:20595 delete_count:1 deleted_byte_count:112145 replica_placement:100 version:3 modified_at_second:1615004407
+ volume id:363 size:1078859496 collection:"collection2" file_count:23177 delete_count:4 deleted_byte_count:9601 replica_placement:100 version:3 modified_at_second:1615004822
+ volume id:364 size:1081280816 collection:"collection2" file_count:22686 delete_count:1 deleted_byte_count:84375 replica_placement:100 version:3 modified_at_second:1615004813
+ volume id:365 size:1075736632 collection:"collection2" file_count:22193 delete_count:5 deleted_byte_count:259033 replica_placement:100 version:3 modified_at_second:1615004776
+ volume id:366 size:1075267272 collection:"collection2" file_count:21856 delete_count:5 deleted_byte_count:138363 replica_placement:100 version:3 modified_at_second:1615004703
+ volume id:367 size:1076403648 collection:"collection2" file_count:22995 delete_count:2 deleted_byte_count:36955 replica_placement:100 version:3 modified_at_second:1615010985
+ volume id:368 size:1074822016 collection:"collection2" file_count:22252 delete_count:4 deleted_byte_count:3291946 replica_placement:100 version:3 modified_at_second:1615010878
+ volume id:369 size:1091472040 collection:"collection2" file_count:23709 delete_count:4 deleted_byte_count:400876 replica_placement:100 version:3 modified_at_second:1615011019
+ volume id:370 size:1076040480 collection:"collection2" file_count:22092 delete_count:2 deleted_byte_count:115388 replica_placement:100 version:3 modified_at_second:1615010877
+ volume id:371 size:1078806160 collection:"collection2" file_count:22685 delete_count:2 deleted_byte_count:68905 replica_placement:100 version:3 modified_at_second:1615010994
+ volume id:372 size:1076193312 collection:"collection2" file_count:22774 delete_count:1 deleted_byte_count:3495 replica_placement:100 version:3 modified_at_second:1615016911
+ volume id:374 size:1085011080 collection:"collection2" file_count:23054 delete_count:2 deleted_byte_count:89034 replica_placement:100 version:3 modified_at_second:1615016917
+ volume id:375 size:1076140688 collection:"collection2" file_count:21880 delete_count:2 deleted_byte_count:51103 replica_placement:100 version:3 modified_at_second:1615016787
+ volume id:378 size:959273824 collection:"collection2" file_count:15031 replica_placement:100 version:3 modified_at_second:1615632323
+ volume id:379 size:1014108592 collection:"collection2" file_count:15360 delete_count:8 deleted_byte_count:2524591 replica_placement:100 version:3 modified_at_second:1615632323
+ volume id:380 size:1010760464 collection:"collection2" file_count:14920 delete_count:7 deleted_byte_count:134859 replica_placement:100 version:3 modified_at_second:1615632322
+ Disk hdd total size:274627838960 file_count:3607097 deleted_file:13594 deleted_bytes:4185524457
+ DataNode 192.168.1.1:8080 total size:274627838960 file_count:3607097 deleted_file:13594 deleted_bytes:4185524457
+ Rack DefaultRack total size:274627838960 file_count:3607097 deleted_file:13594 deleted_bytes:4185524457
+ DataCenter dc4 total size:274627838960 file_count:3607097 deleted_file:13594 deleted_bytes:4185524457
+ DataCenter dc5 hdd(volume:299/3000 active:299 free:2701 remote:0)
+ Rack DefaultRack hdd(volume:299/3000 active:299 free:2701 remote:0)
+ DataNode 192.168.1.5:8080 hdd(volume:299/3000 active:299 free:2701 remote:0)
+ Disk hdd(volume:299/3000 active:299 free:2701 remote:0)
+ volume id:5 size:293806008 file_count:1669 delete_count:2 deleted_byte_count:274334 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631342
+ volume id:11 size:1109224552 collection:"collection0" file_count:44 replica_placement:100 version:3 modified_at_second:1615629606
+ volume id:18 size:1096678688 collection:"collection0" file_count:88 delete_count:4 deleted_byte_count:8633 replica_placement:100 version:3 modified_at_second:1615631673
+ volume id:19 size:1096923792 collection:"collection0" file_count:100 delete_count:10 deleted_byte_count:75779917 replica_placement:100 version:3 compact_revision:4 modified_at_second:1615630117
+ volume id:20 size:1074760432 collection:"collection0" file_count:82 delete_count:5 deleted_byte_count:12156 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615629340
+ volume id:26 size:532657632 collection:"collection0" file_count:100 delete_count:4 deleted_byte_count:9081 replica_placement:100 version:3 modified_at_second:1614170024
+ volume id:27 size:298886792 file_count:1608 replica_placement:100 version:3 modified_at_second:1615632482
+ volume id:28 size:308919192 file_count:1591 delete_count:1 deleted_byte_count:125280 replica_placement:100 version:3 modified_at_second:1615631762
+ volume id:29 size:281582688 file_count:1537 replica_placement:100 version:3 modified_at_second:1615629422
+ volume id:30 size:289466144 file_count:1566 delete_count:1 deleted_byte_count:124972 replica_placement:100 version:3 modified_at_second:1615632422
+ volume id:31 size:273363256 file_count:1498 replica_placement:100 version:3 modified_at_second:1615631642
+ volume id:32 size:281343360 file_count:1497 replica_placement:100 version:3 modified_at_second:1615632362
+ volume id:33 size:1130226344 collection:"collection1" file_count:7322 delete_count:172 deleted_byte_count:45199399 replica_placement:100 version:3 modified_at_second:1615618789
+ volume id:34 size:1077111136 collection:"collection1" file_count:9781 delete_count:110 deleted_byte_count:20894827 replica_placement:100 version:3 modified_at_second:1615619366
+ volume id:35 size:1075241744 collection:"collection1" file_count:10523 delete_count:97 deleted_byte_count:46586217 replica_placement:100 version:3 modified_at_second:1615618790
+ volume id:36 size:1075118336 collection:"collection1" file_count:10341 delete_count:118 deleted_byte_count:24753278 replica_placement:100 version:3 modified_at_second:1615606148
+ volume id:37 size:1075895576 collection:"collection1" file_count:12013 delete_count:98 deleted_byte_count:50747932 replica_placement:100 version:3 modified_at_second:1615594776
+ volume id:38 size:1075545744 collection:"collection1" file_count:13324 delete_count:100 deleted_byte_count:25223906 replica_placement:100 version:3 modified_at_second:1615569830
+ volume id:39 size:1076606536 collection:"collection1" file_count:12612 delete_count:78 deleted_byte_count:17462730 replica_placement:100 version:3 modified_at_second:1615611959
+ volume id:40 size:1075358552 collection:"collection1" file_count:12597 delete_count:62 deleted_byte_count:11657901 replica_placement:100 version:3 modified_at_second:1615612994
+ volume id:41 size:1076283592 collection:"collection1" file_count:12088 delete_count:84 deleted_byte_count:19311237 replica_placement:100 version:3 modified_at_second:1615596736
+ volume id:42 size:1093948352 collection:"collection1" file_count:7889 delete_count:47 deleted_byte_count:5697275 replica_placement:100 version:3 modified_at_second:1615548906
+ volume id:43 size:1116445864 collection:"collection1" file_count:7355 delete_count:57 deleted_byte_count:9727158 replica_placement:100 version:3 modified_at_second:1615566167
+ volume id:44 size:1077582560 collection:"collection1" file_count:7295 delete_count:50 deleted_byte_count:12618414 replica_placement:100 version:3 modified_at_second:1615566170
+ volume id:45 size:1075254640 collection:"collection1" file_count:10772 delete_count:76 deleted_byte_count:22426345 replica_placement:100 version:3 modified_at_second:1615573498
+ volume id:46 size:1075286056 collection:"collection1" file_count:9947 delete_count:309 deleted_byte_count:105601163 replica_placement:100 version:3 modified_at_second:1615569825
+ volume id:47 size:444599784 collection:"collection1" file_count:709 delete_count:19 deleted_byte_count:11913451 replica_placement:100 version:3 modified_at_second:1615632397
+ volume id:48 size:1076778664 collection:"collection1" file_count:9850 delete_count:77 deleted_byte_count:16641907 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630690
+ volume id:49 size:1078775288 collection:"collection1" file_count:9631 delete_count:27 deleted_byte_count:5985628 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615575823
+ volume id:50 size:1076688288 collection:"collection1" file_count:7921 delete_count:26 deleted_byte_count:5162032 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615610876
+ volume id:51 size:1076796120 collection:"collection1" file_count:10550 delete_count:39 deleted_byte_count:12723654 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615547786
+ volume id:53 size:1063089216 collection:"collection1" file_count:9832 delete_count:31 deleted_byte_count:9273066 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632156
+ volume id:54 size:1045022288 collection:"collection1" file_count:9409 delete_count:29 deleted_byte_count:15102818 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630813
+ volume id:55 size:1012890016 collection:"collection1" file_count:8651 delete_count:27 deleted_byte_count:9418841 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631453
+ volume id:56 size:1002412240 collection:"collection1" file_count:8762 delete_count:40 deleted_byte_count:65885831 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615632014
+ volume id:57 size:839849792 collection:"collection1" file_count:7514 delete_count:24 deleted_byte_count:6228543 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631775
+ volume id:58 size:908064192 collection:"collection1" file_count:8128 delete_count:21 deleted_byte_count:6113731 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632343
+ volume id:59 size:988302272 collection:"collection1" file_count:8098 delete_count:20 deleted_byte_count:3947615 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632238
+ volume id:60 size:1010702480 collection:"collection1" file_count:8969 delete_count:79 deleted_byte_count:24782814 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615632439
+ volume id:61 size:975604544 collection:"collection1" file_count:8683 delete_count:20 deleted_byte_count:10276072 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615631176
+ volume id:62 size:873845904 collection:"collection1" file_count:7897 delete_count:23 deleted_byte_count:10920170 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631133
+ volume id:63 size:956941176 collection:"collection1" file_count:8271 delete_count:32 deleted_byte_count:15876189 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632036
+ volume id:64 size:965638424 collection:"collection1" file_count:8218 delete_count:27 deleted_byte_count:6922489 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631032
+ volume id:65 size:823283608 collection:"collection1" file_count:7834 delete_count:29 deleted_byte_count:5950610 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632307
+ volume id:66 size:821343440 collection:"collection1" file_count:7383 delete_count:29 deleted_byte_count:12010343 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631968
+ volume id:67 size:878713880 collection:"collection1" file_count:7299 delete_count:117 deleted_byte_count:24857326 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632156
+ volume id:69 size:863913896 collection:"collection1" file_count:7291 delete_count:100 deleted_byte_count:25335024 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630534
+ volume id:70 size:886695472 collection:"collection1" file_count:7769 delete_count:164 deleted_byte_count:45162513 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632398
+ volume id:71 size:907608392 collection:"collection1" file_count:7658 delete_count:122 deleted_byte_count:27622941 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632307
+ volume id:72 size:903990720 collection:"collection1" file_count:6996 delete_count:240 deleted_byte_count:74147727 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630985
+ volume id:73 size:929047544 collection:"collection1" file_count:7038 delete_count:227 deleted_byte_count:65336664 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630707
+ volume id:75 size:908045000 collection:"collection1" file_count:6911 delete_count:268 deleted_byte_count:73934373 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632430
+ volume id:76 size:985296744 collection:"collection1" file_count:6566 delete_count:61 deleted_byte_count:44464430 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632284
+ volume id:77 size:929398296 collection:"collection1" file_count:7427 delete_count:238 deleted_byte_count:59581579 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632014
+ volume id:78 size:1075671512 collection:"collection1" file_count:7540 delete_count:258 deleted_byte_count:71726846 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615582829
+ volume id:79 size:948225472 collection:"collection1" file_count:6997 delete_count:227 deleted_byte_count:60625763 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631326
+ volume id:80 size:827912928 collection:"collection1" file_count:6916 delete_count:15 deleted_byte_count:5611604 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631159
+ volume id:81 size:880693168 collection:"collection1" file_count:7481 delete_count:238 deleted_byte_count:80880878 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631395
+ volume id:82 size:1041660512 collection:"collection1" file_count:7043 delete_count:207 deleted_byte_count:52275724 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632430
+ volume id:83 size:936194288 collection:"collection1" file_count:7593 delete_count:13 deleted_byte_count:4633917 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632029
+ volume id:84 size:871262320 collection:"collection1" file_count:8190 delete_count:14 deleted_byte_count:3150948 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631161
+ volume id:86 size:1009434632 collection:"collection1" file_count:8474 delete_count:236 deleted_byte_count:64543674 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630812
+ volume id:87 size:922274624 collection:"collection1" file_count:12902 delete_count:13 deleted_byte_count:3412959 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615632438
+ volume id:88 size:1073767976 collection:"collection1" file_count:14994 delete_count:207 deleted_byte_count:82380696 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615541383
+ volume id:89 size:1044421824 collection:"collection1" file_count:14943 delete_count:243 deleted_byte_count:58543159 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615632208
+ volume id:90 size:891163760 collection:"collection1" file_count:14608 delete_count:10 deleted_byte_count:2564369 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615629392
+ volume id:91 size:936573952 collection:"collection1" file_count:14686 delete_count:11 deleted_byte_count:4717727 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631851
+ volume id:92 size:992439144 collection:"collection1" file_count:7061 delete_count:195 deleted_byte_count:60649573 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630566
+ volume id:93 size:1079602592 collection:"collection1" file_count:7878 delete_count:270 deleted_byte_count:74150048 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615556013
+ volume id:94 size:1030684704 collection:"collection1" file_count:7660 delete_count:207 deleted_byte_count:70150733 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631616
+ volume id:95 size:990877824 collection:"collection1" file_count:6620 delete_count:206 deleted_byte_count:60363604 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615631867
+ volume id:96 size:989294848 collection:"collection1" file_count:7544 delete_count:229 deleted_byte_count:59931853 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630778
+ volume id:98 size:1077836472 collection:"collection1" file_count:7605 delete_count:202 deleted_byte_count:73180379 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615523691
+ volume id:99 size:1071718496 collection:"collection1" file_count:7470 delete_count:8 deleted_byte_count:9624950 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615631175
+ volume id:100 size:1083617472 collection:"collection1" file_count:7018 delete_count:187 deleted_byte_count:61304236 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615505914
+ volume id:101 size:1077109408 collection:"collection1" file_count:7706 delete_count:226 deleted_byte_count:77864780 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615630994
+ volume id:102 size:1074359920 collection:"collection1" file_count:7338 delete_count:7 deleted_byte_count:6499151 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615626682
+ volume id:103 size:1075863904 collection:"collection1" file_count:7184 delete_count:186 deleted_byte_count:58872238 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615628417
+ volume id:106 size:1075458680 collection:"collection1" file_count:7182 delete_count:307 deleted_byte_count:69349053 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615598137
+ volume id:107 size:1073811776 collection:"collection1" file_count:7436 delete_count:168 deleted_byte_count:57747428 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615293569
+ volume id:108 size:1074648024 collection:"collection1" file_count:7472 delete_count:194 deleted_byte_count:70864699 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615593231
+ volume id:109 size:1075254560 collection:"collection1" file_count:7556 delete_count:263 deleted_byte_count:55155265 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615502487
+ volume id:110 size:1076575744 collection:"collection1" file_count:6996 delete_count:163 deleted_byte_count:52954032 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615590786
+ volume id:111 size:1073826176 collection:"collection1" file_count:7355 delete_count:155 deleted_byte_count:50083578 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615593232
+ volume id:112 size:1076392512 collection:"collection1" file_count:8291 delete_count:156 deleted_byte_count:74120183 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615569823
+ volume id:113 size:1076709184 collection:"collection1" file_count:9355 delete_count:177 deleted_byte_count:59796765 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615569822
+ volume id:114 size:1074762792 collection:"collection1" file_count:8802 delete_count:156 deleted_byte_count:38470055 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615591826
+ volume id:115 size:1076192296 collection:"collection1" file_count:7690 delete_count:154 deleted_byte_count:32267193 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615285296
+ volume id:117 size:1073917192 collection:"collection1" file_count:9520 delete_count:114 deleted_byte_count:21835126 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615573712
+ volume id:118 size:1074064344 collection:"collection1" file_count:8738 delete_count:15 deleted_byte_count:3460697 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615516264
+ volume id:120 size:1076115928 collection:"collection1" file_count:9639 delete_count:118 deleted_byte_count:33357871 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615482567
+ volume id:121 size:1078803320 collection:"collection1" file_count:10113 delete_count:441 deleted_byte_count:94128627 replica_placement:100 version:3 modified_at_second:1615506626
+ volume id:122 size:1076235312 collection:"collection1" file_count:9106 delete_count:252 deleted_byte_count:93041272 replica_placement:100 version:3 modified_at_second:1615585912
+ volume id:123 size:1080491112 collection:"collection1" file_count:10623 delete_count:302 deleted_byte_count:83956998 replica_placement:100 version:3 modified_at_second:1615585916
+ volume id:124 size:1074519360 collection:"collection1" file_count:9457 delete_count:286 deleted_byte_count:74752459 replica_placement:100 version:3 modified_at_second:1615585913
+ volume id:125 size:1088687040 collection:"collection1" file_count:9518 delete_count:281 deleted_byte_count:76037905 replica_placement:100 version:3 modified_at_second:1615585913
+ volume id:126 size:1073867408 collection:"collection1" file_count:9320 delete_count:278 deleted_byte_count:94547424 replica_placement:100 version:3 modified_at_second:1615585911
+ volume id:127 size:1074907336 collection:"collection1" file_count:9900 delete_count:133 deleted_byte_count:48570820 replica_placement:100 version:3 modified_at_second:1615612990
+ volume id:128 size:1074874632 collection:"collection1" file_count:9821 delete_count:148 deleted_byte_count:43633334 replica_placement:100 version:3 modified_at_second:1615602670
+ volume id:129 size:1074704328 collection:"collection1" file_count:10012 delete_count:150 deleted_byte_count:64491721 replica_placement:100 version:3 modified_at_second:1615627566
+ volume id:130 size:1075000632 collection:"collection1" file_count:10633 delete_count:161 deleted_byte_count:34768201 replica_placement:100 version:3 modified_at_second:1615582327
+ volume id:131 size:1075279584 collection:"collection1" file_count:10075 delete_count:135 deleted_byte_count:29795712 replica_placement:100 version:3 modified_at_second:1615523898
+ volume id:132 size:1088539552 collection:"collection1" file_count:11051 delete_count:71 deleted_byte_count:17178322 replica_placement:100 version:3 modified_at_second:1615619581
+ volume id:134 size:1074367304 collection:"collection1" file_count:10662 delete_count:69 deleted_byte_count:25530139 replica_placement:100 version:3 modified_at_second:1615555873
+ volume id:135 size:1073906776 collection:"collection1" file_count:10446 delete_count:71 deleted_byte_count:28599975 replica_placement:100 version:3 modified_at_second:1615569816
+ volume id:136 size:1074433552 collection:"collection1" file_count:9593 delete_count:72 deleted_byte_count:26912512 replica_placement:100 version:3 modified_at_second:1615376036
+ volume id:137 size:1074309264 collection:"collection1" file_count:9633 delete_count:50 deleted_byte_count:27487972 replica_placement:100 version:3 modified_at_second:1615572231
+ volume id:138 size:1074465744 collection:"collection1" file_count:10120 delete_count:55 deleted_byte_count:15875438 replica_placement:100 version:3 modified_at_second:1615572231
+ volume id:140 size:1076203744 collection:"collection1" file_count:11219 delete_count:57 deleted_byte_count:19864498 replica_placement:100 version:3 modified_at_second:1615571947
+ volume id:141 size:1074619488 collection:"collection1" file_count:9840 delete_count:45 deleted_byte_count:40890181 replica_placement:100 version:3 modified_at_second:1615630994
+ volume id:142 size:1075733064 collection:"collection1" file_count:9009 delete_count:48 deleted_byte_count:9912854 replica_placement:100 version:3 modified_at_second:1615598913
+ volume id:143 size:1075011280 collection:"collection1" file_count:9608 delete_count:51 deleted_byte_count:37282460 replica_placement:100 version:3 modified_at_second:1615488584
+ volume id:144 size:1074549720 collection:"collection1" file_count:8780 delete_count:50 deleted_byte_count:52475146 replica_placement:100 version:3 modified_at_second:1615573451
+ volume id:145 size:1074394928 collection:"collection1" file_count:9255 delete_count:34 deleted_byte_count:38011392 replica_placement:100 version:3 modified_at_second:1615591825
+ volume id:146 size:1076337576 collection:"collection1" file_count:10492 delete_count:50 deleted_byte_count:17071505 replica_placement:100 version:3 modified_at_second:1615632005
+ volume id:147 size:1077130576 collection:"collection1" file_count:10451 delete_count:27 deleted_byte_count:8290907 replica_placement:100 version:3 modified_at_second:1615604115
+ volume id:148 size:1076066568 collection:"collection1" file_count:9547 delete_count:33 deleted_byte_count:7034089 replica_placement:100 version:3 modified_at_second:1615586390
+ volume id:149 size:1074989016 collection:"collection1" file_count:8352 delete_count:35 deleted_byte_count:7179742 replica_placement:100 version:3 modified_at_second:1615494494
+ volume id:150 size:1076290328 collection:"collection1" file_count:9328 delete_count:33 deleted_byte_count:43417791 replica_placement:100 version:3 modified_at_second:1615611567
+ volume id:152 size:1075941400 collection:"collection1" file_count:9951 delete_count:36 deleted_byte_count:25348335 replica_placement:100 version:3 modified_at_second:1615606614
+ volume id:153 size:1078539784 collection:"collection1" file_count:10924 delete_count:34 deleted_byte_count:12603081 replica_placement:100 version:3 modified_at_second:1615606614
+ volume id:154 size:1081244696 collection:"collection1" file_count:11002 delete_count:31 deleted_byte_count:8467560 replica_placement:100 version:3 modified_at_second:1615478469
+ volume id:155 size:1075140688 collection:"collection1" file_count:10882 delete_count:32 deleted_byte_count:10076804 replica_placement:100 version:3 modified_at_second:1615606614
+ volume id:156 size:1074975832 collection:"collection1" file_count:9535 delete_count:40 deleted_byte_count:11426621 replica_placement:100 version:3 modified_at_second:1615628341
+ volume id:157 size:1076758536 collection:"collection1" file_count:10012 delete_count:19 deleted_byte_count:11688737 replica_placement:100 version:3 modified_at_second:1615597782
+ volume id:158 size:1087251976 collection:"collection1" file_count:9972 delete_count:20 deleted_byte_count:10328429 replica_placement:100 version:3 modified_at_second:1615588879
+ volume id:159 size:1074132368 collection:"collection1" file_count:9382 delete_count:27 deleted_byte_count:11474574 replica_placement:100 version:3 modified_at_second:1615593593
+ volume id:160 size:1075680952 collection:"collection1" file_count:9772 delete_count:22 deleted_byte_count:4981968 replica_placement:100 version:3 modified_at_second:1615597780
+ volume id:162 size:1074286880 collection:"collection1" file_count:11220 delete_count:17 deleted_byte_count:1815547 replica_placement:100 version:3 modified_at_second:1615478126
+ volume id:163 size:1074457192 collection:"collection1" file_count:12524 delete_count:27 deleted_byte_count:6359619 replica_placement:100 version:3 modified_at_second:1615579313
+ volume id:164 size:1074261248 collection:"collection1" file_count:11922 delete_count:25 deleted_byte_count:2923288 replica_placement:100 version:3 modified_at_second:1615620084
+ volume id:165 size:1073891016 collection:"collection1" file_count:9152 delete_count:12 deleted_byte_count:19164659 replica_placement:100 version:3 modified_at_second:1615471907
+ volume id:166 size:1075637536 collection:"collection1" file_count:14211 delete_count:24 deleted_byte_count:20415490 replica_placement:100 version:3 modified_at_second:1615491019
+ volume id:168 size:1074718808 collection:"collection1" file_count:25702 delete_count:40 deleted_byte_count:4024775 replica_placement:100 version:3 modified_at_second:1615585664
+ volume id:169 size:1073863128 collection:"collection1" file_count:25248 delete_count:43 deleted_byte_count:3013817 replica_placement:100 version:3 modified_at_second:1615569832
+ volume id:170 size:1075747096 collection:"collection1" file_count:24596 delete_count:41 deleted_byte_count:3494711 replica_placement:100 version:3 modified_at_second:1615579204
+ volume id:171 size:1081881312 collection:"collection1" file_count:24215 delete_count:36 deleted_byte_count:3191335 replica_placement:100 version:3 modified_at_second:1615596485
+ volume id:172 size:1074787312 collection:"collection1" file_count:31236 delete_count:50 deleted_byte_count:3316482 replica_placement:100 version:3 modified_at_second:1615612385
+ volume id:173 size:1074154648 collection:"collection1" file_count:30884 delete_count:34 deleted_byte_count:2430948 replica_placement:100 version:3 modified_at_second:1615591904
+ volume id:175 size:1077742504 collection:"collection1" file_count:32353 delete_count:33 deleted_byte_count:1861403 replica_placement:100 version:3 modified_at_second:1615559515
+ volume id:176 size:1073854800 collection:"collection1" file_count:30582 delete_count:34 deleted_byte_count:7701976 replica_placement:100 version:3 modified_at_second:1615626169
+ volume id:177 size:1074120120 collection:"collection1" file_count:22293 delete_count:16 deleted_byte_count:3719562 replica_placement:100 version:3 modified_at_second:1615516891
+ volume id:178 size:1087560112 collection:"collection1" file_count:23482 delete_count:22 deleted_byte_count:18810492 replica_placement:100 version:3 modified_at_second:1615541369
+ volume id:180 size:1078438536 collection:"collection1" file_count:23614 delete_count:12 deleted_byte_count:4496474 replica_placement:100 version:3 modified_at_second:1614773242
+ volume id:181 size:1074571768 collection:"collection1" file_count:22898 delete_count:19 deleted_byte_count:6628413 replica_placement:100 version:3 modified_at_second:1614745116
+ volume id:182 size:1076131280 collection:"collection1" file_count:31987 delete_count:21 deleted_byte_count:1416142 replica_placement:100 version:3 modified_at_second:1615568922
+ volume id:183 size:1076361448 collection:"collection1" file_count:31293 delete_count:16 deleted_byte_count:468841 replica_placement:100 version:3 modified_at_second:1615572982
+ volume id:184 size:1074594160 collection:"collection1" file_count:31368 delete_count:22 deleted_byte_count:857453 replica_placement:100 version:3 modified_at_second:1615586578
+ volume id:185 size:1074099624 collection:"collection1" file_count:30612 delete_count:17 deleted_byte_count:2610847 replica_placement:100 version:3 modified_at_second:1615506832
+ volume id:186 size:1074220864 collection:"collection1" file_count:31450 delete_count:15 deleted_byte_count:391855 replica_placement:100 version:3 modified_at_second:1615614933
+ volume id:187 size:1074395944 collection:"collection1" file_count:31853 delete_count:17 deleted_byte_count:454283 replica_placement:100 version:3 modified_at_second:1615590490
+ volume id:188 size:1074732792 collection:"collection1" file_count:31867 delete_count:19 deleted_byte_count:393743 replica_placement:100 version:3 modified_at_second:1615487645
+ volume id:189 size:1074847896 collection:"collection1" file_count:31450 delete_count:16 deleted_byte_count:1040552 replica_placement:100 version:3 modified_at_second:1615335661
+ volume id:190 size:1074008912 collection:"collection1" file_count:31987 delete_count:11 deleted_byte_count:685125 replica_placement:100 version:3 modified_at_second:1615447161
+ volume id:191 size:1075493024 collection:"collection1" file_count:31301 delete_count:19 deleted_byte_count:708401 replica_placement:100 version:3 modified_at_second:1615357456
+ volume id:192 size:1075857400 collection:"collection1" file_count:31490 delete_count:25 deleted_byte_count:720617 replica_placement:100 version:3 modified_at_second:1615621632
+ volume id:193 size:1076616768 collection:"collection1" file_count:31907 delete_count:16 deleted_byte_count:464900 replica_placement:100 version:3 modified_at_second:1615507875
+ volume id:194 size:1073985624 collection:"collection1" file_count:31434 delete_count:18 deleted_byte_count:391432 replica_placement:100 version:3 modified_at_second:1615559499
+ volume id:195 size:1074158312 collection:"collection1" file_count:31453 delete_count:15 deleted_byte_count:718266 replica_placement:100 version:3 modified_at_second:1615559331
+ volume id:196 size:1074594784 collection:"collection1" file_count:31665 delete_count:18 deleted_byte_count:3468922 replica_placement:100 version:3 modified_at_second:1615501688
+ volume id:197 size:1075423296 collection:"collection1" file_count:16473 delete_count:15 deleted_byte_count:12552442 replica_placement:100 version:3 modified_at_second:1615485253
+ volume id:198 size:1075104712 collection:"collection1" file_count:16577 delete_count:18 deleted_byte_count:6583181 replica_placement:100 version:3 modified_at_second:1615623369
+ volume id:199 size:1078117688 collection:"collection1" file_count:16497 delete_count:14 deleted_byte_count:1514286 replica_placement:100 version:3 modified_at_second:1615585984
+ volume id:200 size:1075630536 collection:"collection1" file_count:16380 delete_count:18 deleted_byte_count:1103109 replica_placement:100 version:3 modified_at_second:1615485252
+ volume id:201 size:1091460440 collection:"collection1" file_count:16684 delete_count:26 deleted_byte_count:5590335 replica_placement:100 version:3 modified_at_second:1615585987
+ volume id:202 size:1077533160 collection:"collection1" file_count:2847 delete_count:67 deleted_byte_count:65172985 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615588497
+ volume id:203 size:1027316272 collection:"collection1" file_count:3040 delete_count:11 deleted_byte_count:3993230 replica_placement:100 version:3 compact_revision:3 modified_at_second:1615631728
+ volume id:204 size:1079766872 collection:"collection1" file_count:3233 delete_count:255 deleted_byte_count:104707641 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615565701
+ volume id:205 size:1078485304 collection:"collection1" file_count:2869 delete_count:43 deleted_byte_count:18290259 replica_placement:100 version:3 compact_revision:2 modified_at_second:1615579314
+ volume id:206 size:1082045848 collection:"collection1" file_count:2979 delete_count:225 deleted_byte_count:88220074 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630989
+ volume id:207 size:1081939960 collection:"collection1" file_count:3010 delete_count:4 deleted_byte_count:692350 replica_placement:100 version:3 modified_at_second:1615269061
+ volume id:208 size:1077863624 collection:"collection1" file_count:3147 delete_count:6 deleted_byte_count:858726 replica_placement:100 version:3 modified_at_second:1615495515
+ volume id:210 size:1094311304 collection:"collection1" file_count:3468 delete_count:4 deleted_byte_count:466433 replica_placement:100 version:3 modified_at_second:1615495515
+ volume id:212 size:1078293448 collection:"collection1" file_count:3106 delete_count:6 deleted_byte_count:2085755 replica_placement:100 version:3 modified_at_second:1615586387
+ volume id:213 size:1093588072 collection:"collection1" file_count:3681 delete_count:12 deleted_byte_count:3138791 replica_placement:100 version:3 modified_at_second:1615586387
+ volume id:214 size:1074486992 collection:"collection1" file_count:3217 delete_count:10 deleted_byte_count:6392871 replica_placement:100 version:3 modified_at_second:1615586383
+ volume id:215 size:1074798704 collection:"collection1" file_count:2819 delete_count:31 deleted_byte_count:10873569 replica_placement:100 version:3 modified_at_second:1615586386
+ volume id:217 size:1075381872 collection:"collection1" file_count:3331 delete_count:14 deleted_byte_count:2009141 replica_placement:100 version:3 modified_at_second:1615401638
+ volume id:218 size:1081263944 collection:"collection1" file_count:3433 delete_count:14 deleted_byte_count:3454237 replica_placement:100 version:3 modified_at_second:1615603637
+ volume id:219 size:1092298816 collection:"collection1" file_count:3193 delete_count:17 deleted_byte_count:2047576 replica_placement:100 version:3 modified_at_second:1615579316
+ volume id:220 size:1081928312 collection:"collection1" file_count:3166 delete_count:13 deleted_byte_count:4127709 replica_placement:100 version:3 modified_at_second:1615579317
+ volume id:221 size:1106545456 collection:"collection1" file_count:3153 delete_count:11 deleted_byte_count:1496835 replica_placement:100 version:3 modified_at_second:1615269138
+ volume id:222 size:1106623104 collection:"collection1" file_count:3273 delete_count:11 deleted_byte_count:2114627 replica_placement:100 version:3 modified_at_second:1615586243
+ volume id:223 size:1075233064 collection:"collection1" file_count:2966 delete_count:9 deleted_byte_count:744001 replica_placement:100 version:3 modified_at_second:1615586244
+ volume id:224 size:1093691520 collection:"collection1" file_count:3463 delete_count:10 deleted_byte_count:1128328 replica_placement:100 version:3 modified_at_second:1615601870
+ volume id:225 size:1080698928 collection:"collection1" file_count:3115 delete_count:7 deleted_byte_count:18170416 replica_placement:100 version:3 modified_at_second:1615434684
+ volume id:226 size:1103504768 collection:"collection1" file_count:2965 delete_count:10 deleted_byte_count:2639254 replica_placement:100 version:3 modified_at_second:1615601867
+ volume id:228 size:1109784072 collection:"collection1" file_count:2504 delete_count:24 deleted_byte_count:5458950 replica_placement:100 version:3 modified_at_second:1615610489
+ volume id:230 size:1080722984 collection:"collection1" file_count:2898 delete_count:15 deleted_byte_count:3929261 replica_placement:100 version:3 modified_at_second:1615610537
+ volume id:232 size:1073901520 collection:"collection1" file_count:3004 delete_count:54 deleted_byte_count:10273081 replica_placement:100 version:3 modified_at_second:1615611351
+ volume id:234 size:1073835280 collection:"collection1" file_count:2965 delete_count:41 deleted_byte_count:4960354 replica_placement:100 version:3 modified_at_second:1615611351
+ volume id:235 size:1075586104 collection:"collection1" file_count:2767 delete_count:33 deleted_byte_count:3216540 replica_placement:100 version:3 modified_at_second:1615611354
+ volume id:236 size:1089476136 collection:"collection1" file_count:3231 delete_count:53 deleted_byte_count:11625921 replica_placement:100 version:3 modified_at_second:1615611351
+ volume id:237 size:375722792 collection:"collection1" file_count:736 delete_count:16 deleted_byte_count:4464870 replica_placement:100 version:3 modified_at_second:1615631727
+ volume id:238 size:354320000 collection:"collection1" file_count:701 delete_count:17 deleted_byte_count:5940420 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615632030
+ volume id:239 size:426569024 collection:"collection1" file_count:693 delete_count:19 deleted_byte_count:13020783 replica_placement:100 version:3 compact_revision:1 modified_at_second:1615630841
+ volume id:240 size:424791528 collection:"collection1" file_count:733 delete_count:13 deleted_byte_count:7515220 replica_placement:100 version:3 modified_at_second:1615631670
+ volume id:241 size:380217424 collection:"collection1" file_count:633 delete_count:6 deleted_byte_count:1715768 replica_placement:100 version:3 modified_at_second:1615632006
+ volume id:242 size:1075383392 collection:"collection2" file_count:10470 replica_placement:100 version:3 modified_at_second:1614852116
+ volume id:243 size:1088174704 collection:"collection2" file_count:11109 delete_count:1 deleted_byte_count:938 replica_placement:100 version:3 modified_at_second:1614852203
+ volume id:244 size:1080295352 collection:"collection2" file_count:10812 delete_count:1 deleted_byte_count:795 replica_placement:100 version:3 modified_at_second:1615628825
+ volume id:246 size:1075998672 collection:"collection2" file_count:10365 delete_count:1 deleted_byte_count:13112 replica_placement:100 version:3 modified_at_second:1614852106
+ volume id:247 size:1075859808 collection:"collection2" file_count:10443 delete_count:2 deleted_byte_count:564486 replica_placement:100 version:3 modified_at_second:1614856152
+ volume id:248 size:1084301208 collection:"collection2" file_count:11217 delete_count:4 deleted_byte_count:746488 replica_placement:100 version:3 modified_at_second:1614856285
+ volume id:250 size:1080572168 collection:"collection2" file_count:10220 replica_placement:100 version:3 modified_at_second:1614856129
+ volume id:252 size:1075065264 collection:"collection2" file_count:14622 delete_count:2 deleted_byte_count:5228 replica_placement:100 version:3 modified_at_second:1614861200
+ volume id:253 size:1087328880 collection:"collection2" file_count:14920 delete_count:3 deleted_byte_count:522994 replica_placement:100 version:3 modified_at_second:1614861258
+ volume id:254 size:1074830736 collection:"collection2" file_count:14140 delete_count:2 deleted_byte_count:105892 replica_placement:100 version:3 modified_at_second:1614861115
+ volume id:255 size:1079581640 collection:"collection2" file_count:14877 delete_count:3 deleted_byte_count:101223 replica_placement:100 version:3 modified_at_second:1614861233
+ volume id:256 size:1074283592 collection:"collection2" file_count:14157 delete_count:1 deleted_byte_count:18156 replica_placement:100 version:3 modified_at_second:1614861100
+ volume id:257 size:1082621720 collection:"collection2" file_count:18172 delete_count:2 deleted_byte_count:25125 replica_placement:100 version:3 modified_at_second:1614866402
+ volume id:258 size:1075527216 collection:"collection2" file_count:18421 delete_count:4 deleted_byte_count:267833 replica_placement:100 version:3 modified_at_second:1614866420
+ volume id:259 size:1075507848 collection:"collection2" file_count:18079 delete_count:2 deleted_byte_count:71992 replica_placement:100 version:3 modified_at_second:1614866381
+ volume id:260 size:1075105664 collection:"collection2" file_count:17316 delete_count:4 deleted_byte_count:2015310 replica_placement:100 version:3 modified_at_second:1614866226
+ volume id:261 size:1076628592 collection:"collection2" file_count:18355 delete_count:1 deleted_byte_count:1155 replica_placement:100 version:3 modified_at_second:1614866420
+ volume id:262 size:1078492584 collection:"collection2" file_count:20390 delete_count:3 deleted_byte_count:287601 replica_placement:100 version:3 modified_at_second:1614871601
+ volume id:264 size:1081624192 collection:"collection2" file_count:21151 replica_placement:100 version:3 modified_at_second:1614871629
+ volume id:265 size:1076401104 collection:"collection2" file_count:19932 delete_count:2 deleted_byte_count:160823 replica_placement:100 version:3 modified_at_second:1614871543
+ volume id:266 size:1075617552 collection:"collection2" file_count:20075 delete_count:1 deleted_byte_count:1039 replica_placement:100 version:3 modified_at_second:1614871526
+ volume id:267 size:1075699376 collection:"collection2" file_count:21039 delete_count:3 deleted_byte_count:59956 replica_placement:100 version:3 modified_at_second:1614877294
+ volume id:270 size:1076876424 collection:"collection2" file_count:22057 delete_count:1 deleted_byte_count:43916 replica_placement:100 version:3 modified_at_second:1614877469
+ volume id:271 size:1076992704 collection:"collection2" file_count:22640 delete_count:1 deleted_byte_count:30645 replica_placement:100 version:3 modified_at_second:1614877504
+ volume id:272 size:1076145912 collection:"collection2" file_count:21034 delete_count:2 deleted_byte_count:216564 replica_placement:100 version:3 modified_at_second:1614884139
+ volume id:273 size:1074873432 collection:"collection2" file_count:20511 delete_count:3 deleted_byte_count:46076 replica_placement:100 version:3 modified_at_second:1614884046
+ volume id:274 size:1075994184 collection:"collection2" file_count:20997 replica_placement:100 version:3 modified_at_second:1614884113
+ volume id:275 size:1078349024 collection:"collection2" file_count:20808 delete_count:1 deleted_byte_count:1118 replica_placement:100 version:3 modified_at_second:1614884147
+ volume id:276 size:1076899880 collection:"collection2" file_count:20190 delete_count:1 deleted_byte_count:8798 replica_placement:100 version:3 modified_at_second:1614884003
+ volume id:278 size:1078798632 collection:"collection2" file_count:20597 delete_count:5 deleted_byte_count:400060 replica_placement:100 version:3 modified_at_second:1614890292
+ volume id:280 size:1077432160 collection:"collection2" file_count:20286 delete_count:1 deleted_byte_count:879 replica_placement:100 version:3 modified_at_second:1614890262
+ volume id:281 size:1077581064 collection:"collection2" file_count:20206 delete_count:3 deleted_byte_count:143964 replica_placement:100 version:3 modified_at_second:1614890237
+ volume id:282 size:1075232184 collection:"collection2" file_count:22659 delete_count:4 deleted_byte_count:67915 replica_placement:100 version:3 modified_at_second:1614897304
+ volume id:283 size:1080178880 collection:"collection2" file_count:19462 delete_count:7 deleted_byte_count:660407 replica_placement:100 version:3 modified_at_second:1614896623
+ volume id:286 size:1077464816 collection:"collection2" file_count:23905 delete_count:6 deleted_byte_count:630577 replica_placement:100 version:3 modified_at_second:1614897401
+ volume id:287 size:1074590536 collection:"collection2" file_count:28163 delete_count:5 deleted_byte_count:35727 replica_placement:100 version:3 modified_at_second:1614904875
+ volume id:288 size:1075406920 collection:"collection2" file_count:27243 delete_count:2 deleted_byte_count:51519 replica_placement:100 version:3 modified_at_second:1614904738
+ volume id:289 size:1075284312 collection:"collection2" file_count:29342 delete_count:5 deleted_byte_count:100454 replica_placement:100 version:3 modified_at_second:1614904977
+ volume id:290 size:1074723800 collection:"collection2" file_count:28340 delete_count:4 deleted_byte_count:199064 replica_placement:100 version:3 modified_at_second:1614904924
+ volume id:292 size:1092010672 collection:"collection2" file_count:26781 delete_count:5 deleted_byte_count:508910 replica_placement:100 version:3 modified_at_second:1614912325
+ volume id:295 size:1074702320 collection:"collection2" file_count:24488 delete_count:3 deleted_byte_count:48555 replica_placement:100 version:3 modified_at_second:1614911929
+ volume id:296 size:1077824056 collection:"collection2" file_count:26741 delete_count:4 deleted_byte_count:199906 replica_placement:100 version:3 modified_at_second:1614912301
+ volume id:297 size:1080229176 collection:"collection2" file_count:23409 delete_count:5 deleted_byte_count:46268 replica_placement:100 version:3 modified_at_second:1614918481
+ volume id:298 size:1075410024 collection:"collection2" file_count:23222 delete_count:2 deleted_byte_count:46110 replica_placement:100 version:3 modified_at_second:1614918474
+ volume id:302 size:1077559640 collection:"collection2" file_count:23124 delete_count:7 deleted_byte_count:293111 replica_placement:100 version:3 modified_at_second:1614925500
+ volume id:304 size:1081038944 collection:"collection2" file_count:24505 delete_count:2 deleted_byte_count:124447 replica_placement:100 version:3 modified_at_second:1614925569
+ volume id:305 size:1074185376 collection:"collection2" file_count:22074 delete_count:5 deleted_byte_count:20221 replica_placement:100 version:3 modified_at_second:1614925312
+ volume id:306 size:1074763952 collection:"collection2" file_count:22939 replica_placement:100 version:3 modified_at_second:1614925462
+ volume id:307 size:1076567912 collection:"collection2" file_count:23377 delete_count:2 deleted_byte_count:25453 replica_placement:100 version:3 modified_at_second:1614931448
+ volume id:308 size:1074022336 collection:"collection2" file_count:23086 delete_count:2 deleted_byte_count:2127 replica_placement:100 version:3 modified_at_second:1614931401
+ volume id:311 size:1088248344 collection:"collection2" file_count:23553 delete_count:6 deleted_byte_count:191716 replica_placement:100 version:3 modified_at_second:1614931463
+ volume id:312 size:1075037528 collection:"collection2" file_count:22524 replica_placement:100 version:3 modified_at_second:1614937831
+ volume id:313 size:1074875960 collection:"collection2" file_count:22404 delete_count:4 deleted_byte_count:51728 replica_placement:100 version:3 modified_at_second:1614937755
+ volume id:316 size:1077720776 collection:"collection2" file_count:22605 delete_count:1 deleted_byte_count:8503 replica_placement:100 version:3 modified_at_second:1614937838
+ volume id:318 size:1075965168 collection:"collection2" file_count:22459 delete_count:2 deleted_byte_count:37778 replica_placement:100 version:3 modified_at_second:1614943862
+ volume id:322 size:1078471536 collection:"collection2" file_count:21905 delete_count:3 deleted_byte_count:145002 replica_placement:100 version:3 modified_at_second:1614950572
+ volume id:323 size:1074608056 collection:"collection2" file_count:21605 delete_count:4 deleted_byte_count:172090 replica_placement:100 version:3 modified_at_second:1614950526
+ volume id:325 size:1080701232 collection:"collection2" file_count:21735 replica_placement:100 version:3 modified_at_second:1614950525
+ volume id:326 size:1076059920 collection:"collection2" file_count:22564 delete_count:2 deleted_byte_count:192886 replica_placement:100 version:3 modified_at_second:1614950619
+ volume id:327 size:1076121304 collection:"collection2" file_count:22007 delete_count:3 deleted_byte_count:60358 replica_placement:100 version:3 modified_at_second:1614956487
+ volume id:328 size:1074767816 collection:"collection2" file_count:21720 delete_count:3 deleted_byte_count:56429 replica_placement:100 version:3 modified_at_second:1614956362
+ volume id:329 size:1076691960 collection:"collection2" file_count:22411 delete_count:5 deleted_byte_count:214092 replica_placement:100 version:3 modified_at_second:1614956485
+ volume id:330 size:1080825760 collection:"collection2" file_count:22464 delete_count:2 deleted_byte_count:15771 replica_placement:100 version:3 modified_at_second:1614956476
+ volume id:331 size:1074957256 collection:"collection2" file_count:21230 delete_count:4 deleted_byte_count:62145 replica_placement:100 version:3 modified_at_second:1614956259
+ volume id:332 size:1075569928 collection:"collection2" file_count:22097 delete_count:3 deleted_byte_count:98273 replica_placement:100 version:3 modified_at_second:1614962869
+ volume id:333 size:1074270160 collection:"collection2" file_count:21271 delete_count:2 deleted_byte_count:168122 replica_placement:100 version:3 modified_at_second:1614962697
+ volume id:334 size:1075607880 collection:"collection2" file_count:22546 delete_count:6 deleted_byte_count:101538 replica_placement:100 version:3 modified_at_second:1614962978
+ volume id:335 size:1076235136 collection:"collection2" file_count:22391 delete_count:3 deleted_byte_count:8838 replica_placement:100 version:3 modified_at_second:1614962970
+ volume id:337 size:1075646896 collection:"collection2" file_count:21934 delete_count:1 deleted_byte_count:3397 replica_placement:100 version:3 modified_at_second:1614969937
+ volume id:339 size:1078402392 collection:"collection2" file_count:22309 replica_placement:100 version:3 modified_at_second:1614969995
+ volume id:340 size:1079462152 collection:"collection2" file_count:22319 delete_count:4 deleted_byte_count:93620 replica_placement:100 version:3 modified_at_second:1614969977
+ volume id:341 size:1074448360 collection:"collection2" file_count:21590 delete_count:5 deleted_byte_count:160085 replica_placement:100 version:3 modified_at_second:1614969858
+ volume id:343 size:1075345072 collection:"collection2" file_count:21095 delete_count:2 deleted_byte_count:20581 replica_placement:100 version:3 modified_at_second:1614977148
+ volume id:346 size:1076464112 collection:"collection2" file_count:22320 delete_count:4 deleted_byte_count:798258 replica_placement:100 version:3 modified_at_second:1614977511
+ volume id:347 size:1075145248 collection:"collection2" file_count:22178 delete_count:1 deleted_byte_count:79392 replica_placement:100 version:3 modified_at_second:1614984727
+ volume id:348 size:1080623544 collection:"collection2" file_count:21667 delete_count:1 deleted_byte_count:2443 replica_placement:100 version:3 modified_at_second:1614984604
+ volume id:349 size:1075957672 collection:"collection2" file_count:22395 delete_count:2 deleted_byte_count:61565 replica_placement:100 version:3 modified_at_second:1614984748
+ volume id:351 size:1078795120 collection:"collection2" file_count:23660 delete_count:3 deleted_byte_count:102141 replica_placement:100 version:3 modified_at_second:1614984816
+ volume id:352 size:1077145936 collection:"collection2" file_count:22066 delete_count:1 deleted_byte_count:1018 replica_placement:100 version:3 modified_at_second:1614992130
+ volume id:353 size:1074897496 collection:"collection2" file_count:21266 delete_count:2 deleted_byte_count:3105374 replica_placement:100 version:3 modified_at_second:1614991951
+ volume id:354 size:1085214104 collection:"collection2" file_count:23150 delete_count:4 deleted_byte_count:82391 replica_placement:100 version:3 modified_at_second:1614992208
+ volume id:357 size:1074276152 collection:"collection2" file_count:23137 delete_count:4 deleted_byte_count:188487 replica_placement:100 version:3 modified_at_second:1614998792
+ volume id:359 size:1074211296 collection:"collection2" file_count:22437 delete_count:2 deleted_byte_count:187953 replica_placement:100 version:3 modified_at_second:1614998711
+ volume id:360 size:1075532512 collection:"collection2" file_count:22574 delete_count:3 deleted_byte_count:1774776 replica_placement:100 version:3 modified_at_second:1614998770
+ volume id:361 size:1075362744 collection:"collection2" file_count:22272 delete_count:1 deleted_byte_count:3497 replica_placement:100 version:3 modified_at_second:1614998668
+ volume id:362 size:1074074176 collection:"collection2" file_count:20595 delete_count:1 deleted_byte_count:112145 replica_placement:100 version:3 modified_at_second:1615004407
+ volume id:363 size:1078859640 collection:"collection2" file_count:23177 delete_count:4 deleted_byte_count:9601 replica_placement:100 version:3 modified_at_second:1615004823
+ volume id:364 size:1081280880 collection:"collection2" file_count:22686 delete_count:1 deleted_byte_count:84375 replica_placement:100 version:3 modified_at_second:1615004813
+ volume id:365 size:1075736632 collection:"collection2" file_count:22193 delete_count:5 deleted_byte_count:259033 replica_placement:100 version:3 modified_at_second:1615004776
+ volume id:366 size:1075267272 collection:"collection2" file_count:21856 delete_count:5 deleted_byte_count:138363 replica_placement:100 version:3 modified_at_second:1615004703
+ volume id:367 size:1076403648 collection:"collection2" file_count:22995 delete_count:2 deleted_byte_count:36955 replica_placement:100 version:3 modified_at_second:1615010985
+ volume id:368 size:1074821960 collection:"collection2" file_count:22252 delete_count:4 deleted_byte_count:3291946 replica_placement:100 version:3 modified_at_second:1615010877
+ volume id:369 size:1091472040 collection:"collection2" file_count:23709 delete_count:4 deleted_byte_count:400876 replica_placement:100 version:3 modified_at_second:1615011021
+ volume id:370 size:1076040544 collection:"collection2" file_count:22092 delete_count:2 deleted_byte_count:115388 replica_placement:100 version:3 modified_at_second:1615010877
+ volume id:371 size:1078806216 collection:"collection2" file_count:22685 delete_count:2 deleted_byte_count:68905 replica_placement:100 version:3 modified_at_second:1615010995
+ volume id:372 size:1076193344 collection:"collection2" file_count:22774 delete_count:1 deleted_byte_count:3495 replica_placement:100 version:3 modified_at_second:1615016911
+ volume id:373 size:1080928088 collection:"collection2" file_count:22617 delete_count:4 deleted_byte_count:91849 replica_placement:100 version:3 modified_at_second:1615016878
+ volume id:374 size:1085011176 collection:"collection2" file_count:23054 delete_count:2 deleted_byte_count:89034 replica_placement:100 version:3 modified_at_second:1615016917
+ volume id:376 size:1074845832 collection:"collection2" file_count:22908 delete_count:4 deleted_byte_count:432305 replica_placement:100 version:3 modified_at_second:1615016916
+ volume id:377 size:957434264 collection:"collection2" file_count:14929 delete_count:1 deleted_byte_count:43099 replica_placement:100 version:3 modified_at_second:1615632323
+ volume id:379 size:1014108528 collection:"collection2" file_count:15362 delete_count:6 deleted_byte_count:2481613 replica_placement:100 version:3 modified_at_second:1615632323
+ Disk hdd total size:306912958016 file_count:4201794 deleted_file:15268 deleted_bytes:4779359660
+ DataNode 192.168.1.5:8080 total size:306912958016 file_count:4201794 deleted_file:15268 deleted_bytes:4779359660
+ Rack DefaultRack total size:306912958016 file_count:4201794 deleted_file:15268 deleted_bytes:4779359660
+ DataCenter dc5 total size:306912958016 file_count:4201794 deleted_file:15268 deleted_bytes:4779359660
+total size:775256653592 file_count:10478712 deleted_file:33754 deleted_bytes:10839266043
+`
diff --git a/weed/shell/command_volume_server_evacuate_test.go b/weed/shell/command_volume_server_evacuate_test.go
new file mode 100644
index 000000000..5753af78b
--- /dev/null
+++ b/weed/shell/command_volume_server_evacuate_test.go
@@ -0,0 +1,17 @@
+package shell
+
+import (
+ "os"
+ "testing"
+)
+
+func TestVolumeServerEvacuate(t *testing.T) {
+ topologyInfo := parseOutput(topoData)
+
+ volumeServer := "192.168.1.4:8080"
+
+ if err := evacuateNormalVolumes(nil, topologyInfo, volumeServer, true, false, os.Stdout); err != nil {
+ t.Errorf("evacuate: %v", err)
+ }
+
+}
diff --git a/weed/shell/shell_liner.go b/weed/shell/shell_liner.go
index 00adb1abc..d79f67032 100644
--- a/weed/shell/shell_liner.go
+++ b/weed/shell/shell_liner.go
@@ -19,6 +19,10 @@ var (
func RunShell(options ShellOptions) {
+ sort.Slice(Commands, func(i, j int) bool {
+ return strings.Compare(Commands[i].Name(), Commands[j].Name()) < 0
+ })
+
line = liner.NewLiner()
defer line.Close()
@@ -96,9 +100,6 @@ func printGenericHelp() {
`
fmt.Print(msg)
- sort.Slice(Commands, func(i, j int) bool {
- return strings.Compare(Commands[i].Name(), Commands[j].Name()) < 0
- })
for _, c := range Commands {
helpTexts := strings.SplitN(c.Help(), "\n", 2)
fmt.Printf(" %-30s\t# %s \n", c.Name(), helpTexts[0])
@@ -114,10 +115,6 @@ func printHelp(cmds []string) {
} else {
cmd := strings.ToLower(args[0])
- sort.Slice(Commands, func(i, j int) bool {
- return strings.Compare(Commands[i].Name(), Commands[j].Name()) < 0
- })
-
for _, c := range Commands {
if c.Name() == cmd {
fmt.Printf(" %s\t# %s\n", c.Name(), c.Help())
diff --git a/weed/storage/backend/backend.go b/weed/storage/backend/backend.go
index b8b883be6..2dc61d02e 100644
--- a/weed/storage/backend/backend.go
+++ b/weed/storage/backend/backend.go
@@ -58,6 +58,9 @@ func LoadConfiguration(config *util.ViperProxy) {
if !config.GetBool(StorageBackendPrefix + "." + backendTypeName + "." + backendStorageId + ".enabled") {
continue
}
+ if _, found := BackendStorages[backendTypeName+"."+backendStorageId]; found {
+ continue
+ }
backendStorage, buildErr := backendStorageFactory.BuildStorage(config,
StorageBackendPrefix+"."+backendTypeName+"."+backendStorageId+".", backendStorageId)
if buildErr != nil {
@@ -81,6 +84,9 @@ func LoadFromPbStorageBackends(storageBackends []*master_pb.StorageBackend) {
glog.Warningf("storage type %s not found", storageBackend.Type)
continue
}
+ if _, found := BackendStorages[storageBackend.Type+"."+storageBackend.Id]; found {
+ continue
+ }
backendStorage, buildErr := backendStorageFactory.BuildStorage(newProperties(storageBackend.Properties), "", storageBackend.Id)
if buildErr != nil {
glog.Fatalf("fail to create backend storage %s.%s", storageBackend.Type, storageBackend.Id)
diff --git a/weed/storage/backend/disk_file.go b/weed/storage/backend/disk_file.go
index 498963c31..3b42429cf 100644
--- a/weed/storage/backend/disk_file.go
+++ b/weed/storage/backend/disk_file.go
@@ -52,7 +52,7 @@ func (df *DiskFile) WriteAt(p []byte, off int64) (n int, err error) {
return
}
-func (df *DiskFile) Append(p []byte) (n int, err error) {
+func (df *DiskFile) Write(p []byte) (n int, err error) {
return df.WriteAt(p, df.fileSize)
}
diff --git a/weed/storage/backend/s3_backend/s3_sessions.go b/weed/storage/backend/s3_backend/s3_sessions.go
index e2fdf1eb6..b8378c379 100644
--- a/weed/storage/backend/s3_backend/s3_sessions.go
+++ b/weed/storage/backend/s3_backend/s3_sessions.go
@@ -34,8 +34,9 @@ func createSession(awsAccessKeyId, awsSecretAccessKey, region, endpoint string)
}
config := &aws.Config{
- Region: aws.String(region),
- Endpoint: aws.String(endpoint),
+ Region: aws.String(region),
+ Endpoint: aws.String(endpoint),
+ S3ForcePathStyle: aws.Bool(true),
}
if awsAccessKeyId != "" && awsSecretAccessKey != "" {
config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, awsSecretAccessKey, "")
diff --git a/weed/storage/erasure_coding/ec_volume.go b/weed/storage/erasure_coding/ec_volume.go
index 85d6a5fc8..171db92a4 100644
--- a/weed/storage/erasure_coding/ec_volume.go
+++ b/weed/storage/erasure_coding/ec_volume.go
@@ -63,7 +63,7 @@ func NewEcVolume(diskType types.DiskType, dir string, dirIdx string, collection
// read volume info
ev.Version = needle.Version3
- if volumeInfo, found, _ := pb.MaybeLoadVolumeInfo(dataBaseFileName + ".vif"); found {
+ if volumeInfo, _, found, _ := pb.MaybeLoadVolumeInfo(dataBaseFileName + ".vif"); found {
ev.Version = needle.Version(volumeInfo.Version)
} else {
pb.SaveVolumeInfo(dataBaseFileName+".vif", &volume_server_pb.VolumeInfo{Version: uint32(ev.Version)})
diff --git a/weed/storage/needle/crc.go b/weed/storage/needle/crc.go
index 6fd910bb7..4476631c2 100644
--- a/weed/storage/needle/crc.go
+++ b/weed/storage/needle/crc.go
@@ -2,6 +2,7 @@ package needle
import (
"fmt"
+ "io"
"github.com/klauspost/crc32"
@@ -29,3 +30,25 @@ func (n *Needle) Etag() string {
util.Uint32toBytes(bits, uint32(n.Checksum))
return fmt.Sprintf("%x", bits)
}
+
+func NewCRCwriter(w io.Writer) *CRCwriter {
+
+ return &CRCwriter{
+ crc: CRC(0),
+ w: w,
+ }
+
+}
+
+type CRCwriter struct {
+ crc CRC
+ w io.Writer
+}
+
+func (c *CRCwriter) Write(p []byte) (n int, err error) {
+ n, err = c.w.Write(p) // with each write ...
+ c.crc = c.crc.Update(p)
+ return
+}
+
+func (c *CRCwriter) Sum() uint32 { return c.crc.Value() } // final hash
diff --git a/weed/storage/needle/needle_read_write.go b/weed/storage/needle/needle_read_write.go
index 0f72bc0bb..16c2fd06b 100644
--- a/weed/storage/needle/needle_read_write.go
+++ b/weed/storage/needle/needle_read_write.go
@@ -3,13 +3,12 @@ package needle
import (
"errors"
"fmt"
- "io"
- "math"
-
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
+ "io"
+ "math"
)
const (
@@ -156,6 +155,35 @@ func (n *Needle) Append(w backend.BackendStorageFile, version Version) (offset u
return offset, size, actualSize, err
}
+func WriteNeedleBlob(w backend.BackendStorageFile, dataSlice []byte, size Size, appendAtNs uint64, version Version) (offset uint64, err error) {
+
+ if end, _, e := w.GetStat(); e == nil {
+ defer func(w backend.BackendStorageFile, off int64) {
+ if err != nil {
+ if te := w.Truncate(end); te != nil {
+ glog.V(0).Infof("Failed to truncate %s back to %d with error: %v", w.Name(), end, te)
+ }
+ }
+ }(w, end)
+ offset = uint64(end)
+ } else {
+ err = fmt.Errorf("Cannot Read Current Volume Position: %v", e)
+ return
+ }
+
+ if version == Version3 {
+ tsOffset := NeedleHeaderSize + size + NeedleChecksumSize
+ util.Uint64toBytes(dataSlice[tsOffset:tsOffset+TimestampSize], appendAtNs)
+ }
+
+ if err == nil {
+ _, err = w.WriteAt(dataSlice, int64(offset))
+ }
+
+ return
+
+}
+
func ReadNeedleBlob(r backend.BackendStorageFile, offset int64, size Size, version Version) (dataSlice []byte, err error) {
dataSize := GetActualSize(size, version)
@@ -168,7 +196,7 @@ func ReadNeedleBlob(r backend.BackendStorageFile, offset int64, size Size, versi
}
if err != nil {
fileSize, _, _ := r.GetStat()
- println("n",n, "dataSize", dataSize, "offset", offset, "fileSize", fileSize)
+ println("n", n, "dataSize", dataSize, "offset", offset, "fileSize", fileSize)
}
return dataSlice, err
diff --git a/weed/storage/needle_map/memdb.go b/weed/storage/needle_map/memdb.go
index b25b5e89a..ba1fd3d1e 100644
--- a/weed/storage/needle_map/memdb.go
+++ b/weed/storage/needle_map/memdb.go
@@ -2,6 +2,7 @@ package needle_map
import (
"fmt"
+ "io"
"os"
"github.com/syndtr/goleveldb/leveldb"
@@ -104,7 +105,13 @@ func (cm *MemDb) LoadFromIdx(idxName string) (ret error) {
}
defer idxFile.Close()
- return idx.WalkIndexFile(idxFile, func(key NeedleId, offset Offset, size Size) error {
+ return cm.LoadFromReaderAt(idxFile)
+
+}
+
+func (cm *MemDb) LoadFromReaderAt(readerAt io.ReaderAt) (ret error) {
+
+ return idx.WalkIndexFile(readerAt, func(key NeedleId, offset Offset, size Size) error {
if offset.IsZero() || size.IsDeleted() {
return cm.Delete(key)
}
diff --git a/weed/storage/needle_map_leveldb.go b/weed/storage/needle_map_leveldb.go
index 9716e9729..31c86d124 100644
--- a/weed/storage/needle_map_leveldb.go
+++ b/weed/storage/needle_map_leveldb.go
@@ -152,8 +152,10 @@ func (m *LevelDbNeedleMap) Close() {
glog.Warningf("close index file %s failed: %v", indexFileName, err)
}
- if err := m.db.Close(); err != nil {
- glog.Warningf("close levelDB failed: %v", err)
+ if m.db != nil {
+ if err := m.db.Close(); err != nil {
+ glog.Warningf("close levelDB failed: %v", err)
+ }
}
}
diff --git a/weed/storage/needle_map_sorted_file.go b/weed/storage/needle_map_sorted_file.go
index 3449ff9dc..662b90531 100644
--- a/weed/storage/needle_map_sorted_file.go
+++ b/weed/storage/needle_map_sorted_file.go
@@ -94,8 +94,12 @@ func (m *SortedFileNeedleMap) Delete(key NeedleId, offset Offset) error {
}
func (m *SortedFileNeedleMap) Close() {
- m.indexFile.Close()
- m.dbFile.Close()
+ if m.indexFile != nil {
+ m.indexFile.Close()
+ }
+ if m.dbFile != nil {
+ m.dbFile.Close()
+ }
}
func (m *SortedFileNeedleMap) Destroy() error {
diff --git a/weed/storage/store.go b/weed/storage/store.go
index 47829666a..c3507c0e2 100644
--- a/weed/storage/store.go
+++ b/weed/storage/store.go
@@ -217,23 +217,36 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
location.volumesLock.RLock()
for _, v := range location.volumes {
curMaxFileKey, volumeMessage := v.ToVolumeInformationMessage()
+ if volumeMessage == nil {
+ continue
+ }
if maxFileKey < curMaxFileKey {
maxFileKey = curMaxFileKey
}
+ deleteVolume := false
if !v.expired(volumeMessage.Size, s.GetVolumeSizeLimit()) {
volumeMessages = append(volumeMessages, volumeMessage)
} else {
if v.expiredLongEnough(MAX_TTL_VOLUME_REMOVAL_DELAY) {
deleteVids = append(deleteVids, v.Id)
+ deleteVolume = true
} else {
glog.V(0).Infof("volume %d is expired", v.Id)
}
if v.lastIoError != nil {
deleteVids = append(deleteVids, v.Id)
+ deleteVolume = true
glog.Warningf("volume %d has IO error: %v", v.Id, v.lastIoError)
}
}
- collectionVolumeSize[v.Collection] += volumeMessage.Size
+
+ if _, exist := collectionVolumeSize[v.Collection]; !exist {
+ collectionVolumeSize[v.Collection] = 0
+ }
+ if !deleteVolume {
+ collectionVolumeSize[v.Collection] += volumeMessage.Size
+ }
+
if _, exist := collectionVolumeReadOnlyCount[v.Collection]; !exist {
collectionVolumeReadOnlyCount[v.Collection] = map[string]uint8{
"IsReadOnly": 0,
@@ -242,7 +255,7 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
"isDiskSpaceLow": 0,
}
}
- if v.IsReadOnly() {
+ if !deleteVolume && v.IsReadOnly() {
collectionVolumeReadOnlyCount[v.Collection]["IsReadOnly"] += 1
if v.noWriteOrDelete {
collectionVolumeReadOnlyCount[v.Collection]["noWriteOrDelete"] += 1
@@ -267,7 +280,7 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
glog.V(0).Infof("volume %d is deleted", vid)
}
} else {
- glog.V(0).Infof("delete volume %d: %v", vid, err)
+ glog.Warningf("delete volume %d: %v", vid, err)
}
}
location.volumesLock.Unlock()
@@ -446,7 +459,7 @@ func (s *Store) ConfigureVolume(i needle.VolumeId, replication string) error {
// load, modify, save
baseFileName := strings.TrimSuffix(fileInfo.Name(), filepath.Ext(fileInfo.Name()))
vifFile := filepath.Join(location.Directory, baseFileName+".vif")
- volumeInfo, _, err := pb.MaybeLoadVolumeInfo(vifFile)
+ volumeInfo, _, _, err := pb.MaybeLoadVolumeInfo(vifFile)
if err != nil {
return fmt.Errorf("volume %d fail to load vif", i)
}
diff --git a/weed/storage/super_block/replica_placement.go b/weed/storage/super_block/replica_placement.go
index 65ec53819..a263e6669 100644
--- a/weed/storage/super_block/replica_placement.go
+++ b/weed/storage/super_block/replica_placement.go
@@ -36,6 +36,9 @@ func NewReplicaPlacementFromByte(b byte) (*ReplicaPlacement, error) {
}
func (rp *ReplicaPlacement) Byte() byte {
+ if rp == nil {
+ return 0
+ }
ret := rp.DiffDataCenterCount*100 + rp.DiffRackCount*10 + rp.SameRackCount
return byte(ret)
}
diff --git a/weed/storage/volume.go b/weed/storage/volume.go
index 366449c53..e0638d8a8 100644
--- a/weed/storage/volume.go
+++ b/weed/storage/volume.go
@@ -234,10 +234,16 @@ func (v *Volume) expiredLongEnough(maxDelayMinutes uint32) bool {
return false
}
-func (v *Volume) CollectStatus() (maxFileKey types.NeedleId, datFileSize int64, modTime time.Time, fileCount, deletedCount, deletedSize uint64) {
+func (v *Volume) collectStatus() (maxFileKey types.NeedleId, datFileSize int64, modTime time.Time, fileCount, deletedCount, deletedSize uint64, ok bool) {
v.dataFileAccessLock.RLock()
defer v.dataFileAccessLock.RUnlock()
- glog.V(3).Infof("CollectStatus volume %d", v.Id)
+ glog.V(3).Infof("collectStatus volume %d", v.Id)
+
+ if v.nm == nil {
+ return
+ }
+
+ ok = true
maxFileKey = v.nm.MaxFileKey()
datFileSize, modTime, _ = v.DataBackend.GetStat()
@@ -251,7 +257,11 @@ func (v *Volume) CollectStatus() (maxFileKey types.NeedleId, datFileSize int64,
func (v *Volume) ToVolumeInformationMessage() (types.NeedleId, *master_pb.VolumeInformationMessage) {
- maxFileKey, volumeSize, modTime, fileCount, deletedCount, deletedSize := v.CollectStatus()
+ maxFileKey, volumeSize, modTime, fileCount, deletedCount, deletedSize, ok := v.collectStatus()
+
+ if !ok {
+ return 0, nil
+ }
volumeInfo := &master_pb.VolumeInformationMessage{
Id: uint32(v.Id),
diff --git a/weed/storage/volume_loading.go b/weed/storage/volume_loading.go
index bff1055bb..0cf603ad8 100644
--- a/weed/storage/volume_loading.go
+++ b/weed/storage/volume_loading.go
@@ -39,12 +39,12 @@ func (v *Volume) load(alsoLoadIndex bool, createDatIfMissing bool, needleMapKind
}
}()
- hasVolumeInfoFile := v.maybeLoadVolumeInfo() && v.volumeInfo.Version != 0
+ hasVolumeInfoFile := v.maybeLoadVolumeInfo()
if v.HasRemoteFile() {
v.noWriteCanDelete = true
v.noWriteOrDelete = false
- glog.V(0).Infof("loading volume %d from remote %v", v.Id, v.volumeInfo.Files)
+ glog.V(0).Infof("loading volume %d from remote %v", v.Id, v.volumeInfo)
v.LoadRemoteFile()
alreadyHasSuperBlock = true
} else if exists, canRead, canWrite, modifiedTime, fileSize := util.CheckFile(v.FileName(".dat")); exists {
@@ -83,6 +83,12 @@ func (v *Volume) load(alsoLoadIndex bool, createDatIfMissing bool, needleMapKind
if alreadyHasSuperBlock {
err = v.readSuperBlock()
+ glog.V(0).Infof("readSuperBlock volume %d version %v", v.Id, v.SuperBlock.Version)
+ if v.HasRemoteFile() {
+ // maybe temporary network problem
+ glog.Errorf("readSuperBlock remote volume %d: %v", v.Id, err)
+ err = nil
+ }
} else {
if !v.SuperBlock.Initialized() {
return fmt.Errorf("volume %s not initialized", v.FileName(".dat"))
diff --git a/weed/storage/volume_read.go b/weed/storage/volume_read.go
new file mode 100644
index 000000000..f689eeec0
--- /dev/null
+++ b/weed/storage/volume_read.go
@@ -0,0 +1,131 @@
+package storage
+
+import (
+ "fmt"
+ "io"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+// read fills in Needle content by looking up n.Id from NeedleMapper
+func (v *Volume) readNeedle(n *needle.Needle, readOption *ReadOption) (int, error) {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+
+ nv, ok := v.nm.Get(n.Id)
+ if !ok || nv.Offset.IsZero() {
+ return -1, ErrorNotFound
+ }
+ readSize := nv.Size
+ if readSize.IsDeleted() {
+ if readOption != nil && readOption.ReadDeleted && readSize != TombstoneFileSize {
+ glog.V(3).Infof("reading deleted %s", n.String())
+ readSize = -readSize
+ } else {
+ return -1, ErrorDeleted
+ }
+ }
+ if readSize == 0 {
+ return 0, nil
+ }
+ err := n.ReadData(v.DataBackend, nv.Offset.ToActualOffset(), readSize, v.Version())
+ if err == needle.ErrorSizeMismatch && OffsetSize == 4 {
+ err = n.ReadData(v.DataBackend, nv.Offset.ToActualOffset()+int64(MaxPossibleVolumeSize), readSize, v.Version())
+ }
+ v.checkReadWriteError(err)
+ if err != nil {
+ return 0, err
+ }
+ bytesRead := len(n.Data)
+ if !n.HasTtl() {
+ return bytesRead, nil
+ }
+ ttlMinutes := n.Ttl.Minutes()
+ if ttlMinutes == 0 {
+ return bytesRead, nil
+ }
+ if !n.HasLastModifiedDate() {
+ return bytesRead, nil
+ }
+ if time.Now().Before(time.Unix(0, int64(n.AppendAtNs)).Add(time.Duration(ttlMinutes) * time.Minute)) {
+ return bytesRead, nil
+ }
+ return -1, ErrorNotFound
+}
+
+// read fills in Needle content by looking up n.Id from NeedleMapper
+func (v *Volume) ReadNeedleBlob(offset int64, size Size) ([]byte, error) {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+
+ return needle.ReadNeedleBlob(v.DataBackend, offset, size, v.Version())
+}
+
+type VolumeFileScanner interface {
+ VisitSuperBlock(super_block.SuperBlock) error
+ ReadNeedleBody() bool
+ VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error
+}
+
+func ScanVolumeFile(dirname string, collection string, id needle.VolumeId,
+ needleMapKind NeedleMapKind,
+ volumeFileScanner VolumeFileScanner) (err error) {
+ var v *Volume
+ if v, err = loadVolumeWithoutIndex(dirname, collection, id, needleMapKind); err != nil {
+ return fmt.Errorf("failed to load volume %d: %v", id, err)
+ }
+ if err = volumeFileScanner.VisitSuperBlock(v.SuperBlock); err != nil {
+ return fmt.Errorf("failed to process volume %d super block: %v", id, err)
+ }
+ defer v.Close()
+
+ version := v.Version()
+
+ offset := int64(v.SuperBlock.BlockSize())
+
+ return ScanVolumeFileFrom(version, v.DataBackend, offset, volumeFileScanner)
+}
+
+func ScanVolumeFileFrom(version needle.Version, datBackend backend.BackendStorageFile, offset int64, volumeFileScanner VolumeFileScanner) (err error) {
+ n, nh, rest, e := needle.ReadNeedleHeader(datBackend, version, offset)
+ if e != nil {
+ if e == io.EOF {
+ return nil
+ }
+ return fmt.Errorf("cannot read %s at offset %d: %v", datBackend.Name(), offset, e)
+ }
+ for n != nil {
+ var needleBody []byte
+ if volumeFileScanner.ReadNeedleBody() {
+ // println("needle", n.Id.String(), "offset", offset, "size", n.Size, "rest", rest)
+ if needleBody, err = n.ReadNeedleBody(datBackend, version, offset+NeedleHeaderSize, rest); err != nil {
+ glog.V(0).Infof("cannot read needle head [%d, %d) body [%d, %d) body length %d: %v", offset, offset+NeedleHeaderSize, offset+NeedleHeaderSize, offset+NeedleHeaderSize+rest, rest, err)
+ // err = fmt.Errorf("cannot read needle body: %v", err)
+ // return
+ }
+ }
+ err := volumeFileScanner.VisitNeedle(n, offset, nh, needleBody)
+ if err == io.EOF {
+ return nil
+ }
+ if err != nil {
+ glog.V(0).Infof("visit needle error: %v", err)
+ return fmt.Errorf("visit needle error: %v", err)
+ }
+ offset += NeedleHeaderSize + rest
+ glog.V(4).Infof("==> new entry offset %d", offset)
+ if n, nh, rest, err = needle.ReadNeedleHeader(datBackend, version, offset); err != nil {
+ if err == io.EOF {
+ return nil
+ }
+ return fmt.Errorf("cannot read needle header at offset %d: %v", offset, err)
+ }
+ glog.V(4).Infof("new entry needle size:%d rest:%d", n.Size, rest)
+ }
+ return nil
+}
diff --git a/weed/storage/volume_stream_write.go b/weed/storage/volume_stream_write.go
new file mode 100644
index 000000000..d229bdf20
--- /dev/null
+++ b/weed/storage/volume_stream_write.go
@@ -0,0 +1,104 @@
+package storage
+
+import (
+ "bufio"
+ "fmt"
+ "github.com/chrislusf/seaweedfs/weed/util"
+ "io"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+func (v *Volume) StreamWrite(n *needle.Needle, data io.Reader, dataSize uint32) (err error) {
+
+ v.dataFileAccessLock.Lock()
+ defer v.dataFileAccessLock.Unlock()
+
+ df, ok := v.DataBackend.(*backend.DiskFile)
+ if !ok {
+ return fmt.Errorf("unexpected volume backend")
+ }
+ offset, _, _ := v.DataBackend.GetStat()
+
+ header := make([]byte, NeedleHeaderSize+TimestampSize) // adding timestamp to reuse it and avoid extra allocation
+ CookieToBytes(header[0:CookieSize], n.Cookie)
+ NeedleIdToBytes(header[CookieSize:CookieSize+NeedleIdSize], n.Id)
+ n.Size = 4 + Size(dataSize) + 1
+ SizeToBytes(header[CookieSize+NeedleIdSize:CookieSize+NeedleIdSize+SizeSize], n.Size)
+
+ n.DataSize = dataSize
+
+ // needle header
+ df.Write(header[0:NeedleHeaderSize])
+
+ // data size and data
+ util.Uint32toBytes(header[0:4], n.DataSize)
+ df.Write(header[0:4])
+ // write and calculate CRC
+ crcWriter := needle.NewCRCwriter(df)
+ io.Copy(crcWriter, io.LimitReader(data, int64(dataSize)))
+
+ // flags
+ util.Uint8toBytes(header[0:1], n.Flags)
+ df.Write(header[0:1])
+
+ // data checksum
+ util.Uint32toBytes(header[0:needle.NeedleChecksumSize], crcWriter.Sum())
+ // write timestamp, padding
+ n.AppendAtNs = uint64(time.Now().UnixNano())
+ util.Uint64toBytes(header[needle.NeedleChecksumSize:needle.NeedleChecksumSize+TimestampSize], n.AppendAtNs)
+ padding := needle.PaddingLength(n.Size, needle.Version3)
+ df.Write(header[0 : needle.NeedleChecksumSize+TimestampSize+padding])
+
+ // add to needle map
+ if err = v.nm.Put(n.Id, ToOffset(int64(offset)), n.Size); err != nil {
+ glog.V(4).Infof("failed to save in needle map %d: %v", n.Id, err)
+ }
+ return
+}
+
+func (v *Volume) StreamRead(n *needle.Needle, writer io.Writer) (err error) {
+
+ v.dataFileAccessLock.Lock()
+ defer v.dataFileAccessLock.Unlock()
+
+ nv, ok := v.nm.Get(n.Id)
+ if !ok || nv.Offset.IsZero() {
+ return ErrorNotFound
+ }
+
+ sr := &StreamReader{
+ readerAt: v.DataBackend,
+ offset: nv.Offset.ToActualOffset(),
+ }
+ bufReader := bufio.NewReader(sr)
+ bufReader.Discard(NeedleHeaderSize)
+ sizeBuf := make([]byte, 4)
+ bufReader.Read(sizeBuf)
+ if _, err = writer.Write(sizeBuf); err != nil {
+ return err
+ }
+ dataSize := util.BytesToUint32(sizeBuf)
+
+ _, err = io.Copy(writer, io.LimitReader(bufReader, int64(dataSize)))
+
+ return
+}
+
+type StreamReader struct {
+ offset int64
+ readerAt io.ReaderAt
+}
+
+func (sr *StreamReader) Read(p []byte) (n int, err error) {
+ n, err = sr.readerAt.ReadAt(p, sr.offset)
+ if err != nil {
+ return
+ }
+ sr.offset += int64(n)
+ return
+}
diff --git a/weed/storage/volume_tier.go b/weed/storage/volume_tier.go
index 77efd8a14..23160906b 100644
--- a/weed/storage/volume_tier.go
+++ b/weed/storage/volume_tier.go
@@ -6,6 +6,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
_ "github.com/chrislusf/seaweedfs/weed/storage/backend/s3_backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
)
func (v *Volume) GetVolumeInfo() *volume_server_pb.VolumeInfo {
@@ -14,13 +15,23 @@ func (v *Volume) GetVolumeInfo() *volume_server_pb.VolumeInfo {
func (v *Volume) maybeLoadVolumeInfo() (found bool) {
- v.volumeInfo, v.hasRemoteFile, _ = pb.MaybeLoadVolumeInfo(v.FileName(".vif"))
+ var err error
+ v.volumeInfo, v.hasRemoteFile, found, err = pb.MaybeLoadVolumeInfo(v.FileName(".vif"))
+
+ if v.volumeInfo.Version == 0 {
+ v.volumeInfo.Version = uint32(needle.CurrentVersion)
+ }
if v.hasRemoteFile {
glog.V(0).Infof("volume %d is tiered to %s as %s and read only", v.Id,
v.volumeInfo.Files[0].BackendName(), v.volumeInfo.Files[0].Key)
}
+ if err != nil {
+ glog.Warningf("load volume %d.vif file: %v", v.Id, err)
+ return
+ }
+
return
}
diff --git a/weed/storage/volume_vacuum.go b/weed/storage/volume_vacuum.go
index 0ee1e61c6..be84f8a13 100644
--- a/weed/storage/volume_vacuum.go
+++ b/weed/storage/volume_vacuum.go
@@ -286,7 +286,7 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
if err != nil {
return fmt.Errorf("ReadNeedleBlob %s key %d offset %d size %d failed: %v", oldDatFile.Name(), key, increIdxEntry.offset.ToActualOffset(), increIdxEntry.size, err)
}
- dstDatBackend.Append(needleBytes)
+ dstDatBackend.Write(needleBytes)
util.Uint32toBytes(idxEntryBytes[8:12], uint32(offset/NeedlePaddingSize))
} else { //deleted needle
//fakeDelNeedle 's default Data field is nil
diff --git a/weed/storage/volume_read_write.go b/weed/storage/volume_write.go
index 07376bc88..a286c5dd5 100644
--- a/weed/storage/volume_read_write.go
+++ b/weed/storage/volume_write.go
@@ -4,14 +4,12 @@ import (
"bytes"
"errors"
"fmt"
- "io"
"os"
"time"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
- "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -104,47 +102,8 @@ func (v *Volume) syncWrite(n *needle.Needle) (offset uint64, size Size, isUnchan
err = fmt.Errorf("volume size limit %d exceeded! current size is %d", MaxPossibleVolumeSize, v.nm.ContentSize())
return
}
- if v.isFileUnchanged(n) {
- size = Size(n.DataSize)
- isUnchanged = true
- return
- }
-
- // check whether existing needle cookie matches
- nv, ok := v.nm.Get(n.Id)
- if ok {
- existingNeedle, _, _, existingNeedleReadErr := needle.ReadNeedleHeader(v.DataBackend, v.Version(), nv.Offset.ToActualOffset())
- if existingNeedleReadErr != nil {
- err = fmt.Errorf("reading existing needle: %v", existingNeedleReadErr)
- return
- }
- if existingNeedle.Cookie != n.Cookie {
- glog.V(0).Infof("write cookie mismatch: existing %x, new %x", existingNeedle.Cookie, n.Cookie)
- err = fmt.Errorf("mismatching cookie %x", n.Cookie)
- return
- }
- }
-
- // append to dat file
- n.AppendAtNs = uint64(time.Now().UnixNano())
- offset, size, _, err = n.Append(v.DataBackend, v.Version())
- v.checkReadWriteError(err)
- if err != nil {
- return
- }
- v.lastAppendAtNs = n.AppendAtNs
-
- // add to needle map
- if !ok || uint64(nv.Offset.ToActualOffset()) < offset {
- if err = v.nm.Put(n.Id, ToOffset(int64(offset)), n.Size); err != nil {
- glog.V(4).Infof("failed to save in needle map %d: %v", n.Id, err)
- }
- }
- if v.lastModifiedTsSeconds < n.LastModified {
- v.lastModifiedTsSeconds = n.LastModified
- }
- return
+ return v.doWriteRequest(n)
}
func (v *Volume) writeNeedle2(n *needle.Needle, fsync bool) (offset uint64, size Size, isUnchanged bool, err error) {
@@ -185,7 +144,8 @@ func (v *Volume) doWriteRequest(n *needle.Needle) (offset uint64, size Size, isU
return
}
if existingNeedle.Cookie != n.Cookie {
- glog.V(0).Infof("write cookie mismatch: existing %x, new %x", existingNeedle.Cookie, n.Cookie)
+ glog.V(0).Infof("write cookie mismatch: existing %s, new %s",
+ needle.NewFileIdFromNeedle(v.Id, existingNeedle), needle.NewFileIdFromNeedle(v.Id, n))
err = fmt.Errorf("mismatching cookie %x", n.Cookie)
return
}
@@ -223,24 +183,7 @@ func (v *Volume) syncDelete(n *needle.Needle) (Size, error) {
return 0, err
}
- nv, ok := v.nm.Get(n.Id)
- // fmt.Println("key", n.Id, "volume offset", nv.Offset, "data_size", n.Size, "cached size", nv.Size)
- if ok && nv.Size.IsValid() {
- size := nv.Size
- n.Data = nil
- n.AppendAtNs = uint64(time.Now().UnixNano())
- offset, _, _, err := n.Append(v.DataBackend, v.Version())
- v.checkReadWriteError(err)
- if err != nil {
- return size, err
- }
- v.lastAppendAtNs = n.AppendAtNs
- if err = v.nm.Delete(n.Id, ToOffset(int64(offset))); err != nil {
- return size, err
- }
- return size, err
- }
- return 0, nil
+ return v.doDeleteRequest(n)
}
func (v *Volume) deleteNeedle2(n *needle.Needle) (Size, error) {
@@ -282,52 +225,6 @@ func (v *Volume) doDeleteRequest(n *needle.Needle) (Size, error) {
return 0, nil
}
-// read fills in Needle content by looking up n.Id from NeedleMapper
-func (v *Volume) readNeedle(n *needle.Needle, readOption *ReadOption) (int, error) {
- v.dataFileAccessLock.RLock()
- defer v.dataFileAccessLock.RUnlock()
-
- nv, ok := v.nm.Get(n.Id)
- if !ok || nv.Offset.IsZero() {
- return -1, ErrorNotFound
- }
- readSize := nv.Size
- if readSize.IsDeleted() {
- if readOption != nil && readOption.ReadDeleted && readSize != TombstoneFileSize {
- glog.V(3).Infof("reading deleted %s", n.String())
- readSize = -readSize
- } else {
- return -1, ErrorDeleted
- }
- }
- if readSize == 0 {
- return 0, nil
- }
- err := n.ReadData(v.DataBackend, nv.Offset.ToActualOffset(), readSize, v.Version())
- if err == needle.ErrorSizeMismatch && OffsetSize == 4 {
- err = n.ReadData(v.DataBackend, nv.Offset.ToActualOffset()+int64(MaxPossibleVolumeSize), readSize, v.Version())
- }
- v.checkReadWriteError(err)
- if err != nil {
- return 0, err
- }
- bytesRead := len(n.Data)
- if !n.HasTtl() {
- return bytesRead, nil
- }
- ttlMinutes := n.Ttl.Minutes()
- if ttlMinutes == 0 {
- return bytesRead, nil
- }
- if !n.HasLastModifiedDate() {
- return bytesRead, nil
- }
- if time.Now().Before(time.Unix(0, int64(n.AppendAtNs)).Add(time.Duration(ttlMinutes) * time.Minute)) {
- return bytesRead, nil
- }
- return -1, ErrorNotFound
-}
-
func (v *Volume) startWorker() {
go func() {
chanClosed := false
@@ -403,66 +300,28 @@ func (v *Volume) startWorker() {
}()
}
-type VolumeFileScanner interface {
- VisitSuperBlock(super_block.SuperBlock) error
- ReadNeedleBody() bool
- VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error
-}
-
-func ScanVolumeFile(dirname string, collection string, id needle.VolumeId,
- needleMapKind NeedleMapKind,
- volumeFileScanner VolumeFileScanner) (err error) {
- var v *Volume
- if v, err = loadVolumeWithoutIndex(dirname, collection, id, needleMapKind); err != nil {
- return fmt.Errorf("failed to load volume %d: %v", id, err)
- }
- if err = volumeFileScanner.VisitSuperBlock(v.SuperBlock); err != nil {
- return fmt.Errorf("failed to process volume %d super block: %v", id, err)
- }
- defer v.Close()
+func (v *Volume) WriteNeedleBlob(needleId NeedleId, needleBlob []byte, size Size) error {
- version := v.Version()
+ v.dataFileAccessLock.Lock()
+ defer v.dataFileAccessLock.Unlock()
- offset := int64(v.SuperBlock.BlockSize())
+ if MaxPossibleVolumeSize < v.nm.ContentSize()+uint64(len(needleBlob)) {
+ return fmt.Errorf("volume size limit %d exceeded! current size is %d", MaxPossibleVolumeSize, v.nm.ContentSize())
+ }
- return ScanVolumeFileFrom(version, v.DataBackend, offset, volumeFileScanner)
-}
+ appendAtNs := uint64(time.Now().UnixNano())
+ offset, err := needle.WriteNeedleBlob(v.DataBackend, needleBlob, size, appendAtNs, v.Version())
-func ScanVolumeFileFrom(version needle.Version, datBackend backend.BackendStorageFile, offset int64, volumeFileScanner VolumeFileScanner) (err error) {
- n, nh, rest, e := needle.ReadNeedleHeader(datBackend, version, offset)
- if e != nil {
- if e == io.EOF {
- return nil
- }
- return fmt.Errorf("cannot read %s at offset %d: %v", datBackend.Name(), offset, e)
+ v.checkReadWriteError(err)
+ if err != nil {
+ return err
}
- for n != nil {
- var needleBody []byte
- if volumeFileScanner.ReadNeedleBody() {
- // println("needle", n.Id.String(), "offset", offset, "size", n.Size, "rest", rest)
- if needleBody, err = n.ReadNeedleBody(datBackend, version, offset+NeedleHeaderSize, rest); err != nil {
- glog.V(0).Infof("cannot read needle head [%d, %d) body [%d, %d) body length %d: %v", offset, offset+NeedleHeaderSize, offset+NeedleHeaderSize, offset+NeedleHeaderSize+rest, rest, err)
- // err = fmt.Errorf("cannot read needle body: %v", err)
- // return
- }
- }
- err := volumeFileScanner.VisitNeedle(n, offset, nh, needleBody)
- if err == io.EOF {
- return nil
- }
- if err != nil {
- glog.V(0).Infof("visit needle error: %v", err)
- return fmt.Errorf("visit needle error: %v", err)
- }
- offset += NeedleHeaderSize + rest
- glog.V(4).Infof("==> new entry offset %d", offset)
- if n, nh, rest, err = needle.ReadNeedleHeader(datBackend, version, offset); err != nil {
- if err == io.EOF {
- return nil
- }
- return fmt.Errorf("cannot read needle header at offset %d: %v", offset, err)
- }
- glog.V(4).Infof("new entry needle size:%d rest:%d", n.Size, rest)
+ v.lastAppendAtNs = appendAtNs
+
+ // add to needle map
+ if err = v.nm.Put(needleId, ToOffset(int64(offset)), size); err != nil {
+ glog.V(4).Infof("failed to put in needle map %d: %v", needleId, err)
}
- return nil
+
+ return err
}
diff --git a/weed/topology/data_node_ec.go b/weed/topology/data_node_ec.go
index df1b6d658..330b16b24 100644
--- a/weed/topology/data_node_ec.go
+++ b/weed/topology/data_node_ec.go
@@ -25,7 +25,7 @@ func (dn *DataNode) UpdateEcShards(actualShards []*erasure_coding.EcVolumeInfo)
existingEcShards := dn.GetEcShards()
- // found out the newShards and deletedShards
+ // find out the newShards and deletedShards
var newShardCount, deletedShardCount int
for _, ecShards := range existingEcShards {
@@ -56,20 +56,19 @@ func (dn *DataNode) UpdateEcShards(actualShards []*erasure_coding.EcVolumeInfo)
disk.UpAdjustDiskUsageDelta(deltaDiskUsages)
}
+
for _, ecShards := range actualShards {
+ if dn.hasEcShards(ecShards.VolumeId) {
+ continue
+ }
+
+ newShards = append(newShards, ecShards)
disk := dn.getOrCreateDisk(ecShards.DiskType)
deltaDiskUsages := newDiskUsages()
deltaDiskUsage := deltaDiskUsages.getOrCreateDisk(types.ToDiskType(ecShards.DiskType))
-
- if !dn.hasEcShards(ecShards.VolumeId) {
- newShards = append(newShards, ecShards)
- newShardCount += ecShards.ShardIdCount()
- }
-
- deltaDiskUsage.ecShardCount = int64(newShardCount)
+ deltaDiskUsage.ecShardCount = int64(ecShards.ShardIdCount())
disk.UpAdjustDiskUsageDelta(deltaDiskUsages)
-
}
if len(newShards) > 0 || len(deletedShards) > 0 {
diff --git a/weed/topology/volume_layout.go b/weed/topology/volume_layout.go
index 5784c894b..c7e171248 100644
--- a/weed/topology/volume_layout.go
+++ b/weed/topology/volume_layout.go
@@ -432,7 +432,7 @@ func (vl *VolumeLayout) Stats() *VolumeLayoutStats {
if vl.readonlyVolumes.IsTrue(vid) {
ret.TotalSize += size
} else {
- ret.TotalSize += vl.volumeSizeLimit
+ ret.TotalSize += vl.volumeSizeLimit * uint64(vll.Length())
}
}
diff --git a/weed/topology/volume_location_list.go b/weed/topology/volume_location_list.go
index 64c13ca52..548c4cd25 100644
--- a/weed/topology/volume_location_list.go
+++ b/weed/topology/volume_location_list.go
@@ -82,7 +82,7 @@ func (dnll *VolumeLocationList) Stats(vid needle.VolumeId, freshThreshHold int64
if dnl.LastSeen < freshThreshHold {
vinfo, err := dnl.GetVolumesById(vid)
if err == nil {
- return vinfo.Size - vinfo.DeletedByteCount, vinfo.FileCount - vinfo.DeleteCount
+ return (vinfo.Size - vinfo.DeletedByteCount) * uint64(len(dnll.list)), vinfo.FileCount - vinfo.DeleteCount
}
}
}
diff --git a/weed/util/chunk_cache/chunk_cache_in_memory.go b/weed/util/chunk_cache/chunk_cache_in_memory.go
index 1eb00e1fa..5f26b8c78 100644
--- a/weed/util/chunk_cache/chunk_cache_in_memory.go
+++ b/weed/util/chunk_cache/chunk_cache_in_memory.go
@@ -32,5 +32,7 @@ func (c *ChunkCacheInMemory) GetChunk(fileId string) []byte {
}
func (c *ChunkCacheInMemory) SetChunk(fileId string, data []byte) {
- c.cache.Set(fileId, data, time.Hour)
+ localCopy := make([]byte, len(data))
+ copy(localCopy, data)
+ c.cache.Set(fileId, localCopy, time.Hour)
}
diff --git a/weed/util/constants.go b/weed/util/constants.go
index 55f5b52d6..40f4deae2 100644
--- a/weed/util/constants.go
+++ b/weed/util/constants.go
@@ -5,7 +5,7 @@ import (
)
var (
- VERSION = fmt.Sprintf("%s %d.%02d", sizeLimit, 2, 28)
+ VERSION = fmt.Sprintf("%s %d.%02d", sizeLimit, 2, 38)
COMMIT = ""
)
diff --git a/weed/util/fasthttp_util.go b/weed/util/fasthttp_util.go
deleted file mode 100644
index 82575af98..000000000
--- a/weed/util/fasthttp_util.go
+++ /dev/null
@@ -1,115 +0,0 @@
-package util
-
-import (
- "bytes"
- "fmt"
- "github.com/valyala/fasthttp"
- "sync"
- "time"
-)
-
-var (
- fastClient = &fasthttp.Client{
- NoDefaultUserAgentHeader: true, // Don't send: User-Agent: fasthttp
- MaxConnsPerHost: 1024,
- ReadBufferSize: 4096, // Make sure to set this big enough that your whole request can be read at once.
- WriteBufferSize: 64 * 1024, // Same but for your response.
- ReadTimeout: time.Second,
- WriteTimeout: time.Second,
- MaxIdleConnDuration: time.Minute,
- DisableHeaderNamesNormalizing: true, // If you set the case on your headers correctly you can enable this.
- }
-
- // Put everything in pools to prevent garbage.
- bytesPool = sync.Pool{
- New: func() interface{} {
- b := make([]byte, 0)
- return &b
- },
- }
-
- responsePool = sync.Pool{
- New: func() interface{} {
- return make(chan *fasthttp.Response)
- },
- }
-)
-
-func FastGet(url string) ([]byte, bool, error) {
-
- req := fasthttp.AcquireRequest()
- res := fasthttp.AcquireResponse()
- defer fasthttp.ReleaseRequest(req)
- defer fasthttp.ReleaseResponse(res)
-
- req.SetRequestURIBytes([]byte(url))
- req.Header.Add("Accept-Encoding", "gzip")
-
- err := fastClient.Do(req, res)
- if err != nil {
- return nil, true, err
- }
-
- var data []byte
- contentEncoding := res.Header.Peek("Content-Encoding")
- if bytes.Compare(contentEncoding, []byte("gzip")) == 0 {
- data, err = res.BodyGunzip()
- } else {
- data = res.Body()
- }
-
- out := make([]byte, len(data))
- copy(out, data)
-
- if res.StatusCode() >= 400 {
- retryable := res.StatusCode() >= 500
- return nil, retryable, fmt.Errorf("%s: %d", url, res.StatusCode())
- }
- if err != nil {
- return nil, false, err
- }
- return out, false, nil
-}
-
-func FastReadUrlAsStream(fileUrl string, cipherKey []byte, isContentGzipped bool, isFullChunk bool, offset int64, size int, fn func(data []byte)) (retryable bool, err error) {
-
- if cipherKey != nil {
- return readEncryptedUrl(fileUrl, cipherKey, isContentGzipped, isFullChunk, offset, size, fn)
- }
-
- req := fasthttp.AcquireRequest()
- res := fasthttp.AcquireResponse()
- defer fasthttp.ReleaseRequest(req)
- defer fasthttp.ReleaseResponse(res)
-
- req.SetRequestURIBytes([]byte(fileUrl))
-
- if isFullChunk {
- req.Header.Add("Accept-Encoding", "gzip")
- } else {
- req.Header.Add("Range", fmt.Sprintf("bytes=%d-%d", offset, offset+int64(size)-1))
- }
-
- if err = fastClient.Do(req, res); err != nil {
- return true, err
- }
-
- if res.StatusCode() >= 400 {
- retryable = res.StatusCode() >= 500
- return retryable, fmt.Errorf("%s: %d", fileUrl, res.StatusCode())
- }
-
- contentEncoding := res.Header.Peek("Content-Encoding")
- if bytes.Compare(contentEncoding, []byte("gzip")) == 0 {
- bodyData, err := res.BodyGunzip()
- if err != nil {
- return false, err
- }
- fn(bodyData)
- } else {
- fn(res.Body())
- }
-
- return false, nil
-
-}
diff --git a/weed/util/fla9/fla9.go b/weed/util/fla9/fla9.go
index 4a5884e9b..eb5700e8c 100644
--- a/weed/util/fla9/fla9.go
+++ b/weed/util/fla9/fla9.go
@@ -886,7 +886,7 @@ func (f *FlagSet) parseOne() (bool, error) {
// The return value will be ErrHelp if -help or -h were set but not defined.
func (f *FlagSet) Parse(arguments []string) error {
if _, ok := f.formal[DefaultConfigFlagName]; !ok {
- f.String(DefaultConfigFlagName, "", "config file")
+ f.String(DefaultConfigFlagName, "", "a file of command line options, each line in optionName=optionValue format")
}
f.parsed = true
@@ -995,6 +995,7 @@ func NewFlagSet(name string, errorHandling ErrorHandling) *FlagSet {
f := &FlagSet{
name: name,
errorHandling: errorHandling,
+ envPrefix: EnvPrefix,
}
return f
}
@@ -1078,7 +1079,7 @@ func NewFlagSetWithEnvPrefix(name string, prefix string, errorHandling ErrorHand
// DefaultConfigFlagName defines the flag name of the optional config file
// path. Used to lookup and parse the config file when a default is set and
// available on disk.
-var DefaultConfigFlagName = "conf"
+var DefaultConfigFlagName = "options"
// ParseFile parses flags from the file in path.
// Same format as commandline arguments, newlines and lines beginning with a
diff --git a/weed/util/http_util.go b/weed/util/http_util.go
index eff282bab..135d10c45 100644
--- a/weed/util/http_util.go
+++ b/weed/util/http_util.go
@@ -313,7 +313,7 @@ func ReadUrlAsStream(fileUrl string, cipherKey []byte, isContentGzipped bool, is
}
func readEncryptedUrl(fileUrl string, cipherKey []byte, isContentCompressed bool, isFullChunk bool, offset int64, size int, fn func(data []byte)) (bool, error) {
- encryptedData, retryable, err := FastGet(fileUrl)
+ encryptedData, retryable, err := Get(fileUrl)
if err != nil {
return retryable, fmt.Errorf("fetch %s: %v", fileUrl, err)
}
diff --git a/weed/util/limiter.go b/weed/util/limiter.go
index 91499632c..2debaaa85 100644
--- a/weed/util/limiter.go
+++ b/weed/util/limiter.go
@@ -1,5 +1,12 @@
package util
+import (
+ "math/rand"
+ "reflect"
+ "sync"
+ "sync/atomic"
+)
+
// initial version comes from https://github.com/korovkin/limiter/blob/master/limiter.go
// LimitedConcurrentExecutor object
@@ -38,3 +45,70 @@ func (c *LimitedConcurrentExecutor) Execute(job func()) {
job()
}()
}
+
+// a different implementation, but somehow more "conservative"
+type OperationRequest func()
+
+type LimitedOutOfOrderProcessor struct {
+ processorSlots uint32
+ processors []chan OperationRequest
+ processorLimit int32
+ processorLimitCond *sync.Cond
+ currentProcessor int32
+}
+
+func NewLimitedOutOfOrderProcessor(limit int32) (c *LimitedOutOfOrderProcessor) {
+
+ processorSlots := uint32(32)
+ c = &LimitedOutOfOrderProcessor{
+ processorSlots: processorSlots,
+ processors: make([]chan OperationRequest, processorSlots),
+ processorLimit: limit,
+ processorLimitCond: sync.NewCond(new(sync.Mutex)),
+ }
+
+ for i := 0; i < int(processorSlots); i++ {
+ c.processors[i] = make(chan OperationRequest)
+ }
+
+ cases := make([]reflect.SelectCase, processorSlots)
+ for i, ch := range c.processors {
+ cases[i] = reflect.SelectCase{Dir: reflect.SelectRecv, Chan: reflect.ValueOf(ch)}
+ }
+
+ go func() {
+ for {
+ _, value, ok := reflect.Select(cases)
+ if !ok {
+ continue
+ }
+
+ request := value.Interface().(OperationRequest)
+
+ if c.processorLimit > 0 {
+ c.processorLimitCond.L.Lock()
+ for atomic.LoadInt32(&c.currentProcessor) > c.processorLimit {
+ c.processorLimitCond.Wait()
+ }
+ atomic.AddInt32(&c.currentProcessor, 1)
+ c.processorLimitCond.L.Unlock()
+ }
+
+ go func() {
+ if c.processorLimit > 0 {
+ defer atomic.AddInt32(&c.currentProcessor, -1)
+ defer c.processorLimitCond.Signal()
+ }
+ request()
+ }()
+
+ }
+ }()
+
+ return c
+}
+
+func (c *LimitedOutOfOrderProcessor) Execute(request OperationRequest) {
+ index := rand.Uint32() % c.processorSlots
+ c.processors[index] <- request
+}
diff --git a/weed/wdclient/net2/base_connection_pool.go b/weed/wdclient/net2/base_connection_pool.go
new file mode 100644
index 000000000..5cc037d0f
--- /dev/null
+++ b/weed/wdclient/net2/base_connection_pool.go
@@ -0,0 +1,159 @@
+package net2
+
+import (
+ "net"
+ "strings"
+ "time"
+
+ rp "github.com/chrislusf/seaweedfs/weed/wdclient/resource_pool"
+)
+
+const defaultDialTimeout = 1 * time.Second
+
+func defaultDialFunc(network string, address string) (net.Conn, error) {
+ return net.DialTimeout(network, address, defaultDialTimeout)
+}
+
+func parseResourceLocation(resourceLocation string) (
+ network string,
+ address string) {
+
+ idx := strings.Index(resourceLocation, " ")
+ if idx >= 0 {
+ return resourceLocation[:idx], resourceLocation[idx+1:]
+ }
+
+ return "", resourceLocation
+}
+
+// A thin wrapper around the underlying resource pool.
+type connectionPoolImpl struct {
+ options ConnectionOptions
+
+ pool rp.ResourcePool
+}
+
+// This returns a connection pool where all connections are connected
+// to the same (network, address)
+func newBaseConnectionPool(
+ options ConnectionOptions,
+ createPool func(rp.Options) rp.ResourcePool) ConnectionPool {
+
+ dial := options.Dial
+ if dial == nil {
+ dial = defaultDialFunc
+ }
+
+ openFunc := func(loc string) (interface{}, error) {
+ network, address := parseResourceLocation(loc)
+ return dial(network, address)
+ }
+
+ closeFunc := func(handle interface{}) error {
+ return handle.(net.Conn).Close()
+ }
+
+ poolOptions := rp.Options{
+ MaxActiveHandles: options.MaxActiveConnections,
+ MaxIdleHandles: options.MaxIdleConnections,
+ MaxIdleTime: options.MaxIdleTime,
+ OpenMaxConcurrency: options.DialMaxConcurrency,
+ Open: openFunc,
+ Close: closeFunc,
+ NowFunc: options.NowFunc,
+ }
+
+ return &connectionPoolImpl{
+ options: options,
+ pool: createPool(poolOptions),
+ }
+}
+
+// This returns a connection pool where all connections are connected
+// to the same (network, address)
+func NewSimpleConnectionPool(options ConnectionOptions) ConnectionPool {
+ return newBaseConnectionPool(options, rp.NewSimpleResourcePool)
+}
+
+// This returns a connection pool that manages multiple (network, address)
+// entries. The connections to each (network, address) entry acts
+// independently. For example ("tcp", "localhost:11211") could act as memcache
+// shard 0 and ("tcp", "localhost:11212") could act as memcache shard 1.
+func NewMultiConnectionPool(options ConnectionOptions) ConnectionPool {
+ return newBaseConnectionPool(
+ options,
+ func(poolOptions rp.Options) rp.ResourcePool {
+ return rp.NewMultiResourcePool(poolOptions, nil)
+ })
+}
+
+// See ConnectionPool for documentation.
+func (p *connectionPoolImpl) NumActive() int32 {
+ return p.pool.NumActive()
+}
+
+// See ConnectionPool for documentation.
+func (p *connectionPoolImpl) ActiveHighWaterMark() int32 {
+ return p.pool.ActiveHighWaterMark()
+}
+
+// This returns the number of alive idle connections. This method is not part
+// of ConnectionPool's API. It is used only for testing.
+func (p *connectionPoolImpl) NumIdle() int {
+ return p.pool.NumIdle()
+}
+
+// BaseConnectionPool can only register a single (network, address) entry.
+// Register should be call before any Get calls.
+func (p *connectionPoolImpl) Register(network string, address string) error {
+ return p.pool.Register(network + " " + address)
+}
+
+// BaseConnectionPool has nothing to do on Unregister.
+func (p *connectionPoolImpl) Unregister(network string, address string) error {
+ return nil
+}
+
+func (p *connectionPoolImpl) ListRegistered() []NetworkAddress {
+ result := make([]NetworkAddress, 0, 1)
+ for _, location := range p.pool.ListRegistered() {
+ network, address := parseResourceLocation(location)
+
+ result = append(
+ result,
+ NetworkAddress{
+ Network: network,
+ Address: address,
+ })
+ }
+ return result
+}
+
+// This gets an active connection from the connection pool. Note that network
+// and address arguments are ignored (The connections with point to the
+// network/address provided by the first Register call).
+func (p *connectionPoolImpl) Get(
+ network string,
+ address string) (ManagedConn, error) {
+
+ handle, err := p.pool.Get(network + " " + address)
+ if err != nil {
+ return nil, err
+ }
+ return NewManagedConn(network, address, handle, p, p.options), nil
+}
+
+// See ConnectionPool for documentation.
+func (p *connectionPoolImpl) Release(conn ManagedConn) error {
+ return conn.ReleaseConnection()
+}
+
+// See ConnectionPool for documentation.
+func (p *connectionPoolImpl) Discard(conn ManagedConn) error {
+ return conn.DiscardConnection()
+}
+
+// See ConnectionPool for documentation.
+func (p *connectionPoolImpl) EnterLameDuckMode() {
+ p.pool.EnterLameDuckMode()
+}
diff --git a/weed/wdclient/net2/connection_pool.go b/weed/wdclient/net2/connection_pool.go
new file mode 100644
index 000000000..5b8d4d232
--- /dev/null
+++ b/weed/wdclient/net2/connection_pool.go
@@ -0,0 +1,97 @@
+package net2
+
+import (
+ "net"
+ "time"
+)
+
+type ConnectionOptions struct {
+ // The maximum number of connections that can be active per host at any
+ // given time (A non-positive value indicates the number of connections
+ // is unbounded).
+ MaxActiveConnections int32
+
+ // The maximum number of idle connections per host that are kept alive by
+ // the connection pool.
+ MaxIdleConnections uint32
+
+ // The maximum amount of time an idle connection can alive (if specified).
+ MaxIdleTime *time.Duration
+
+ // This limits the number of concurrent Dial calls (there's no limit when
+ // DialMaxConcurrency is non-positive).
+ DialMaxConcurrency int
+
+ // Dial specifies the dial function for creating network connections.
+ // If Dial is nil, net.DialTimeout is used, with timeout set to 1 second.
+ Dial func(network string, address string) (net.Conn, error)
+
+ // This specifies the now time function. When the function is non-nil, the
+ // connection pool will use the specified function instead of time.Now to
+ // generate the current time.
+ NowFunc func() time.Time
+
+ // This specifies the timeout for any Read() operation.
+ // Note that setting this to 0 (i.e. not setting it) will make
+ // read operations block indefinitely.
+ ReadTimeout time.Duration
+
+ // This specifies the timeout for any Write() operation.
+ // Note that setting this to 0 (i.e. not setting it) will make
+ // write operations block indefinitely.
+ WriteTimeout time.Duration
+}
+
+func (o ConnectionOptions) getCurrentTime() time.Time {
+ if o.NowFunc == nil {
+ return time.Now()
+ } else {
+ return o.NowFunc()
+ }
+}
+
+// A generic interface for managed connection pool. All connection pool
+// implementations must be threadsafe.
+type ConnectionPool interface {
+ // This returns the number of active connections that are on loan.
+ NumActive() int32
+
+ // This returns the highest number of active connections for the entire
+ // lifetime of the pool.
+ ActiveHighWaterMark() int32
+
+ // This returns the number of idle connections that are in the pool.
+ NumIdle() int
+
+ // This associates (network, address) to the connection pool; afterwhich,
+ // the user can get connections to (network, address).
+ Register(network string, address string) error
+
+ // This dissociate (network, address) from the connection pool;
+ // afterwhich, the user can no longer get connections to
+ // (network, address).
+ Unregister(network string, address string) error
+
+ // This returns the list of registered (network, address) entries.
+ ListRegistered() []NetworkAddress
+
+ // This gets an active connection from the connection pool. The connection
+ // will remain active until one of the following is called:
+ // 1. conn.ReleaseConnection()
+ // 2. conn.DiscardConnection()
+ // 3. pool.Release(conn)
+ // 4. pool.Discard(conn)
+ Get(network string, address string) (ManagedConn, error)
+
+ // This releases an active connection back to the connection pool.
+ Release(conn ManagedConn) error
+
+ // This discards an active connection from the connection pool.
+ Discard(conn ManagedConn) error
+
+ // Enter the connection pool into lame duck mode. The connection pool
+ // will no longer return connections, and all idle connections are closed
+ // immediately (including active connections that are released back to the
+ // pool afterward).
+ EnterLameDuckMode()
+}
diff --git a/weed/wdclient/net2/doc.go b/weed/wdclient/net2/doc.go
new file mode 100644
index 000000000..fd1c6323d
--- /dev/null
+++ b/weed/wdclient/net2/doc.go
@@ -0,0 +1,6 @@
+// net2 is a collection of functions meant to supplement the capabilities
+// provided by the standard "net" package.
+package net2
+
+// copied from https://github.com/dropbox/godropbox/tree/master/net2
+// removed other dependencies
diff --git a/weed/wdclient/net2/ip.go b/weed/wdclient/net2/ip.go
new file mode 100644
index 000000000..ff5e3b24e
--- /dev/null
+++ b/weed/wdclient/net2/ip.go
@@ -0,0 +1,177 @@
+package net2
+
+import (
+ "fmt"
+ "log"
+ "net"
+ "os"
+ "strings"
+ "sync"
+)
+
+var myHostname string
+var myHostnameOnce sync.Once
+
+// Like os.Hostname but caches first successful result, making it cheap to call it
+// over and over.
+// It will also crash whole process if fetching Hostname fails!
+func MyHostname() string {
+ myHostnameOnce.Do(func() {
+ var err error
+ myHostname, err = os.Hostname()
+ if err != nil {
+ log.Fatal(err)
+ }
+ })
+ return myHostname
+}
+
+var myIp4 *net.IPAddr
+var myIp4Once sync.Once
+
+// Resolves `MyHostname()` to an Ip4 address. Caches first successful result, making it
+// cheap to call it over and over.
+// It will also crash whole process if resolving the IP fails!
+func MyIp4() *net.IPAddr {
+ myIp4Once.Do(func() {
+ var err error
+ myIp4, err = net.ResolveIPAddr("ip4", MyHostname())
+ if err != nil {
+ log.Fatal(err)
+ }
+ })
+ return myIp4
+}
+
+var myIp6 *net.IPAddr
+var myIp6Once sync.Once
+
+// Resolves `MyHostname()` to an Ip6 address. Caches first successful result, making it
+// cheap to call it over and over.
+// It will also crash whole process if resolving the IP fails!
+func MyIp6() *net.IPAddr {
+ myIp6Once.Do(func() {
+ var err error
+ myIp6, err = net.ResolveIPAddr("ip6", MyHostname())
+ if err != nil {
+ log.Fatal(err)
+ }
+ })
+ return myIp6
+}
+
+// This returns the list of local ip addresses which other hosts can connect
+// to (NOTE: Loopback ip is ignored).
+// Also resolves Hostname to an address and adds it to the list too, so
+// IPs from /etc/hosts can work too.
+func GetLocalIPs() ([]*net.IP, error) {
+ hostname, err := os.Hostname()
+ if err != nil {
+ return nil, fmt.Errorf("Failed to lookup hostname: %v", err)
+ }
+ // Resolves IP Address from Hostname, this way overrides in /etc/hosts
+ // can work too for IP resolution.
+ ipInfo, err := net.ResolveIPAddr("ip4", hostname)
+ if err != nil {
+ return nil, fmt.Errorf("Failed to resolve ip: %v", err)
+ }
+ ips := []*net.IP{&ipInfo.IP}
+
+ // TODO(zviad): Is rest of the code really necessary?
+ addrs, err := net.InterfaceAddrs()
+ if err != nil {
+ return nil, fmt.Errorf("Failed to get interface addresses: %v", err)
+ }
+ for _, addr := range addrs {
+ ipnet, ok := addr.(*net.IPNet)
+ if !ok {
+ continue
+ }
+
+ if ipnet.IP.IsLoopback() {
+ continue
+ }
+
+ ips = append(ips, &ipnet.IP)
+ }
+ return ips, nil
+}
+
+var localhostIPNets []*net.IPNet
+
+func init() {
+ for _, mask := range []string{"127.0.0.1/8", "::1/128"} {
+ _, ipnet, err := net.ParseCIDR(mask)
+ if err != nil {
+ panic(err)
+ }
+ localhostIPNets = append(localhostIPNets, ipnet)
+ }
+}
+
+func IsLocalhostIp(ipStr string) bool {
+ ip := net.ParseIP(ipStr)
+ if ip == nil {
+ return false
+ }
+ for _, ipnet := range localhostIPNets {
+ if ipnet.Contains(ip) {
+ return true
+ }
+ }
+ return false
+}
+
+// Given a host string, return true if the host is an ip (v4/v6) localhost.
+func IsLocalhost(host string) bool {
+ return IsLocalhostIp(host) ||
+ host == "localhost" ||
+ host == "ip6-localhost" ||
+ host == "ipv6-localhost"
+}
+
+// Resolves hostnames in addresses to actual IP4 addresses. Skips all invalid addresses
+// and all addresses that can't be resolved.
+// `addrs` are assumed to be of form: ["<hostname>:<port>", ...]
+// Returns an error in addition to resolved addresses if not all resolutions succeed.
+func ResolveIP4s(addrs []string) ([]string, error) {
+ resolvedAddrs := make([]string, 0, len(addrs))
+ var lastErr error
+
+ for _, server := range addrs {
+ hostPort := strings.Split(server, ":")
+ if len(hostPort) != 2 {
+ lastErr = fmt.Errorf("Skipping invalid address: %s", server)
+ continue
+ }
+
+ ip, err := net.ResolveIPAddr("ip4", hostPort[0])
+ if err != nil {
+ lastErr = err
+ continue
+ }
+ resolvedAddrs = append(resolvedAddrs, ip.IP.String()+":"+hostPort[1])
+ }
+ return resolvedAddrs, lastErr
+}
+
+func LookupValidAddrs() (map[string]bool, error) {
+ hostName, err := os.Hostname()
+ if err != nil {
+ return nil, err
+ }
+ addrs, err := net.LookupHost(hostName)
+ if err != nil {
+ return nil, err
+ }
+ validAddrs := make(map[string]bool)
+ validAddrs[hostName] = true
+ for _, addr := range addrs {
+ validAddrs[addr] = true
+ }
+ // Special case localhost/127.0.0.1 so that this works on devVMs. It should
+ // have no affect in production.
+ validAddrs["127.0.0.1"] = true
+ validAddrs["localhost"] = true
+ return validAddrs, nil
+}
diff --git a/weed/wdclient/net2/managed_connection.go b/weed/wdclient/net2/managed_connection.go
new file mode 100644
index 000000000..a886210d1
--- /dev/null
+++ b/weed/wdclient/net2/managed_connection.go
@@ -0,0 +1,185 @@
+package net2
+
+import (
+ "fmt"
+ "net"
+ "time"
+
+ "errors"
+ "github.com/chrislusf/seaweedfs/weed/wdclient/resource_pool"
+)
+
+// Dial's arguments.
+type NetworkAddress struct {
+ Network string
+ Address string
+}
+
+// A connection managed by a connection pool. NOTE: SetDeadline,
+// SetReadDeadline and SetWriteDeadline are disabled for managed connections.
+// (The deadlines are set by the connection pool).
+type ManagedConn interface {
+ net.Conn
+
+ // This returns the original (network, address) entry used for creating
+ // the connection.
+ Key() NetworkAddress
+
+ // This returns the underlying net.Conn implementation.
+ RawConn() net.Conn
+
+ // This returns the connection pool which owns this connection.
+ Owner() ConnectionPool
+
+ // This indictes a user is done with the connection and releases the
+ // connection back to the connection pool.
+ ReleaseConnection() error
+
+ // This indicates the connection is an invalid state, and that the
+ // connection should be discarded from the connection pool.
+ DiscardConnection() error
+}
+
+// A physical implementation of ManagedConn
+type managedConnImpl struct {
+ addr NetworkAddress
+ handle resource_pool.ManagedHandle
+ pool ConnectionPool
+ options ConnectionOptions
+}
+
+// This creates a managed connection wrapper.
+func NewManagedConn(
+ network string,
+ address string,
+ handle resource_pool.ManagedHandle,
+ pool ConnectionPool,
+ options ConnectionOptions) ManagedConn {
+
+ addr := NetworkAddress{
+ Network: network,
+ Address: address,
+ }
+
+ return &managedConnImpl{
+ addr: addr,
+ handle: handle,
+ pool: pool,
+ options: options,
+ }
+}
+
+func (c *managedConnImpl) rawConn() (net.Conn, error) {
+ h, err := c.handle.Handle()
+ return h.(net.Conn), err
+}
+
+// See ManagedConn for documentation.
+func (c *managedConnImpl) RawConn() net.Conn {
+ h, _ := c.handle.Handle()
+ return h.(net.Conn)
+}
+
+// See ManagedConn for documentation.
+func (c *managedConnImpl) Key() NetworkAddress {
+ return c.addr
+}
+
+// See ManagedConn for documentation.
+func (c *managedConnImpl) Owner() ConnectionPool {
+ return c.pool
+}
+
+// See ManagedConn for documentation.
+func (c *managedConnImpl) ReleaseConnection() error {
+ return c.handle.Release()
+}
+
+// See ManagedConn for documentation.
+func (c *managedConnImpl) DiscardConnection() error {
+ return c.handle.Discard()
+}
+
+// See net.Conn for documentation
+func (c *managedConnImpl) Read(b []byte) (n int, err error) {
+ conn, err := c.rawConn()
+ if err != nil {
+ return 0, err
+ }
+
+ if c.options.ReadTimeout > 0 {
+ deadline := c.options.getCurrentTime().Add(c.options.ReadTimeout)
+ _ = conn.SetReadDeadline(deadline)
+ }
+ n, err = conn.Read(b)
+ if err != nil {
+ var localAddr string
+ if conn.LocalAddr() != nil {
+ localAddr = conn.LocalAddr().String()
+ } else {
+ localAddr = "(nil)"
+ }
+
+ var remoteAddr string
+ if conn.RemoteAddr() != nil {
+ remoteAddr = conn.RemoteAddr().String()
+ } else {
+ remoteAddr = "(nil)"
+ }
+ err = fmt.Errorf("Read error from host: %s <-> %s: %v", localAddr, remoteAddr, err)
+ }
+ return
+}
+
+// See net.Conn for documentation
+func (c *managedConnImpl) Write(b []byte) (n int, err error) {
+ conn, err := c.rawConn()
+ if err != nil {
+ return 0, err
+ }
+
+ if c.options.WriteTimeout > 0 {
+ deadline := c.options.getCurrentTime().Add(c.options.WriteTimeout)
+ _ = conn.SetWriteDeadline(deadline)
+ }
+ n, err = conn.Write(b)
+ if err != nil {
+ err = fmt.Errorf("Write error: %v", err)
+ }
+ return
+}
+
+// See net.Conn for documentation
+func (c *managedConnImpl) Close() error {
+ return c.handle.Discard()
+}
+
+// See net.Conn for documentation
+func (c *managedConnImpl) LocalAddr() net.Addr {
+ conn, _ := c.rawConn()
+ return conn.LocalAddr()
+}
+
+// See net.Conn for documentation
+func (c *managedConnImpl) RemoteAddr() net.Addr {
+ conn, _ := c.rawConn()
+ return conn.RemoteAddr()
+}
+
+// SetDeadline is disabled for managed connection (The deadline is set by
+// us, with respect to the read/write timeouts specified in ConnectionOptions).
+func (c *managedConnImpl) SetDeadline(t time.Time) error {
+ return errors.New("Cannot set deadline for managed connection")
+}
+
+// SetReadDeadline is disabled for managed connection (The deadline is set by
+// us with respect to the read timeout specified in ConnectionOptions).
+func (c *managedConnImpl) SetReadDeadline(t time.Time) error {
+ return errors.New("Cannot set read deadline for managed connection")
+}
+
+// SetWriteDeadline is disabled for managed connection (The deadline is set by
+// us with respect to the write timeout specified in ConnectionOptions).
+func (c *managedConnImpl) SetWriteDeadline(t time.Time) error {
+ return errors.New("Cannot set write deadline for managed connection")
+}
diff --git a/weed/wdclient/net2/port.go b/weed/wdclient/net2/port.go
new file mode 100644
index 000000000..f83adba28
--- /dev/null
+++ b/weed/wdclient/net2/port.go
@@ -0,0 +1,19 @@
+package net2
+
+import (
+ "net"
+ "strconv"
+)
+
+// Returns the port information.
+func GetPort(addr net.Addr) (int, error) {
+ _, lport, err := net.SplitHostPort(addr.String())
+ if err != nil {
+ return -1, err
+ }
+ lportInt, err := strconv.Atoi(lport)
+ if err != nil {
+ return -1, err
+ }
+ return lportInt, nil
+}
diff --git a/weed/wdclient/resource_pool/doc.go b/weed/wdclient/resource_pool/doc.go
new file mode 100644
index 000000000..b8b3f92fa
--- /dev/null
+++ b/weed/wdclient/resource_pool/doc.go
@@ -0,0 +1,5 @@
+// A generic resource pool for managing resources such as network connections.
+package resource_pool
+
+// copied from https://github.com/dropbox/godropbox/tree/master/resource_pool
+// removed other dependencies
diff --git a/weed/wdclient/resource_pool/managed_handle.go b/weed/wdclient/resource_pool/managed_handle.go
new file mode 100644
index 000000000..e1d82ca7b
--- /dev/null
+++ b/weed/wdclient/resource_pool/managed_handle.go
@@ -0,0 +1,97 @@
+package resource_pool
+
+import (
+ "sync/atomic"
+
+ "errors"
+)
+
+// A resource handle managed by a resource pool.
+type ManagedHandle interface {
+ // This returns the handle's resource location.
+ ResourceLocation() string
+
+ // This returns the underlying resource handle (or error if the handle
+ // is no longer active).
+ Handle() (interface{}, error)
+
+ // This returns the resource pool which owns this handle.
+ Owner() ResourcePool
+
+ // The releases the underlying resource handle to the caller and marks the
+ // managed handle as inactive. The caller is responsible for cleaning up
+ // the released handle. This returns nil if the managed handle no longer
+ // owns the resource.
+ ReleaseUnderlyingHandle() interface{}
+
+ // This indictes a user is done with the handle and releases the handle
+ // back to the resource pool.
+ Release() error
+
+ // This indicates the handle is an invalid state, and that the
+ // connection should be discarded from the connection pool.
+ Discard() error
+}
+
+// A physical implementation of ManagedHandle
+type managedHandleImpl struct {
+ location string
+ handle interface{}
+ pool ResourcePool
+ isActive int32 // atomic bool
+ options Options
+}
+
+// This creates a managed handle wrapper.
+func NewManagedHandle(
+ resourceLocation string,
+ handle interface{},
+ pool ResourcePool,
+ options Options) ManagedHandle {
+
+ h := &managedHandleImpl{
+ location: resourceLocation,
+ handle: handle,
+ pool: pool,
+ options: options,
+ }
+ atomic.StoreInt32(&h.isActive, 1)
+
+ return h
+}
+
+// See ManagedHandle for documentation.
+func (c *managedHandleImpl) ResourceLocation() string {
+ return c.location
+}
+
+// See ManagedHandle for documentation.
+func (c *managedHandleImpl) Handle() (interface{}, error) {
+ if atomic.LoadInt32(&c.isActive) == 0 {
+ return c.handle, errors.New("Resource handle is no longer valid")
+ }
+ return c.handle, nil
+}
+
+// See ManagedHandle for documentation.
+func (c *managedHandleImpl) Owner() ResourcePool {
+ return c.pool
+}
+
+// See ManagedHandle for documentation.
+func (c *managedHandleImpl) ReleaseUnderlyingHandle() interface{} {
+ if atomic.CompareAndSwapInt32(&c.isActive, 1, 0) {
+ return c.handle
+ }
+ return nil
+}
+
+// See ManagedHandle for documentation.
+func (c *managedHandleImpl) Release() error {
+ return c.pool.Release(c)
+}
+
+// See ManagedHandle for documentation.
+func (c *managedHandleImpl) Discard() error {
+ return c.pool.Discard(c)
+}
diff --git a/weed/wdclient/resource_pool/multi_resource_pool.go b/weed/wdclient/resource_pool/multi_resource_pool.go
new file mode 100644
index 000000000..9ac25526d
--- /dev/null
+++ b/weed/wdclient/resource_pool/multi_resource_pool.go
@@ -0,0 +1,200 @@
+package resource_pool
+
+import (
+ "fmt"
+ "sync"
+
+ "errors"
+)
+
+// A resource pool implementation that manages multiple resource location
+// entries. The handles to each resource location entry acts independently.
+// For example "tcp localhost:11211" could act as memcache
+// shard 0 and "tcp localhost:11212" could act as memcache shard 1.
+type multiResourcePool struct {
+ options Options
+
+ createPool func(Options) ResourcePool
+
+ rwMutex sync.RWMutex
+ isLameDuck bool // guarded by rwMutex
+ // NOTE: the locationPools is guarded by rwMutex, but the pool entries
+ // are not.
+ locationPools map[string]ResourcePool
+}
+
+// This returns a MultiResourcePool, which manages multiple
+// resource location entries. The handles to each resource location
+// entry acts independently.
+//
+// When createPool is nil, NewSimpleResourcePool is used as default.
+func NewMultiResourcePool(
+ options Options,
+ createPool func(Options) ResourcePool) ResourcePool {
+
+ if createPool == nil {
+ createPool = NewSimpleResourcePool
+ }
+
+ return &multiResourcePool{
+ options: options,
+ createPool: createPool,
+ rwMutex: sync.RWMutex{},
+ isLameDuck: false,
+ locationPools: make(map[string]ResourcePool),
+ }
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) NumActive() int32 {
+ total := int32(0)
+
+ p.rwMutex.RLock()
+ defer p.rwMutex.RUnlock()
+
+ for _, pool := range p.locationPools {
+ total += pool.NumActive()
+ }
+ return total
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) ActiveHighWaterMark() int32 {
+ high := int32(0)
+
+ p.rwMutex.RLock()
+ defer p.rwMutex.RUnlock()
+
+ for _, pool := range p.locationPools {
+ val := pool.ActiveHighWaterMark()
+ if val > high {
+ high = val
+ }
+ }
+ return high
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) NumIdle() int {
+ total := 0
+
+ p.rwMutex.RLock()
+ defer p.rwMutex.RUnlock()
+
+ for _, pool := range p.locationPools {
+ total += pool.NumIdle()
+ }
+ return total
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) Register(resourceLocation string) error {
+ if resourceLocation == "" {
+ return errors.New("Registering invalid resource location")
+ }
+
+ p.rwMutex.Lock()
+ defer p.rwMutex.Unlock()
+
+ if p.isLameDuck {
+ return fmt.Errorf(
+ "Cannot register %s to lame duck resource pool",
+ resourceLocation)
+ }
+
+ if _, inMap := p.locationPools[resourceLocation]; inMap {
+ return nil
+ }
+
+ pool := p.createPool(p.options)
+ if err := pool.Register(resourceLocation); err != nil {
+ return err
+ }
+
+ p.locationPools[resourceLocation] = pool
+ return nil
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) Unregister(resourceLocation string) error {
+ p.rwMutex.Lock()
+ defer p.rwMutex.Unlock()
+
+ if pool, inMap := p.locationPools[resourceLocation]; inMap {
+ _ = pool.Unregister("")
+ pool.EnterLameDuckMode()
+ delete(p.locationPools, resourceLocation)
+ }
+ return nil
+}
+
+func (p *multiResourcePool) ListRegistered() []string {
+ p.rwMutex.RLock()
+ defer p.rwMutex.RUnlock()
+
+ result := make([]string, 0, len(p.locationPools))
+ for key, _ := range p.locationPools {
+ result = append(result, key)
+ }
+
+ return result
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) Get(
+ resourceLocation string) (ManagedHandle, error) {
+
+ pool := p.getPool(resourceLocation)
+ if pool == nil {
+ return nil, fmt.Errorf(
+ "%s is not registered in the resource pool",
+ resourceLocation)
+ }
+ return pool.Get(resourceLocation)
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) Release(handle ManagedHandle) error {
+ pool := p.getPool(handle.ResourceLocation())
+ if pool == nil {
+ return errors.New(
+ "Resource pool cannot take control of a handle owned " +
+ "by another resource pool")
+ }
+
+ return pool.Release(handle)
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) Discard(handle ManagedHandle) error {
+ pool := p.getPool(handle.ResourceLocation())
+ if pool == nil {
+ return errors.New(
+ "Resource pool cannot take control of a handle owned " +
+ "by another resource pool")
+ }
+
+ return pool.Discard(handle)
+}
+
+// See ResourcePool for documentation.
+func (p *multiResourcePool) EnterLameDuckMode() {
+ p.rwMutex.Lock()
+ defer p.rwMutex.Unlock()
+
+ p.isLameDuck = true
+
+ for _, pool := range p.locationPools {
+ pool.EnterLameDuckMode()
+ }
+}
+
+func (p *multiResourcePool) getPool(resourceLocation string) ResourcePool {
+ p.rwMutex.RLock()
+ defer p.rwMutex.RUnlock()
+
+ if pool, inMap := p.locationPools[resourceLocation]; inMap {
+ return pool
+ }
+ return nil
+}
diff --git a/weed/wdclient/resource_pool/resource_pool.go b/weed/wdclient/resource_pool/resource_pool.go
new file mode 100644
index 000000000..26c433f50
--- /dev/null
+++ b/weed/wdclient/resource_pool/resource_pool.go
@@ -0,0 +1,96 @@
+package resource_pool
+
+import (
+ "time"
+)
+
+type Options struct {
+ // The maximum number of active resource handles per resource location. (A
+ // non-positive value indicates the number of active resource handles is
+ // unbounded).
+ MaxActiveHandles int32
+
+ // The maximum number of idle resource handles per resource location that
+ // are kept alive by the resource pool.
+ MaxIdleHandles uint32
+
+ // The maximum amount of time an idle resource handle can remain alive (if
+ // specified).
+ MaxIdleTime *time.Duration
+
+ // This limits the number of concurrent Open calls (there's no limit when
+ // OpenMaxConcurrency is non-positive).
+ OpenMaxConcurrency int
+
+ // This function creates a resource handle (e.g., a connection) for a
+ // resource location. The function must be thread-safe.
+ Open func(resourceLocation string) (
+ handle interface{},
+ err error)
+
+ // This function destroys a resource handle and performs the necessary
+ // cleanup to free up resources. The function must be thread-safe.
+ Close func(handle interface{}) error
+
+ // This specifies the now time function. When the function is non-nil, the
+ // resource pool will use the specified function instead of time.Now to
+ // generate the current time.
+ NowFunc func() time.Time
+}
+
+func (o Options) getCurrentTime() time.Time {
+ if o.NowFunc == nil {
+ return time.Now()
+ } else {
+ return o.NowFunc()
+ }
+}
+
+// A generic interface for managed resource pool. All resource pool
+// implementations must be threadsafe.
+type ResourcePool interface {
+ // This returns the number of active resource handles.
+ NumActive() int32
+
+ // This returns the highest number of actives handles for the entire
+ // lifetime of the pool. If the pool contains multiple sub-pools, the
+ // high water mark is the max of the sub-pools' high water marks.
+ ActiveHighWaterMark() int32
+
+ // This returns the number of alive idle handles. NOTE: This is only used
+ // for testing.
+ NumIdle() int
+
+ // This associates a resource location to the resource pool; afterwhich,
+ // the user can get resource handles for the resource location.
+ Register(resourceLocation string) error
+
+ // This dissociates a resource location from the resource pool; afterwhich,
+ // the user can no longer get resource handles for the resource location.
+ // If the given resource location corresponds to a sub-pool, the unregistered
+ // sub-pool will enter lame duck mode.
+ Unregister(resourceLocation string) error
+
+ // This returns the list of registered resource location entries.
+ ListRegistered() []string
+
+ // This gets an active resource handle from the resource pool. The
+ // handle will remain active until one of the following is called:
+ // 1. handle.Release()
+ // 2. handle.Discard()
+ // 3. pool.Release(handle)
+ // 4. pool.Discard(handle)
+ Get(key string) (ManagedHandle, error)
+
+ // This releases an active resource handle back to the resource pool.
+ Release(handle ManagedHandle) error
+
+ // This discards an active resource from the resource pool.
+ Discard(handle ManagedHandle) error
+
+ // Enter the resource pool into lame duck mode. The resource pool
+ // will no longer return resource handles, and all idle resource handles
+ // are closed immediately (including active resource handles that are
+ // released back to the pool afterward).
+ EnterLameDuckMode()
+}
diff --git a/weed/wdclient/resource_pool/semaphore.go b/weed/wdclient/resource_pool/semaphore.go
new file mode 100644
index 000000000..ff35d5bc5
--- /dev/null
+++ b/weed/wdclient/resource_pool/semaphore.go
@@ -0,0 +1,154 @@
+package resource_pool
+
+import (
+ "fmt"
+ "sync"
+ "sync/atomic"
+ "time"
+)
+
+type Semaphore interface {
+ // Increment the semaphore counter by one.
+ Release()
+
+ // Decrement the semaphore counter by one, and block if counter < 0
+ Acquire()
+
+ // Decrement the semaphore counter by one, and block if counter < 0
+ // Wait for up to the given duration. Returns true if did not timeout
+ TryAcquire(timeout time.Duration) bool
+}
+
+// A simple counting Semaphore.
+type boundedSemaphore struct {
+ slots chan struct{}
+}
+
+// Create a bounded semaphore. The count parameter must be a positive number.
+// NOTE: The bounded semaphore will panic if the user tries to Release
+// beyond the specified count.
+func NewBoundedSemaphore(count uint) Semaphore {
+ sem := &boundedSemaphore{
+ slots: make(chan struct{}, int(count)),
+ }
+ for i := 0; i < cap(sem.slots); i++ {
+ sem.slots <- struct{}{}
+ }
+ return sem
+}
+
+// Acquire returns on successful acquisition.
+func (sem *boundedSemaphore) Acquire() {
+ <-sem.slots
+}
+
+// TryAcquire returns true if it acquires a resource slot within the
+// timeout, false otherwise.
+func (sem *boundedSemaphore) TryAcquire(timeout time.Duration) bool {
+ if timeout > 0 {
+ // Wait until we get a slot or timeout expires.
+ tm := time.NewTimer(timeout)
+ defer tm.Stop()
+ select {
+ case <-sem.slots:
+ return true
+ case <-tm.C:
+ // Timeout expired. In very rare cases this might happen even if
+ // there is a slot available, e.g. GC pause after we create the timer
+ // and select randomly picked this one out of the two available channels.
+ // We should do one final immediate check below.
+ }
+ }
+
+ // Return true if we have a slot available immediately and false otherwise.
+ select {
+ case <-sem.slots:
+ return true
+ default:
+ return false
+ }
+}
+
+// Release the acquired semaphore. You must not release more than you
+// have acquired.
+func (sem *boundedSemaphore) Release() {
+ select {
+ case sem.slots <- struct{}{}:
+ default:
+ // slots is buffered. If a send blocks, it indicates a programming
+ // error.
+ panic(fmt.Errorf("too many releases for boundedSemaphore"))
+ }
+}
+
+// This returns an unbound counting semaphore with the specified initial count.
+// The semaphore counter can be arbitrary large (i.e., Release can be called
+// unlimited amount of times).
+//
+// NOTE: In general, users should use bounded semaphore since it is more
+// efficient than unbounded semaphore.
+func NewUnboundedSemaphore(initialCount int) Semaphore {
+ res := &unboundedSemaphore{
+ counter: int64(initialCount),
+ }
+ res.cond.L = &res.lock
+ return res
+}
+
+type unboundedSemaphore struct {
+ lock sync.Mutex
+ cond sync.Cond
+ counter int64
+}
+
+func (s *unboundedSemaphore) Release() {
+ s.lock.Lock()
+ s.counter += 1
+ if s.counter > 0 {
+ // Not broadcasting here since it's unlike we can satify all waiting
+ // goroutines. Instead, we will Signal again if there are left over
+ // quota after Acquire, in case of lost wakeups.
+ s.cond.Signal()
+ }
+ s.lock.Unlock()
+}
+
+func (s *unboundedSemaphore) Acquire() {
+ s.lock.Lock()
+ for s.counter < 1 {
+ s.cond.Wait()
+ }
+ s.counter -= 1
+ if s.counter > 0 {
+ s.cond.Signal()
+ }
+ s.lock.Unlock()
+}
+
+func (s *unboundedSemaphore) TryAcquire(timeout time.Duration) bool {
+ done := make(chan bool, 1)
+ // Gate used to communicate between the threads and decide what the result
+ // is. If the main thread decides, we have timed out, otherwise we succeed.
+ decided := new(int32)
+ atomic.StoreInt32(decided, 0)
+ go func() {
+ s.Acquire()
+ if atomic.SwapInt32(decided, 1) == 0 {
+ // Acquire won the race
+ done <- true
+ } else {
+ // If we already decided the result, and this thread did not win
+ s.Release()
+ }
+ }()
+ select {
+ case <-done:
+ return true
+ case <-time.After(timeout):
+ if atomic.SwapInt32(decided, 1) == 1 {
+ // The other thread already decided the result
+ return true
+ }
+ return false
+ }
+}
diff --git a/weed/wdclient/resource_pool/simple_resource_pool.go b/weed/wdclient/resource_pool/simple_resource_pool.go
new file mode 100644
index 000000000..b0c539100
--- /dev/null
+++ b/weed/wdclient/resource_pool/simple_resource_pool.go
@@ -0,0 +1,343 @@
+package resource_pool
+
+import (
+ "errors"
+ "fmt"
+ "sync"
+ "sync/atomic"
+ "time"
+)
+
+type idleHandle struct {
+ handle interface{}
+ keepUntil *time.Time
+}
+
+type TooManyHandles struct {
+ location string
+}
+
+func (t TooManyHandles) Error() string {
+ return fmt.Sprintf("Too many handles to %s", t.location)
+}
+
+type OpenHandleError struct {
+ location string
+ err error
+}
+
+func (o OpenHandleError) Error() string {
+ return fmt.Sprintf("Failed to open resource handle: %s (%v)", o.location, o.err)
+}
+
+// A resource pool implementation where all handles are associated to the
+// same resource location.
+type simpleResourcePool struct {
+ options Options
+
+ numActive *int32 // atomic counter
+
+ activeHighWaterMark *int32 // atomic / monotonically increasing value
+
+ openTokens Semaphore
+
+ mutex sync.Mutex
+ location string // guard by mutex
+ idleHandles []*idleHandle // guarded by mutex
+ isLameDuck bool // guarded by mutex
+}
+
+// This returns a SimpleResourcePool, where all handles are associated to a
+// single resource location.
+func NewSimpleResourcePool(options Options) ResourcePool {
+ numActive := new(int32)
+ atomic.StoreInt32(numActive, 0)
+
+ activeHighWaterMark := new(int32)
+ atomic.StoreInt32(activeHighWaterMark, 0)
+
+ var tokens Semaphore
+ if options.OpenMaxConcurrency > 0 {
+ tokens = NewBoundedSemaphore(uint(options.OpenMaxConcurrency))
+ }
+
+ return &simpleResourcePool{
+ location: "",
+ options: options,
+ numActive: numActive,
+ activeHighWaterMark: activeHighWaterMark,
+ openTokens: tokens,
+ mutex: sync.Mutex{},
+ idleHandles: make([]*idleHandle, 0, 0),
+ isLameDuck: false,
+ }
+}
+
+// See ResourcePool for documentation.
+func (p *simpleResourcePool) NumActive() int32 {
+ return atomic.LoadInt32(p.numActive)
+}
+
+// See ResourcePool for documentation.
+func (p *simpleResourcePool) ActiveHighWaterMark() int32 {
+ return atomic.LoadInt32(p.activeHighWaterMark)
+}
+
+// See ResourcePool for documentation.
+func (p *simpleResourcePool) NumIdle() int {
+ p.mutex.Lock()
+ defer p.mutex.Unlock()
+ return len(p.idleHandles)
+}
+
+// SimpleResourcePool can only register a single (network, address) entry.
+// Register should be call before any Get calls.
+func (p *simpleResourcePool) Register(resourceLocation string) error {
+ if resourceLocation == "" {
+ return errors.New("Invalid resource location")
+ }
+
+ p.mutex.Lock()
+ defer p.mutex.Unlock()
+
+ if p.isLameDuck {
+ return fmt.Errorf(
+ "cannot register %s to lame duck resource pool",
+ resourceLocation)
+ }
+
+ if p.location == "" {
+ p.location = resourceLocation
+ return nil
+ }
+ return errors.New("SimpleResourcePool can only register one location")
+}
+
+// SimpleResourcePool will enter lame duck mode upon calling Unregister.
+func (p *simpleResourcePool) Unregister(resourceLocation string) error {
+ p.EnterLameDuckMode()
+ return nil
+}
+
+func (p *simpleResourcePool) ListRegistered() []string {
+ p.mutex.Lock()
+ defer p.mutex.Unlock()
+
+ if p.location != "" {
+ return []string{p.location}
+ }
+ return []string{}
+}
+
+func (p *simpleResourcePool) getLocation() (string, error) {
+ p.mutex.Lock()
+ defer p.mutex.Unlock()
+
+ if p.location == "" {
+ return "", fmt.Errorf(
+ "resource location is not set for SimpleResourcePool")
+ }
+
+ if p.isLameDuck {
+ return "", fmt.Errorf(
+ "lame duck resource pool cannot return handles to %s",
+ p.location)
+ }
+
+ return p.location, nil
+}
+
+// This gets an active resource from the resource pool. Note that the
+// resourceLocation argument is ignored (The handles are associated to the
+// resource location provided by the first Register call).
+func (p *simpleResourcePool) Get(unused string) (ManagedHandle, error) {
+ activeCount := atomic.AddInt32(p.numActive, 1)
+ if p.options.MaxActiveHandles > 0 &&
+ activeCount > p.options.MaxActiveHandles {
+
+ atomic.AddInt32(p.numActive, -1)
+ return nil, TooManyHandles{p.location}
+ }
+
+ highest := atomic.LoadInt32(p.activeHighWaterMark)
+ for activeCount > highest &&
+ !atomic.CompareAndSwapInt32(
+ p.activeHighWaterMark,
+ highest,
+ activeCount) {
+
+ highest = atomic.LoadInt32(p.activeHighWaterMark)
+ }
+
+ if h := p.getIdleHandle(); h != nil {
+ return h, nil
+ }
+
+ location, err := p.getLocation()
+ if err != nil {
+ atomic.AddInt32(p.numActive, -1)
+ return nil, err
+ }
+
+ if p.openTokens != nil {
+ // Current implementation does not wait for tokens to become available.
+ // If that causes availability hits, we could increase the wait,
+ // similar to simple_pool.go.
+ if p.openTokens.TryAcquire(0) {
+ defer p.openTokens.Release()
+ } else {
+ // We could not immediately acquire a token.
+ // Instead of waiting
+ atomic.AddInt32(p.numActive, -1)
+ return nil, OpenHandleError{
+ p.location, errors.New("Open Error: reached OpenMaxConcurrency")}
+ }
+ }
+
+ handle, err := p.options.Open(location)
+ if err != nil {
+ atomic.AddInt32(p.numActive, -1)
+ return nil, OpenHandleError{p.location, err}
+ }
+
+ return NewManagedHandle(p.location, handle, p, p.options), nil
+}
+
+// See ResourcePool for documentation.
+func (p *simpleResourcePool) Release(handle ManagedHandle) error {
+ if pool, ok := handle.Owner().(*simpleResourcePool); !ok || pool != p {
+ return errors.New(
+ "Resource pool cannot take control of a handle owned " +
+ "by another resource pool")
+ }
+
+ h := handle.ReleaseUnderlyingHandle()
+ if h != nil {
+ // We can unref either before or after queuing the idle handle.
+ // The advantage of unref-ing before queuing is that there is
+ // a higher chance of successful Get when number of active handles
+ // is close to the limit (but potentially more handle creation).
+ // The advantage of queuing before unref-ing is that there's a
+ // higher chance of reusing handle (but potentially more Get failures).
+ atomic.AddInt32(p.numActive, -1)
+ p.queueIdleHandles(h)
+ }
+
+ return nil
+}
+
+// See ResourcePool for documentation.
+func (p *simpleResourcePool) Discard(handle ManagedHandle) error {
+ if pool, ok := handle.Owner().(*simpleResourcePool); !ok || pool != p {
+ return errors.New(
+ "Resource pool cannot take control of a handle owned " +
+ "by another resource pool")
+ }
+
+ h := handle.ReleaseUnderlyingHandle()
+ if h != nil {
+ atomic.AddInt32(p.numActive, -1)
+ if err := p.options.Close(h); err != nil {
+ return fmt.Errorf("failed to close resource handle: %v", err)
+ }
+ }
+ return nil
+}
+
+// See ResourcePool for documentation.
+func (p *simpleResourcePool) EnterLameDuckMode() {
+ p.mutex.Lock()
+
+ toClose := p.idleHandles
+ p.isLameDuck = true
+ p.idleHandles = []*idleHandle{}
+
+ p.mutex.Unlock()
+
+ p.closeHandles(toClose)
+}
+
+// This returns an idle resource, if there is one.
+func (p *simpleResourcePool) getIdleHandle() ManagedHandle {
+ var toClose []*idleHandle
+ defer func() {
+ // NOTE: Must keep the closure around to late bind the toClose slice.
+ p.closeHandles(toClose)
+ }()
+
+ now := p.options.getCurrentTime()
+
+ p.mutex.Lock()
+ defer p.mutex.Unlock()
+
+ var i int
+ for i = 0; i < len(p.idleHandles); i++ {
+ idle := p.idleHandles[i]
+ if idle.keepUntil == nil || now.Before(*idle.keepUntil) {
+ break
+ }
+ }
+ if i > 0 {
+ toClose = p.idleHandles[0:i]
+ }
+
+ if i < len(p.idleHandles) {
+ idle := p.idleHandles[i]
+ p.idleHandles = p.idleHandles[i+1:]
+ return NewManagedHandle(p.location, idle.handle, p, p.options)
+ }
+
+ if len(p.idleHandles) > 0 {
+ p.idleHandles = []*idleHandle{}
+ }
+ return nil
+}
+
+// This adds an idle resource to the pool.
+func (p *simpleResourcePool) queueIdleHandles(handle interface{}) {
+ var toClose []*idleHandle
+ defer func() {
+ // NOTE: Must keep the closure around to late bind the toClose slice.
+ p.closeHandles(toClose)
+ }()
+
+ now := p.options.getCurrentTime()
+ var keepUntil *time.Time
+ if p.options.MaxIdleTime != nil {
+ // NOTE: Assign to temp variable first to work around compiler bug
+ x := now.Add(*p.options.MaxIdleTime)
+ keepUntil = &x
+ }
+
+ p.mutex.Lock()
+ defer p.mutex.Unlock()
+
+ if p.isLameDuck {
+ toClose = []*idleHandle{
+ {handle: handle},
+ }
+ return
+ }
+
+ p.idleHandles = append(
+ p.idleHandles,
+ &idleHandle{
+ handle: handle,
+ keepUntil: keepUntil,
+ })
+
+ nIdleHandles := uint32(len(p.idleHandles))
+ if nIdleHandles > p.options.MaxIdleHandles {
+ handlesToClose := nIdleHandles - p.options.MaxIdleHandles
+ toClose = p.idleHandles[0:handlesToClose]
+ p.idleHandles = p.idleHandles[handlesToClose:nIdleHandles]
+ }
+}
+
+// Closes resources, at this point it is assumed that this resources
+// are no longer referenced from the main idleHandles slice.
+func (p *simpleResourcePool) closeHandles(handles []*idleHandle) {
+ for _, handle := range handles {
+ _ = p.options.Close(handle.handle)
+ }
+}
diff --git a/weed/wdclient/volume_tcp_client.go b/weed/wdclient/volume_tcp_client.go
new file mode 100644
index 000000000..afebd71eb
--- /dev/null
+++ b/weed/wdclient/volume_tcp_client.go
@@ -0,0 +1,97 @@
+package wdclient
+
+import (
+ "bufio"
+ "bytes"
+ "fmt"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb"
+ "github.com/chrislusf/seaweedfs/weed/util"
+ "github.com/chrislusf/seaweedfs/weed/wdclient/net2"
+ "io"
+ "net"
+ "time"
+)
+
+// VolumeTcpClient put/get/delete file chunks directly on volume servers without replication
+type VolumeTcpClient struct {
+ cp net2.ConnectionPool
+}
+
+type VolumeTcpConn struct {
+ net.Conn
+ bufWriter *bufio.Writer
+ bufReader *bufio.Reader
+}
+
+func NewVolumeTcpClient() *VolumeTcpClient {
+ MaxIdleTime := 10 * time.Second
+ return &VolumeTcpClient{
+ cp: net2.NewMultiConnectionPool(net2.ConnectionOptions{
+ MaxActiveConnections: 16,
+ MaxIdleConnections: 1,
+ MaxIdleTime: &MaxIdleTime,
+ DialMaxConcurrency: 0,
+ Dial: func(network string, address string) (net.Conn, error) {
+ conn, err := net.Dial(network, address)
+ return &VolumeTcpConn{
+ conn,
+ bufio.NewWriter(conn),
+ bufio.NewReader(conn),
+ }, err
+ },
+ NowFunc: nil,
+ ReadTimeout: 0,
+ WriteTimeout: 0,
+ }),
+ }
+}
+func (c *VolumeTcpClient) PutFileChunk(volumeServerAddress string, fileId string, fileSize uint32, fileReader io.Reader) (err error) {
+
+ tcpAddress, parseErr := pb.ParseServerAddress(volumeServerAddress, 20000)
+ if parseErr != nil {
+ return parseErr
+ }
+
+ c.cp.Register("tcp", tcpAddress)
+ tcpConn, getErr := c.cp.Get("tcp", tcpAddress)
+ if getErr != nil {
+ return fmt.Errorf("get connection to %s: %v", tcpAddress, getErr)
+ }
+ conn := tcpConn.RawConn().(*VolumeTcpConn)
+ defer func() {
+ if err != nil {
+ tcpConn.DiscardConnection()
+ } else {
+ tcpConn.ReleaseConnection()
+ }
+ }()
+
+ buf := []byte("+" + fileId + "\n")
+ _, err = conn.bufWriter.Write([]byte(buf))
+ if err != nil {
+ return
+ }
+ util.Uint32toBytes(buf[0:4], fileSize)
+ _, err = conn.bufWriter.Write(buf[0:4])
+ if err != nil {
+ return
+ }
+ _, err = io.Copy(conn.bufWriter, fileReader)
+ if err != nil {
+ return
+ }
+ conn.bufWriter.Write([]byte("!\n"))
+ conn.bufWriter.Flush()
+
+ ret, _, err := conn.bufReader.ReadLine()
+ if err != nil {
+ glog.V(0).Infof("upload by tcp: %v", err)
+ return
+ }
+ if !bytes.HasPrefix(ret, []byte("+OK")) {
+ glog.V(0).Infof("upload by tcp: %v", string(ret))
+ }
+
+ return nil
+}