mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
Merge pull request #2241 from chrislusf/add_remote_storage
WIP: remote storage
This commit is contained in:
commit
52fcce81c6
|
@ -92,6 +92,12 @@ message ListEntriesResponse {
|
|||
Entry entry = 1;
|
||||
}
|
||||
|
||||
message RemoteEntry {
|
||||
int64 last_modified_at = 1;
|
||||
int64 size = 2;
|
||||
string e_tag = 3;
|
||||
string storage_name = 4;
|
||||
}
|
||||
message Entry {
|
||||
string name = 1;
|
||||
bool is_directory = 2;
|
||||
|
@ -102,12 +108,7 @@ message Entry {
|
|||
int32 hard_link_counter = 8; // only exists in hard link meta data
|
||||
bytes content = 9; // if not empty, the file content
|
||||
|
||||
message Remote {
|
||||
int64 last_modified_at = 1;
|
||||
int64 size = 2;
|
||||
string e_tag = 3;
|
||||
}
|
||||
Remote remote = 10;
|
||||
RemoteEntry remote_entry = 10;
|
||||
}
|
||||
|
||||
message FullEntry {
|
||||
|
@ -343,7 +344,9 @@ message LocateBrokerResponse {
|
|||
repeated Resource resources = 2;
|
||||
}
|
||||
|
||||
/////////////////////////
|
||||
// Key-Value operations
|
||||
/////////////////////////
|
||||
message KvGetRequest {
|
||||
bytes key = 1;
|
||||
}
|
||||
|
@ -359,7 +362,9 @@ message KvPutResponse {
|
|||
string error = 1;
|
||||
}
|
||||
|
||||
/////////////////////////
|
||||
// path-based configurations
|
||||
/////////////////////////
|
||||
message FilerConf {
|
||||
int32 version = 1;
|
||||
message PathConf {
|
||||
|
@ -375,6 +380,9 @@ message FilerConf {
|
|||
repeated PathConf locations = 2;
|
||||
}
|
||||
|
||||
/////////////////////////
|
||||
// Remote Storage related
|
||||
/////////////////////////
|
||||
message RemoteConf {
|
||||
string type = 1;
|
||||
string name = 2;
|
||||
|
@ -383,3 +391,12 @@ message RemoteConf {
|
|||
string s3_region = 6;
|
||||
string s3_endpoint = 7;
|
||||
}
|
||||
|
||||
message RemoteStorageMapping {
|
||||
map<string,RemoteStorageLocation> mappings = 1;
|
||||
}
|
||||
message RemoteStorageLocation {
|
||||
string name = 1;
|
||||
string bucket = 2;
|
||||
string path = 3;
|
||||
}
|
||||
|
|
|
@ -1,14 +1,12 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"google.golang.org/grpc"
|
||||
"io"
|
||||
"strconv"
|
||||
"time"
|
||||
)
|
||||
|
@ -74,38 +72,9 @@ func startGenerateMetadata() {
|
|||
|
||||
func startSubscribeMetadata(eachEntryFunc func(event *filer_pb.SubscribeMetadataResponse) error) {
|
||||
|
||||
lastTsNs := int64(0)
|
||||
tailErr := pb.FollowMetadata(*tailFiler, grpc.WithInsecure(), "tail",
|
||||
*dir, 0, 0, eachEntryFunc, false)
|
||||
|
||||
tailErr := pb.WithFilerClient(*tailFiler, grpc.WithInsecure(), func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "tail",
|
||||
PathPrefix: *dir,
|
||||
SinceNs: lastTsNs,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("listen: %v", err)
|
||||
}
|
||||
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
if listenErr == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
if err = eachEntryFunc(resp); err != nil {
|
||||
glog.V(0).Infof("tail last record:%+v", time.Unix(0, lastTsNs))
|
||||
return err
|
||||
}
|
||||
lastTsNs = resp.TsNs
|
||||
}
|
||||
|
||||
})
|
||||
if tailErr != nil {
|
||||
fmt.Printf("tail %s: %v\n", *tailFiler, tailErr)
|
||||
}
|
||||
|
|
|
@ -37,3 +37,7 @@ debug_s3:
|
|||
debug_filer_copy:
|
||||
go build -gcflags="all=-N -l"
|
||||
dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec weed -- -v=4 filer.backup -filer=localhost:8888 -filerProxy -timeAgo=10h
|
||||
|
||||
debug_filer_remote_sync:
|
||||
go build -gcflags="all=-N -l"
|
||||
dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec weed -- -v=4 filer.remote.sync -filer="localhost:8888" -dir=/buckets/b2 -timeAgo=10000h
|
||||
|
|
|
@ -21,6 +21,7 @@ var Commands = []*Command{
|
|||
cmdFilerCopy,
|
||||
cmdFilerMetaBackup,
|
||||
cmdFilerMetaTail,
|
||||
cmdFilerRemoteSynchronize,
|
||||
cmdFilerReplicate,
|
||||
cmdFilerSynchronize,
|
||||
cmdFix,
|
||||
|
|
|
@ -3,7 +3,6 @@ package command
|
|||
import (
|
||||
"fmt"
|
||||
"net/http"
|
||||
_ "net/http/pprof"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
|
|
@ -1,16 +1,13 @@
|
|||
package command
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/replication/source"
|
||||
"github.com/chrislusf/seaweedfs/weed/security"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"google.golang.org/grpc"
|
||||
"io"
|
||||
"time"
|
||||
)
|
||||
|
||||
|
@ -110,48 +107,12 @@ func doFilerBackup(grpcDialOption grpc.DialOption, backupOption *FilerBackupOpti
|
|||
|
||||
processEventFn := genProcessFunction(sourcePath, targetPath, dataSink, debug)
|
||||
|
||||
return pb.WithFilerClient(sourceFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "backup_" + dataSink.GetName(),
|
||||
PathPrefix: sourcePath,
|
||||
SinceNs: startFrom.UnixNano(),
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("listen: %v", err)
|
||||
}
|
||||
|
||||
var counter int64
|
||||
var lastWriteTime time.Time
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
|
||||
if listenErr == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
|
||||
if err := processEventFn(resp); err != nil {
|
||||
return fmt.Errorf("processEventFn: %v", err)
|
||||
}
|
||||
|
||||
counter++
|
||||
if lastWriteTime.Add(3 * time.Second).Before(time.Now()) {
|
||||
glog.V(0).Infof("backup %s progressed to %v %0.2f/sec", sourceFiler, time.Unix(0, resp.TsNs), float64(counter)/float64(3))
|
||||
counter = 0
|
||||
lastWriteTime = time.Now()
|
||||
if err := setOffset(grpcDialOption, sourceFiler, BackupKeyPrefix, int32(sinkId), resp.TsNs); err != nil {
|
||||
return fmt.Errorf("setOffset: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
processEventFnWithOffset := pb.AddOffsetFunc(processEventFn, 3 * time.Second, func(counter int64, lastTsNs int64) error {
|
||||
glog.V(0).Infof("backup %s progressed to %v %0.2f/sec", sourceFiler, time.Unix(0, lastTsNs), float64(counter)/float64(3))
|
||||
return setOffset(grpcDialOption, sourceFiler, BackupKeyPrefix, int32(sinkId), lastTsNs)
|
||||
})
|
||||
|
||||
return pb.FollowMetadata(sourceFiler, grpcDialOption, "backup_" + dataSink.GetName(),
|
||||
sourcePath, startFrom.UnixNano(), 0, processEventFnWithOffset, false)
|
||||
|
||||
}
|
||||
|
|
|
@ -7,7 +7,6 @@ import (
|
|||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/spf13/viper"
|
||||
"google.golang.org/grpc"
|
||||
"io"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
|
@ -190,48 +189,15 @@ func (metaBackup *FilerMetaBackupOptions) streamMetadataBackup() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
tailErr := pb.WithFilerClient(*metaBackup.filerAddress, metaBackup.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "meta_backup",
|
||||
PathPrefix: *metaBackup.filerDirectory,
|
||||
SinceNs: startTime.UnixNano(),
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("listen: %v", err)
|
||||
}
|
||||
|
||||
var counter int64
|
||||
var lastWriteTime time.Time
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
if listenErr == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
if err = eachEntryFunc(resp); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
counter++
|
||||
if lastWriteTime.Add(3 * time.Second).Before(time.Now()) {
|
||||
glog.V(0).Infof("meta backup %s progressed to %v %0.2f/sec", *metaBackup.filerAddress, time.Unix(0, resp.TsNs), float64(counter)/float64(3))
|
||||
counter = 0
|
||||
lastWriteTime = time.Now()
|
||||
if err2 := metaBackup.setOffset(lastWriteTime); err2 != nil {
|
||||
return err2
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
processEventFnWithOffset := pb.AddOffsetFunc(eachEntryFunc, 3 * time.Second, func(counter int64, lastTsNs int64) error {
|
||||
lastTime := time.Unix(0, lastTsNs)
|
||||
glog.V(0).Infof("meta backup %s progressed to %v %0.2f/sec", *metaBackup.filerAddress, lastTime, float64(counter)/float64(3))
|
||||
return metaBackup.setOffset(lastTime)
|
||||
})
|
||||
return tailErr
|
||||
|
||||
return pb.FollowMetadata(*metaBackup.filerAddress, metaBackup.grpcDialOption, "meta_backup",
|
||||
*metaBackup.filerDirectory, startTime.UnixNano(), 0, processEventFnWithOffset, false)
|
||||
|
||||
}
|
||||
|
||||
func (metaBackup *FilerMetaBackupOptions) getOffset() (lastWriteTime time.Time, err error) {
|
||||
|
|
|
@ -3,16 +3,15 @@ package command
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
jsoniter "github.com/json-iterator/go"
|
||||
"github.com/olivere/elastic/v7"
|
||||
"io"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/security"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
|
@ -104,37 +103,18 @@ func runFilerMetaTail(cmd *Command, args []string) bool {
|
|||
}
|
||||
}
|
||||
|
||||
tailErr := pb.WithFilerClient(*tailFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "tail",
|
||||
PathPrefix: *tailTarget,
|
||||
SinceNs: time.Now().Add(-*tailStart).UnixNano(),
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("listen: %v", err)
|
||||
}
|
||||
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
if listenErr == io.EOF {
|
||||
tailErr := pb.FollowMetadata(*tailFiler, grpcDialOption, "tail",
|
||||
*tailTarget, time.Now().Add(-*tailStart).UnixNano(), 0,
|
||||
func(resp *filer_pb.SubscribeMetadataResponse) error {
|
||||
if !shouldPrint(resp) {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
if !shouldPrint(resp) {
|
||||
continue
|
||||
}
|
||||
if err = eachEntryFunc(resp); err != nil {
|
||||
if err := eachEntryFunc(resp); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}, false)
|
||||
|
||||
})
|
||||
if tailErr != nil {
|
||||
fmt.Printf("tail %s: %v\n", *tailFiler, tailErr)
|
||||
}
|
||||
|
|
260
weed/command/filer_remote_sync.go
Normal file
260
weed/command/filer_remote_sync.go
Normal file
|
@ -0,0 +1,260 @@
|
|||
package command
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/filer"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/remote_storage"
|
||||
"github.com/chrislusf/seaweedfs/weed/replication/source"
|
||||
"github.com/chrislusf/seaweedfs/weed/security"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"google.golang.org/grpc"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
type RemoteSyncOptions struct {
|
||||
filerAddress *string
|
||||
grpcDialOption grpc.DialOption
|
||||
readChunkFromFiler *bool
|
||||
debug *bool
|
||||
timeAgo *time.Duration
|
||||
dir *string
|
||||
}
|
||||
|
||||
const (
|
||||
RemoteSyncKeyPrefix = "remote.sync."
|
||||
)
|
||||
|
||||
var _ = filer_pb.FilerClient(&RemoteSyncOptions{})
|
||||
|
||||
func (option *RemoteSyncOptions) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
|
||||
return pb.WithFilerClient(*option.filerAddress, option.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
return fn(client)
|
||||
})
|
||||
}
|
||||
func (option *RemoteSyncOptions) AdjustedUrl(location *filer_pb.Location) string {
|
||||
return location.Url
|
||||
}
|
||||
|
||||
var (
|
||||
remoteSyncOptions RemoteSyncOptions
|
||||
)
|
||||
|
||||
func init() {
|
||||
cmdFilerRemoteSynchronize.Run = runFilerRemoteSynchronize // break init cycle
|
||||
remoteSyncOptions.filerAddress = cmdFilerRemoteSynchronize.Flag.String("filer", "localhost:8888", "filer of the SeaweedFS cluster")
|
||||
remoteSyncOptions.dir = cmdFilerRemoteSynchronize.Flag.String("dir", "/", "a mounted directory on filer")
|
||||
remoteSyncOptions.readChunkFromFiler = cmdFilerRemoteSynchronize.Flag.Bool("filerProxy", false, "read file chunks from filer instead of volume servers")
|
||||
remoteSyncOptions.debug = cmdFilerRemoteSynchronize.Flag.Bool("debug", false, "debug mode to print out filer updated remote files")
|
||||
remoteSyncOptions.timeAgo = cmdFilerRemoteSynchronize.Flag.Duration("timeAgo", 0, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"")
|
||||
}
|
||||
|
||||
var cmdFilerRemoteSynchronize = &Command{
|
||||
UsageLine: "filer.remote.sync -filer=<filerHost>:<filerPort> -dir=/mount/s3_on_cloud",
|
||||
Short: "resumeable continuously write back updates to remote storage if the directory is mounted to the remote storage",
|
||||
Long: `resumeable continuously write back updates to remote storage if the directory is mounted to the remote storage
|
||||
|
||||
filer.remote.sync listens on filer update events.
|
||||
If any mounted remote file is updated, it will fetch the updated content,
|
||||
and write to the remote storage.
|
||||
`,
|
||||
}
|
||||
|
||||
func runFilerRemoteSynchronize(cmd *Command, args []string) bool {
|
||||
|
||||
util.LoadConfiguration("security", false)
|
||||
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client")
|
||||
remoteSyncOptions.grpcDialOption = grpcDialOption
|
||||
|
||||
// read filer remote storage mount mappings
|
||||
mappings, readErr := filer.ReadMountMappings(grpcDialOption, *remoteSyncOptions.filerAddress)
|
||||
if readErr != nil {
|
||||
fmt.Printf("read mount mapping: %v", readErr)
|
||||
return false
|
||||
}
|
||||
|
||||
filerSource := &source.FilerSource{}
|
||||
filerSource.DoInitialize(
|
||||
*remoteSyncOptions.filerAddress,
|
||||
pb.ServerToGrpcAddress(*remoteSyncOptions.filerAddress),
|
||||
"/", // does not matter
|
||||
*remoteSyncOptions.readChunkFromFiler,
|
||||
)
|
||||
|
||||
var found bool
|
||||
for dir, remoteStorageMountLocation := range mappings.Mappings {
|
||||
if *remoteSyncOptions.dir == dir {
|
||||
found = true
|
||||
storageConf, readErr := filer.ReadRemoteStorageConf(grpcDialOption, *remoteSyncOptions.filerAddress, remoteStorageMountLocation.Name)
|
||||
if readErr != nil {
|
||||
fmt.Printf("read remote storage configuration for %s: %v", dir, readErr)
|
||||
continue
|
||||
}
|
||||
fmt.Printf("synchronize %s to remote storage...\n", *remoteSyncOptions.dir)
|
||||
if err := util.Retry("filer.remote.sync "+dir, func() error {
|
||||
return followUpdatesAndUploadToRemote(&remoteSyncOptions, filerSource, dir, storageConf, remoteStorageMountLocation)
|
||||
}); err != nil {
|
||||
fmt.Printf("synchronize %s: %v\n", *remoteSyncOptions.dir, err)
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
if !found {
|
||||
fmt.Printf("directory %s is not mounted to any remote storage\n", *remoteSyncOptions.dir)
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func followUpdatesAndUploadToRemote(option *RemoteSyncOptions, filerSource *source.FilerSource, mountedDir string, remoteStorage *filer_pb.RemoteConf, remoteStorageMountLocation *filer_pb.RemoteStorageLocation) error {
|
||||
|
||||
dirHash := util.HashStringToLong(mountedDir)
|
||||
|
||||
// 1. specified by timeAgo
|
||||
// 2. last offset timestamp for this directory
|
||||
// 3. directory creation time
|
||||
var lastOffsetTs time.Time
|
||||
if *option.timeAgo == 0 {
|
||||
mountedDirEntry, err := filer_pb.GetEntry(option, util.FullPath(mountedDir))
|
||||
if err != nil {
|
||||
return fmt.Errorf("lookup %s: %v", mountedDir, err)
|
||||
}
|
||||
|
||||
lastOffsetTsNs, err := getOffset(option.grpcDialOption, *option.filerAddress, RemoteSyncKeyPrefix, int32(dirHash))
|
||||
if err == nil && mountedDirEntry.Attributes.Crtime < lastOffsetTsNs/1000000 {
|
||||
lastOffsetTs = time.Unix(0, lastOffsetTsNs)
|
||||
glog.V(0).Infof("resume from %v", lastOffsetTs)
|
||||
} else {
|
||||
lastOffsetTs = time.Unix(mountedDirEntry.Attributes.Crtime, 0)
|
||||
}
|
||||
} else {
|
||||
lastOffsetTs = time.Now().Add(-*option.timeAgo)
|
||||
}
|
||||
|
||||
client, err := remote_storage.GetRemoteStorage(remoteStorage)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
eachEntryFunc := func(resp *filer_pb.SubscribeMetadataResponse) error {
|
||||
message := resp.EventNotification
|
||||
if message.OldEntry == nil && message.NewEntry == nil {
|
||||
return nil
|
||||
}
|
||||
if message.OldEntry == nil && message.NewEntry != nil {
|
||||
if len(message.NewEntry.Chunks) == 0 {
|
||||
return nil
|
||||
}
|
||||
fmt.Printf("create: %+v\n", resp)
|
||||
if !shouldSendToRemote(message.NewEntry) {
|
||||
fmt.Printf("skipping creating: %+v\n", resp)
|
||||
return nil
|
||||
}
|
||||
dest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(message.NewParentPath, message.NewEntry.Name), remoteStorageMountLocation)
|
||||
reader := filer.NewChunkStreamReader(filerSource, message.NewEntry.Chunks)
|
||||
remoteEntry, writeErr := client.WriteFile(dest, message.NewEntry, reader)
|
||||
if writeErr != nil {
|
||||
return writeErr
|
||||
}
|
||||
return updateLocalEntry(&remoteSyncOptions, message.NewParentPath, message.NewEntry, remoteEntry)
|
||||
}
|
||||
if message.OldEntry != nil && message.NewEntry == nil {
|
||||
fmt.Printf("delete: %+v\n", resp)
|
||||
dest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(resp.Directory, message.OldEntry.Name), remoteStorageMountLocation)
|
||||
return client.DeleteFile(dest)
|
||||
}
|
||||
if message.OldEntry != nil && message.NewEntry != nil {
|
||||
oldDest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(resp.Directory, message.OldEntry.Name), remoteStorageMountLocation)
|
||||
dest := toRemoteStorageLocation(util.FullPath(mountedDir), util.NewFullPath(message.NewParentPath, message.NewEntry.Name), remoteStorageMountLocation)
|
||||
if !shouldSendToRemote(message.NewEntry) {
|
||||
fmt.Printf("skipping updating: %+v\n", resp)
|
||||
return nil
|
||||
}
|
||||
if resp.Directory == message.NewParentPath && message.OldEntry.Name == message.NewEntry.Name {
|
||||
if isSameChunks(message.OldEntry.Chunks, message.NewEntry.Chunks) {
|
||||
fmt.Printf("update meta: %+v\n", resp)
|
||||
return client.UpdateFileMetadata(dest, message.NewEntry)
|
||||
}
|
||||
}
|
||||
fmt.Printf("update: %+v\n", resp)
|
||||
if err := client.DeleteFile(oldDest); err != nil {
|
||||
return err
|
||||
}
|
||||
reader := filer.NewChunkStreamReader(filerSource, message.NewEntry.Chunks)
|
||||
remoteEntry, writeErr := client.WriteFile(dest, message.NewEntry, reader)
|
||||
if writeErr != nil {
|
||||
return writeErr
|
||||
}
|
||||
return updateLocalEntry(&remoteSyncOptions, message.NewParentPath, message.NewEntry, remoteEntry)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
processEventFnWithOffset := pb.AddOffsetFunc(eachEntryFunc, 3*time.Second, func(counter int64, lastTsNs int64) error {
|
||||
lastTime := time.Unix(0, lastTsNs)
|
||||
glog.V(0).Infof("remote sync %s progressed to %v %0.2f/sec", *option.filerAddress, lastTime, float64(counter)/float64(3))
|
||||
return setOffset(option.grpcDialOption, *option.filerAddress, RemoteSyncKeyPrefix, int32(dirHash), lastTsNs)
|
||||
})
|
||||
|
||||
return pb.FollowMetadata(*option.filerAddress, option.grpcDialOption,
|
||||
"filer.remote.sync", mountedDir, lastOffsetTs.UnixNano(), 0, processEventFnWithOffset, false)
|
||||
}
|
||||
|
||||
func toRemoteStorageLocation(mountDir, sourcePath util.FullPath, remoteMountLocation *filer_pb.RemoteStorageLocation) *filer_pb.RemoteStorageLocation {
|
||||
var dest string
|
||||
source := string(sourcePath[len(mountDir):])
|
||||
if strings.HasSuffix(remoteMountLocation.Path, "/") {
|
||||
dest = remoteMountLocation.Path + source[1:]
|
||||
} else {
|
||||
dest = remoteMountLocation.Path + source
|
||||
}
|
||||
return &filer_pb.RemoteStorageLocation{
|
||||
Name: remoteMountLocation.Name,
|
||||
Bucket: remoteMountLocation.Bucket,
|
||||
Path: dest,
|
||||
}
|
||||
}
|
||||
|
||||
func isSameChunks(a, b []*filer_pb.FileChunk) bool {
|
||||
if len(a) != len(b) {
|
||||
return false
|
||||
}
|
||||
for i := 0; i < len(a); i++ {
|
||||
x, y := a[i], b[i]
|
||||
if !proto.Equal(x, y) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func shouldSendToRemote(entry *filer_pb.Entry) bool {
|
||||
if entry.RemoteEntry == nil {
|
||||
return true
|
||||
}
|
||||
if entry.RemoteEntry.Size != int64(filer.FileSize(entry)) {
|
||||
return true
|
||||
}
|
||||
if entry.RemoteEntry.LastModifiedAt < entry.Attributes.Mtime {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func updateLocalEntry(filerClient filer_pb.FilerClient, dir string, entry *filer_pb.Entry, remoteEntry *filer_pb.RemoteEntry) error {
|
||||
entry.RemoteEntry = remoteEntry
|
||||
return filerClient.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
_, err := client.UpdateEntry(context.Background(), &filer_pb.UpdateEntryRequest{
|
||||
Directory: dir,
|
||||
Entry: entry,
|
||||
})
|
||||
return err
|
||||
})
|
||||
}
|
|
@ -7,12 +7,6 @@ import (
|
|||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/replication"
|
||||
"github.com/chrislusf/seaweedfs/weed/replication/sink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/azuresink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/b2sink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/filersink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/gcssink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/localsink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/s3sink"
|
||||
"github.com/chrislusf/seaweedfs/weed/replication/sub"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
)
|
||||
|
|
|
@ -15,7 +15,6 @@ import (
|
|||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"github.com/chrislusf/seaweedfs/weed/util/grace"
|
||||
"google.golang.org/grpc"
|
||||
"io"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
@ -166,50 +165,14 @@ func doSubscribeFilerMetaChanges(grpcDialOption grpc.DialOption, sourceFiler, so
|
|||
return persistEventFn(resp)
|
||||
}
|
||||
|
||||
return pb.WithFilerClient(sourceFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "syncTo_" + targetFiler,
|
||||
PathPrefix: sourcePath,
|
||||
SinceNs: sourceFilerOffsetTsNs,
|
||||
Signature: targetFilerSignature,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("listen: %v", err)
|
||||
}
|
||||
|
||||
var counter int64
|
||||
var lastWriteTime time.Time
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
if listenErr == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
|
||||
if err := processEventFn(resp); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
counter++
|
||||
if lastWriteTime.Add(3 * time.Second).Before(time.Now()) {
|
||||
glog.V(0).Infof("sync %s to %s progressed to %v %0.2f/sec", sourceFiler, targetFiler, time.Unix(0, resp.TsNs), float64(counter)/float64(3))
|
||||
counter = 0
|
||||
lastWriteTime = time.Now()
|
||||
if err := setOffset(grpcDialOption, targetFiler, SyncKeyPrefix, sourceFilerSignature, resp.TsNs); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
processEventFnWithOffset := pb.AddOffsetFunc(processEventFn, 3 * time.Second, func(counter int64, lastTsNs int64) error {
|
||||
glog.V(0).Infof("sync %s to %s progressed to %v %0.2f/sec", sourceFiler, targetFiler, time.Unix(0, lastTsNs), float64(counter)/float64(3))
|
||||
return setOffset(grpcDialOption, targetFiler, SyncKeyPrefix, sourceFilerSignature, lastTsNs)
|
||||
})
|
||||
|
||||
return pb.FollowMetadata(sourceFiler, grpcDialOption, "syncTo_" + targetFiler,
|
||||
sourcePath, sourceFilerOffsetTsNs, targetFilerSignature, processEventFnWithOffset, false)
|
||||
|
||||
}
|
||||
|
||||
const (
|
||||
|
|
31
weed/command/imports.go
Normal file
31
weed/command/imports.go
Normal file
|
@ -0,0 +1,31 @@
|
|||
package command
|
||||
|
||||
import (
|
||||
_ "net/http/pprof"
|
||||
|
||||
_ "github.com/chrislusf/seaweedfs/weed/remote_storage/s3"
|
||||
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/azuresink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/b2sink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/filersink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/gcssink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/localsink"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/replication/sink/s3sink"
|
||||
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/cassandra"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/elastic/v7"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/etcd"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/hbase"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/leveldb"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/leveldb2"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/leveldb3"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/mongodb"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/mysql"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/mysql2"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/postgres"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/postgres2"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/redis"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/redis2"
|
||||
_ "github.com/chrislusf/seaweedfs/weed/filer/sqlite"
|
||||
|
||||
)
|
|
@ -55,6 +55,7 @@ func runShell(command *Command, args []string) bool {
|
|||
|
||||
var err error
|
||||
shellOptions.FilerHost, shellOptions.FilerPort, err = util.ParseHostPort(*shellInitialFiler)
|
||||
shellOptions.FilerAddress = *shellInitialFiler
|
||||
if err != nil {
|
||||
fmt.Printf("failed to parse filer %s: %v\n", *shellInitialFiler, err)
|
||||
return false
|
||||
|
|
|
@ -42,7 +42,7 @@ type Entry struct {
|
|||
HardLinkId HardLinkId
|
||||
HardLinkCounter int32
|
||||
Content []byte
|
||||
Remote *filer_pb.Entry_Remote
|
||||
Remote *filer_pb.RemoteEntry
|
||||
}
|
||||
|
||||
func (entry *Entry) Size() uint64 {
|
||||
|
@ -78,7 +78,7 @@ func (entry *Entry) ToExistingProtoEntry(message *filer_pb.Entry) {
|
|||
message.HardLinkId = entry.HardLinkId
|
||||
message.HardLinkCounter = entry.HardLinkCounter
|
||||
message.Content = entry.Content
|
||||
message.Remote = entry.Remote
|
||||
message.RemoteEntry = entry.Remote
|
||||
}
|
||||
|
||||
func FromPbEntryToExistingEntry(message *filer_pb.Entry, fsEntry *Entry) {
|
||||
|
@ -88,7 +88,7 @@ func FromPbEntryToExistingEntry(message *filer_pb.Entry, fsEntry *Entry) {
|
|||
fsEntry.HardLinkId = HardLinkId(message.HardLinkId)
|
||||
fsEntry.HardLinkCounter = message.HardLinkCounter
|
||||
fsEntry.Content = message.Content
|
||||
fsEntry.Remote = message.Remote
|
||||
fsEntry.Remote = message.RemoteEntry
|
||||
}
|
||||
|
||||
func (entry *Entry) ToProtoFullEntry() *filer_pb.FullEntry {
|
||||
|
|
|
@ -42,6 +42,7 @@ type Filer struct {
|
|||
MetaAggregator *MetaAggregator
|
||||
Signature int32
|
||||
FilerConf *FilerConf
|
||||
RemoteStorage *FilerRemoteStorage
|
||||
}
|
||||
|
||||
func NewFiler(masters []string, grpcDialOption grpc.DialOption,
|
||||
|
@ -51,6 +52,7 @@ func NewFiler(masters []string, grpcDialOption grpc.DialOption,
|
|||
fileIdDeletionQueue: util.NewUnboundedQueue(),
|
||||
GrpcDialOption: grpcDialOption,
|
||||
FilerConf: NewFilerConf(),
|
||||
RemoteStorage: NewFilerRemoteStorage(),
|
||||
}
|
||||
f.LocalMetaLogBuffer = log_buffer.NewLogBuffer("local", LogFlushInterval, f.logFlushFunc, notifyFn)
|
||||
f.metaLogCollection = collection
|
||||
|
|
|
@ -12,6 +12,7 @@ import (
|
|||
// onMetadataChangeEvent is triggered after filer processed change events from local or remote filers
|
||||
func (f *Filer) onMetadataChangeEvent(event *filer_pb.SubscribeMetadataResponse) {
|
||||
f.maybeReloadFilerConfiguration(event)
|
||||
f.maybeReloadRemoteStorageConfigurationAndMapping(event)
|
||||
f.onBucketEvents(event)
|
||||
}
|
||||
|
||||
|
@ -84,3 +85,16 @@ func (f *Filer) LoadFilerConf() {
|
|||
}
|
||||
f.FilerConf = fc
|
||||
}
|
||||
|
||||
////////////////////////////////////
|
||||
// load and maintain remote storages
|
||||
////////////////////////////////////
|
||||
func (f *Filer) LoadRemoteStorageConfAndMapping() {
|
||||
if err := f.RemoteStorage.LoadRemoteStorageConfigurationsAndMapping(f); err != nil {
|
||||
glog.Errorf("read remote conf and mapping: %v", err)
|
||||
return
|
||||
}
|
||||
}
|
||||
func (f *Filer) maybeReloadRemoteStorageConfigurationAndMapping(event *filer_pb.SubscribeMetadataResponse) {
|
||||
// FIXME add reloading
|
||||
}
|
||||
|
|
182
weed/filer/filer_remote_storage.go
Normal file
182
weed/filer/filer_remote_storage.go
Normal file
|
@ -0,0 +1,182 @@
|
|||
package filer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/remote_storage"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"google.golang.org/grpc"
|
||||
"math"
|
||||
"strings"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/viant/ptrie"
|
||||
)
|
||||
|
||||
const REMOTE_STORAGE_CONF_SUFFIX = ".conf"
|
||||
const REMOTE_STORAGE_MOUNT_FILE = "mount.mapping"
|
||||
|
||||
type FilerRemoteStorage struct {
|
||||
rules ptrie.Trie
|
||||
storageNameToConf map[string]*filer_pb.RemoteConf
|
||||
}
|
||||
|
||||
func NewFilerRemoteStorage() (rs *FilerRemoteStorage) {
|
||||
rs = &FilerRemoteStorage{
|
||||
rules: ptrie.New(),
|
||||
storageNameToConf: make(map[string]*filer_pb.RemoteConf),
|
||||
}
|
||||
return rs
|
||||
}
|
||||
|
||||
func (rs *FilerRemoteStorage) LoadRemoteStorageConfigurationsAndMapping(filer *Filer) (err error) {
|
||||
// execute this on filer
|
||||
|
||||
entries, _, err := filer.ListDirectoryEntries(context.Background(), DirectoryEtcRemote, "", false, math.MaxInt64, "", "", "")
|
||||
if err != nil {
|
||||
if err == filer_pb.ErrNotFound {
|
||||
return nil
|
||||
}
|
||||
glog.Errorf("read remote storage %s: %v", DirectoryEtcRemote, err)
|
||||
return
|
||||
}
|
||||
|
||||
for _, entry := range entries {
|
||||
if entry.Name() == REMOTE_STORAGE_MOUNT_FILE {
|
||||
if err := rs.loadRemoteStorageMountMapping(entry.Content); err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
if !strings.HasSuffix(entry.Name(), REMOTE_STORAGE_CONF_SUFFIX) {
|
||||
return nil
|
||||
}
|
||||
conf := &filer_pb.RemoteConf{}
|
||||
if err := proto.Unmarshal(entry.Content, conf); err != nil {
|
||||
return fmt.Errorf("unmarshal %s/%s: %v", DirectoryEtcRemote, entry.Name(), err)
|
||||
}
|
||||
rs.storageNameToConf[conf.Name] = conf
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rs *FilerRemoteStorage) loadRemoteStorageMountMapping(data []byte) (err error) {
|
||||
mappings := &filer_pb.RemoteStorageMapping{}
|
||||
if err := proto.Unmarshal(data, mappings); err != nil {
|
||||
return fmt.Errorf("unmarshal %s/%s: %v", DirectoryEtcRemote, REMOTE_STORAGE_MOUNT_FILE, err)
|
||||
}
|
||||
for dir, storageLocation := range mappings.Mappings {
|
||||
rs.mapDirectoryToRemoteStorage(util.FullPath(dir), storageLocation)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rs *FilerRemoteStorage) mapDirectoryToRemoteStorage(dir util.FullPath, loc *filer_pb.RemoteStorageLocation) {
|
||||
rs.rules.Put([]byte(dir+"/"), loc)
|
||||
}
|
||||
|
||||
func (rs *FilerRemoteStorage) FindMountDirectory(p util.FullPath) (mountDir util.FullPath, remoteLocation *filer_pb.RemoteStorageLocation) {
|
||||
rs.rules.MatchPrefix([]byte(p), func(key []byte, value interface{}) bool {
|
||||
mountDir = util.FullPath(string(key[:len(key)-1]))
|
||||
remoteLocation = value.(*filer_pb.RemoteStorageLocation)
|
||||
return true
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
func (rs *FilerRemoteStorage) FindRemoteStorageClient(p util.FullPath) (client remote_storage.RemoteStorageClient, remoteConf *filer_pb.RemoteConf, found bool) {
|
||||
var storageLocation *filer_pb.RemoteStorageLocation
|
||||
rs.rules.MatchPrefix([]byte(p), func(key []byte, value interface{}) bool {
|
||||
storageLocation = value.(*filer_pb.RemoteStorageLocation)
|
||||
return true
|
||||
})
|
||||
|
||||
if storageLocation == nil {
|
||||
found = false
|
||||
return
|
||||
}
|
||||
|
||||
return rs.GetRemoteStorageClient(storageLocation.Name)
|
||||
}
|
||||
|
||||
func (rs *FilerRemoteStorage) GetRemoteStorageClient(storageName string) (client remote_storage.RemoteStorageClient, remoteConf *filer_pb.RemoteConf, found bool) {
|
||||
remoteConf, found = rs.storageNameToConf[storageName]
|
||||
if !found {
|
||||
return
|
||||
}
|
||||
|
||||
var err error
|
||||
if client, err = remote_storage.GetRemoteStorage(remoteConf); err == nil {
|
||||
found = true
|
||||
return
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func UnmarshalRemoteStorageMappings(oldContent []byte) (mappings *filer_pb.RemoteStorageMapping, err error) {
|
||||
mappings = &filer_pb.RemoteStorageMapping{
|
||||
Mappings: make(map[string]*filer_pb.RemoteStorageLocation),
|
||||
}
|
||||
if len(oldContent) > 0 {
|
||||
if err = proto.Unmarshal(oldContent, mappings); err != nil {
|
||||
glog.Warningf("unmarshal existing mappings: %v", err)
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func AddRemoteStorageMapping(oldContent []byte, dir string, storageLocation *filer_pb.RemoteStorageLocation) (newContent []byte, err error) {
|
||||
mappings, unmarshalErr := UnmarshalRemoteStorageMappings(oldContent)
|
||||
if unmarshalErr != nil {
|
||||
// skip
|
||||
}
|
||||
|
||||
// set the new mapping
|
||||
mappings.Mappings[dir] = storageLocation
|
||||
|
||||
if newContent, err = proto.Marshal(mappings); err != nil {
|
||||
return oldContent, fmt.Errorf("marshal mappings: %v", err)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
|
||||
func ReadMountMappings(grpcDialOption grpc.DialOption, filerAddress string) (mappings *filer_pb.RemoteStorageMapping, readErr error) {
|
||||
var oldContent []byte
|
||||
if readErr = pb.WithFilerClient(filerAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
oldContent, readErr = ReadInsideFiler(client, DirectoryEtcRemote, REMOTE_STORAGE_MOUNT_FILE)
|
||||
return readErr
|
||||
}); readErr != nil {
|
||||
return nil, readErr
|
||||
}
|
||||
|
||||
mappings, readErr = UnmarshalRemoteStorageMappings(oldContent)
|
||||
if readErr != nil {
|
||||
return nil, fmt.Errorf("unmarshal mappings: %v", readErr)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func ReadRemoteStorageConf(grpcDialOption grpc.DialOption, filerAddress string, storageName string) (conf *filer_pb.RemoteConf, readErr error) {
|
||||
var oldContent []byte
|
||||
if readErr = pb.WithFilerClient(filerAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
oldContent, readErr = ReadInsideFiler(client, DirectoryEtcRemote, storageName+REMOTE_STORAGE_CONF_SUFFIX)
|
||||
return readErr
|
||||
}); readErr != nil {
|
||||
return nil, readErr
|
||||
}
|
||||
|
||||
// unmarshal storage configuration
|
||||
conf = &filer_pb.RemoteConf{}
|
||||
if unMarshalErr := proto.Unmarshal(oldContent, conf); unMarshalErr != nil {
|
||||
readErr = fmt.Errorf("unmarshal %s/%s: %v", DirectoryEtcRemote, storageName+REMOTE_STORAGE_CONF_SUFFIX, unMarshalErr)
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
34
weed/filer/filer_remote_storage_test.go
Normal file
34
weed/filer/filer_remote_storage_test.go
Normal file
|
@ -0,0 +1,34 @@
|
|||
package filer
|
||||
|
||||
import (
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilerRemoteStorage_FindRemoteStorageClient(t *testing.T) {
|
||||
conf := &filer_pb.RemoteConf{
|
||||
Name: "s7",
|
||||
Type: "s3",
|
||||
}
|
||||
rs := NewFilerRemoteStorage()
|
||||
rs.storageNameToConf[conf.Name] = conf
|
||||
|
||||
rs.mapDirectoryToRemoteStorage("/a/b/c", &filer_pb.RemoteStorageLocation{
|
||||
Name: "s7",
|
||||
Bucket: "some",
|
||||
Path: "/dir",
|
||||
})
|
||||
|
||||
_, _, found := rs.FindRemoteStorageClient("/a/b/c/d/e/f")
|
||||
assert.Equal(t, true, found, "find storage client")
|
||||
|
||||
_, _, found2 := rs.FindRemoteStorageClient("/a/b")
|
||||
assert.Equal(t, false, found2, "should not find storage client")
|
||||
|
||||
_, _, found3 := rs.FindRemoteStorageClient("/a/b/c")
|
||||
assert.Equal(t, false, found3, "should not find storage client")
|
||||
|
||||
_, _, found4 := rs.FindRemoteStorageClient("/a/b/cc")
|
||||
assert.Equal(t, false, found4, "should not find storage client")
|
||||
}
|
|
@ -3,6 +3,7 @@ package filer
|
|||
import (
|
||||
"context"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"math"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
)
|
||||
|
@ -27,6 +28,10 @@ func (f *Filer) ListDirectoryEntries(ctx context.Context, p util.FullPath, start
|
|||
return true
|
||||
})
|
||||
|
||||
if limit == math.MaxInt64 {
|
||||
limit = math.MaxInt64 - 1
|
||||
}
|
||||
|
||||
hasMore = int64(len(entries)) >= limit+1
|
||||
if hasMore {
|
||||
entries = entries[:limit]
|
||||
|
|
29
weed/filer/read_remote.go
Normal file
29
weed/filer/read_remote.go
Normal file
|
@ -0,0 +1,29 @@
|
|||
package filer
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
)
|
||||
|
||||
func (entry *Entry) IsInRemoteOnly() bool {
|
||||
return len(entry.Chunks) == 0 && entry.Remote != nil && entry.Remote.Size > 0
|
||||
}
|
||||
|
||||
func (f *Filer) ReadRemote(entry *Entry, offset int64, size int64) (data[]byte, err error) {
|
||||
client, _, found := f.RemoteStorage.GetRemoteStorageClient(entry.Remote.StorageName)
|
||||
if !found {
|
||||
return nil, fmt.Errorf("remote storage %v not found", entry.Remote.StorageName)
|
||||
}
|
||||
|
||||
mountDir, remoteLoation := f.RemoteStorage.FindMountDirectory(entry.FullPath)
|
||||
|
||||
remoteFullPath := remoteLoation.Path + string(entry.FullPath[len(mountDir):])
|
||||
|
||||
sourceLoc := &filer_pb.RemoteStorageLocation{
|
||||
Name: remoteLoation.Name,
|
||||
Bucket: remoteLoation.Bucket,
|
||||
Path: remoteFullPath,
|
||||
}
|
||||
|
||||
return client.ReadFile(sourceLoc, offset, size)
|
||||
}
|
|
@ -91,6 +91,7 @@ func ReadAll(masterClient *wdclient.MasterClient, chunks []*filer_pb.FileChunk)
|
|||
type ChunkStreamReader struct {
|
||||
chunkViews []*ChunkView
|
||||
totalSize int64
|
||||
logicOffset int64
|
||||
buffer []byte
|
||||
bufferOffset int64
|
||||
bufferPos int
|
||||
|
@ -137,8 +138,7 @@ func NewChunkStreamReader(filerClient filer_pb.FilerClient, chunks []*filer_pb.F
|
|||
}
|
||||
|
||||
func (c *ChunkStreamReader) ReadAt(p []byte, off int64) (n int, err error) {
|
||||
_, err = c.Seek(off, io.SeekStart)
|
||||
if err != nil {
|
||||
if err = c.prepareBufferFor(c.logicOffset); err != nil {
|
||||
return
|
||||
}
|
||||
return c.Read(p)
|
||||
|
@ -151,12 +151,15 @@ func (c *ChunkStreamReader) Read(p []byte) (n int, err error) {
|
|||
return n, io.EOF
|
||||
}
|
||||
chunkView := c.chunkViews[c.nextChunkViewIndex]
|
||||
c.fetchChunkToBuffer(chunkView)
|
||||
if err = c.fetchChunkToBuffer(chunkView); err != nil {
|
||||
return
|
||||
}
|
||||
c.nextChunkViewIndex++
|
||||
}
|
||||
t := copy(p[n:], c.buffer[c.bufferPos:])
|
||||
c.bufferPos += t
|
||||
n += t
|
||||
c.logicOffset += int64(t)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
@ -171,19 +174,26 @@ func (c *ChunkStreamReader) Seek(offset int64, whence int) (int64, error) {
|
|||
switch whence {
|
||||
case io.SeekStart:
|
||||
case io.SeekCurrent:
|
||||
offset += c.bufferOffset + int64(c.bufferPos)
|
||||
offset += c.logicOffset
|
||||
case io.SeekEnd:
|
||||
offset = c.totalSize + offset
|
||||
}
|
||||
if offset > c.totalSize {
|
||||
err = io.ErrUnexpectedEOF
|
||||
} else {
|
||||
c.logicOffset = offset
|
||||
}
|
||||
|
||||
return offset, err
|
||||
|
||||
}
|
||||
|
||||
func (c *ChunkStreamReader) prepareBufferFor(offset int64) (err error) {
|
||||
// stay in the same chunk
|
||||
if !c.isBufferEmpty() {
|
||||
if c.bufferOffset <= offset && offset < c.bufferOffset+int64(len(c.buffer)) {
|
||||
c.bufferPos = int(offset - c.bufferOffset)
|
||||
return offset, nil
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -192,23 +202,21 @@ func (c *ChunkStreamReader) Seek(offset int64, whence int) (int64, error) {
|
|||
return c.chunkViews[i].LogicOffset <= offset
|
||||
})
|
||||
if currentChunkIndex == len(c.chunkViews) {
|
||||
return 0, io.EOF
|
||||
return io.EOF
|
||||
}
|
||||
|
||||
// positioning within the new chunk
|
||||
chunk := c.chunkViews[currentChunkIndex]
|
||||
if chunk.LogicOffset <= offset && offset < chunk.LogicOffset+int64(chunk.Size) {
|
||||
if c.isBufferEmpty() || c.bufferOffset != chunk.LogicOffset {
|
||||
c.fetchChunkToBuffer(chunk)
|
||||
if err = c.fetchChunkToBuffer(chunk); err != nil {
|
||||
return
|
||||
}
|
||||
c.nextChunkViewIndex = currentChunkIndex + 1
|
||||
}
|
||||
c.bufferPos = int(offset - c.bufferOffset)
|
||||
} else {
|
||||
return 0, io.ErrUnexpectedEOF
|
||||
}
|
||||
|
||||
return offset, err
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (c *ChunkStreamReader) fetchChunkToBuffer(chunkView *ChunkView) error {
|
||||
|
|
|
@ -2,12 +2,9 @@ package meta_cache
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/filer"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
)
|
||||
|
@ -62,38 +59,8 @@ func SubscribeMetaEvents(mc *MetaCache, selfSignature int32, client filer_pb.Fil
|
|||
|
||||
}
|
||||
|
||||
for {
|
||||
err := client.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "mount",
|
||||
PathPrefix: dir,
|
||||
SinceNs: lastTsNs,
|
||||
Signature: selfSignature,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("subscribe: %v", err)
|
||||
}
|
||||
return util.Retry("followMetaUpdates", func() error {
|
||||
return pb.WithFilerClientFollowMetadata(client, "mount", dir, lastTsNs, selfSignature, processEventFn, true)
|
||||
})
|
||||
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
if listenErr == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
|
||||
if err := processEventFn(resp); err != nil {
|
||||
glog.Fatalf("process %v: %v", resp, err)
|
||||
}
|
||||
lastTsNs = resp.TsNs
|
||||
}
|
||||
})
|
||||
if err != nil {
|
||||
glog.Errorf("subscribing filer meta change: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -92,6 +92,12 @@ message ListEntriesResponse {
|
|||
Entry entry = 1;
|
||||
}
|
||||
|
||||
message RemoteEntry {
|
||||
int64 last_modified_at = 1;
|
||||
int64 size = 2;
|
||||
string e_tag = 3;
|
||||
string storage_name = 4;
|
||||
}
|
||||
message Entry {
|
||||
string name = 1;
|
||||
bool is_directory = 2;
|
||||
|
@ -102,12 +108,7 @@ message Entry {
|
|||
int32 hard_link_counter = 8; // only exists in hard link meta data
|
||||
bytes content = 9; // if not empty, the file content
|
||||
|
||||
message Remote {
|
||||
int64 last_modified_at = 1;
|
||||
int64 size = 2;
|
||||
string e_tag = 3;
|
||||
}
|
||||
Remote remote = 10;
|
||||
RemoteEntry remote_entry = 10;
|
||||
}
|
||||
|
||||
message FullEntry {
|
||||
|
@ -343,7 +344,9 @@ message LocateBrokerResponse {
|
|||
repeated Resource resources = 2;
|
||||
}
|
||||
|
||||
/////////////////////////
|
||||
// Key-Value operations
|
||||
/////////////////////////
|
||||
message KvGetRequest {
|
||||
bytes key = 1;
|
||||
}
|
||||
|
@ -359,7 +362,9 @@ message KvPutResponse {
|
|||
string error = 1;
|
||||
}
|
||||
|
||||
/////////////////////////
|
||||
// path-based configurations
|
||||
/////////////////////////
|
||||
message FilerConf {
|
||||
int32 version = 1;
|
||||
message PathConf {
|
||||
|
@ -375,6 +380,9 @@ message FilerConf {
|
|||
repeated PathConf locations = 2;
|
||||
}
|
||||
|
||||
/////////////////////////
|
||||
// Remote Storage related
|
||||
/////////////////////////
|
||||
message RemoteConf {
|
||||
string type = 1;
|
||||
string name = 2;
|
||||
|
@ -383,3 +391,12 @@ message RemoteConf {
|
|||
string s3_region = 6;
|
||||
string s3_endpoint = 7;
|
||||
}
|
||||
|
||||
message RemoteStorageMapping {
|
||||
map<string,RemoteStorageLocation> mappings = 1;
|
||||
}
|
||||
message RemoteStorageLocation {
|
||||
string name = 1;
|
||||
string bucket = 2;
|
||||
string path = 3;
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load diff
|
@ -147,3 +147,7 @@ func (fp *FilerConf_PathConf) Key() interface{} {
|
|||
key, _ := proto.Marshal(fp)
|
||||
return string(key)
|
||||
}
|
||||
func (fp *RemoteStorageLocation) Key() interface{} {
|
||||
key, _ := proto.Marshal(fp)
|
||||
return string(key)
|
||||
}
|
||||
|
|
94
weed/pb/filer_pb_tail.go
Normal file
94
weed/pb/filer_pb_tail.go
Normal file
|
@ -0,0 +1,94 @@
|
|||
package pb
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"google.golang.org/grpc"
|
||||
"io"
|
||||
"time"
|
||||
)
|
||||
|
||||
type ProcessMetadataFunc func(resp *filer_pb.SubscribeMetadataResponse) error
|
||||
|
||||
func FollowMetadata(filerAddress string, grpcDialOption grpc.DialOption,
|
||||
clientName string, pathPrefix string, lastTsNs int64, selfSignature int32,
|
||||
processEventFn ProcessMetadataFunc, fatalOnError bool) error {
|
||||
|
||||
err := WithFilerClient(filerAddress, grpcDialOption, makeFunc(
|
||||
clientName, pathPrefix, lastTsNs, selfSignature, processEventFn, fatalOnError))
|
||||
if err != nil {
|
||||
return fmt.Errorf("subscribing filer meta change: %v", err)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func WithFilerClientFollowMetadata(filerClient filer_pb.FilerClient,
|
||||
clientName string, pathPrefix string, lastTsNs int64, selfSignature int32,
|
||||
processEventFn ProcessMetadataFunc, fatalOnError bool) error {
|
||||
|
||||
err := filerClient.WithFilerClient(makeFunc(
|
||||
clientName, pathPrefix, lastTsNs, selfSignature, processEventFn, fatalOnError))
|
||||
if err != nil {
|
||||
return fmt.Errorf("subscribing filer meta change: %v", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func makeFunc(clientName string, pathPrefix string, lastTsNs int64, selfSignature int32,
|
||||
processEventFn ProcessMetadataFunc, fatalOnError bool) func(client filer_pb.SeaweedFilerClient) error {
|
||||
return func(client filer_pb.SeaweedFilerClient) error {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: clientName,
|
||||
PathPrefix: pathPrefix,
|
||||
SinceNs: lastTsNs,
|
||||
Signature: selfSignature,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("subscribe: %v", err)
|
||||
}
|
||||
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
if listenErr == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
|
||||
if err := processEventFn(resp); err != nil {
|
||||
if fatalOnError {
|
||||
glog.Fatalf("process %v: %v", resp, err)
|
||||
} else {
|
||||
glog.Errorf("process %v: %v", resp, err)
|
||||
}
|
||||
}
|
||||
lastTsNs = resp.TsNs
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func AddOffsetFunc(processEventFn ProcessMetadataFunc, offsetInterval time.Duration, offsetFunc func(counter int64, offset int64) error) ProcessMetadataFunc {
|
||||
var counter int64
|
||||
var lastWriteTime time.Time
|
||||
return func(resp *filer_pb.SubscribeMetadataResponse) error {
|
||||
if err := processEventFn(resp); err != nil {
|
||||
return err
|
||||
}
|
||||
counter++
|
||||
if lastWriteTime.Add(offsetInterval).Before(time.Now()) {
|
||||
counter = 0
|
||||
lastWriteTime = time.Now()
|
||||
if err := offsetFunc(counter, resp.TsNs); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
}
|
|
@ -93,6 +93,10 @@ service VolumeServer {
|
|||
rpc VolumeServerLeave (VolumeServerLeaveRequest) returns (VolumeServerLeaveResponse) {
|
||||
}
|
||||
|
||||
// remote storage
|
||||
rpc FetchAndWriteNeedle (FetchAndWriteNeedleRequest) returns (FetchAndWriteNeedleResponse) {
|
||||
}
|
||||
|
||||
// <experimental> query
|
||||
rpc Query (QueryRequest) returns (stream QueriedStripe) {
|
||||
}
|
||||
|
@ -426,6 +430,7 @@ message VolumeInfo {
|
|||
string replication = 3;
|
||||
}
|
||||
|
||||
// tiered storage
|
||||
message VolumeTierMoveDatToRemoteRequest {
|
||||
uint32 volume_id = 1;
|
||||
string collection = 2;
|
||||
|
@ -460,6 +465,25 @@ message VolumeServerLeaveRequest {
|
|||
message VolumeServerLeaveResponse {
|
||||
}
|
||||
|
||||
// remote storage
|
||||
message FetchAndWriteNeedleRequest {
|
||||
uint32 volume_id = 1;
|
||||
uint64 needle_id = 2;
|
||||
int64 offset = 3;
|
||||
int64 size = 4;
|
||||
// remote conf
|
||||
string remote_type = 5;
|
||||
string remote_name = 6;
|
||||
string s3_access_key = 8;
|
||||
string s3_secret_key = 9;
|
||||
string s3_region = 10;
|
||||
string s3_endpoint = 11;
|
||||
string remote_bucket = 12;
|
||||
string remote_key = 13;
|
||||
}
|
||||
message FetchAndWriteNeedleResponse {
|
||||
}
|
||||
|
||||
// select on volume servers
|
||||
message QueryRequest {
|
||||
repeated string selections = 1;
|
||||
|
|
|
@ -3901,6 +3901,7 @@ func (x *VolumeInfo) GetReplication() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
// tiered storage
|
||||
type VolumeTierMoveDatToRemoteRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
|
@ -4314,6 +4315,181 @@ func (*VolumeServerLeaveResponse) Descriptor() ([]byte, []int) {
|
|||
return file_volume_server_proto_rawDescGZIP(), []int{77}
|
||||
}
|
||||
|
||||
// remote storage
|
||||
type FetchAndWriteNeedleRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"`
|
||||
NeedleId uint64 `protobuf:"varint,2,opt,name=needle_id,json=needleId,proto3" json:"needle_id,omitempty"`
|
||||
Offset int64 `protobuf:"varint,3,opt,name=offset,proto3" json:"offset,omitempty"`
|
||||
Size int64 `protobuf:"varint,4,opt,name=size,proto3" json:"size,omitempty"`
|
||||
// remote conf
|
||||
RemoteType string `protobuf:"bytes,5,opt,name=remote_type,json=remoteType,proto3" json:"remote_type,omitempty"`
|
||||
RemoteName string `protobuf:"bytes,6,opt,name=remote_name,json=remoteName,proto3" json:"remote_name,omitempty"`
|
||||
S3AccessKey string `protobuf:"bytes,8,opt,name=s3_access_key,json=s3AccessKey,proto3" json:"s3_access_key,omitempty"`
|
||||
S3SecretKey string `protobuf:"bytes,9,opt,name=s3_secret_key,json=s3SecretKey,proto3" json:"s3_secret_key,omitempty"`
|
||||
S3Region string `protobuf:"bytes,10,opt,name=s3_region,json=s3Region,proto3" json:"s3_region,omitempty"`
|
||||
S3Endpoint string `protobuf:"bytes,11,opt,name=s3_endpoint,json=s3Endpoint,proto3" json:"s3_endpoint,omitempty"`
|
||||
RemoteBucket string `protobuf:"bytes,12,opt,name=remote_bucket,json=remoteBucket,proto3" json:"remote_bucket,omitempty"`
|
||||
RemoteKey string `protobuf:"bytes,13,opt,name=remote_key,json=remoteKey,proto3" json:"remote_key,omitempty"`
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) Reset() {
|
||||
*x = FetchAndWriteNeedleRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[78]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*FetchAndWriteNeedleRequest) ProtoMessage() {}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[78]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use FetchAndWriteNeedleRequest.ProtoReflect.Descriptor instead.
|
||||
func (*FetchAndWriteNeedleRequest) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78}
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetVolumeId() uint32 {
|
||||
if x != nil {
|
||||
return x.VolumeId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetNeedleId() uint64 {
|
||||
if x != nil {
|
||||
return x.NeedleId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetOffset() int64 {
|
||||
if x != nil {
|
||||
return x.Offset
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetSize() int64 {
|
||||
if x != nil {
|
||||
return x.Size
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetRemoteType() string {
|
||||
if x != nil {
|
||||
return x.RemoteType
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetRemoteName() string {
|
||||
if x != nil {
|
||||
return x.RemoteName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetS3AccessKey() string {
|
||||
if x != nil {
|
||||
return x.S3AccessKey
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetS3SecretKey() string {
|
||||
if x != nil {
|
||||
return x.S3SecretKey
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetS3Region() string {
|
||||
if x != nil {
|
||||
return x.S3Region
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetS3Endpoint() string {
|
||||
if x != nil {
|
||||
return x.S3Endpoint
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetRemoteBucket() string {
|
||||
if x != nil {
|
||||
return x.RemoteBucket
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleRequest) GetRemoteKey() string {
|
||||
if x != nil {
|
||||
return x.RemoteKey
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type FetchAndWriteNeedleResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleResponse) Reset() {
|
||||
*x = FetchAndWriteNeedleResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[79]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *FetchAndWriteNeedleResponse) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*FetchAndWriteNeedleResponse) ProtoMessage() {}
|
||||
|
||||
func (x *FetchAndWriteNeedleResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[79]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use FetchAndWriteNeedleResponse.ProtoReflect.Descriptor instead.
|
||||
func (*FetchAndWriteNeedleResponse) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{79}
|
||||
}
|
||||
|
||||
// select on volume servers
|
||||
type QueryRequest struct {
|
||||
state protoimpl.MessageState
|
||||
|
@ -4330,7 +4506,7 @@ type QueryRequest struct {
|
|||
func (x *QueryRequest) Reset() {
|
||||
*x = QueryRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[78]
|
||||
mi := &file_volume_server_proto_msgTypes[80]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4343,7 +4519,7 @@ func (x *QueryRequest) String() string {
|
|||
func (*QueryRequest) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[78]
|
||||
mi := &file_volume_server_proto_msgTypes[80]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4356,7 +4532,7 @@ func (x *QueryRequest) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use QueryRequest.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80}
|
||||
}
|
||||
|
||||
func (x *QueryRequest) GetSelections() []string {
|
||||
|
@ -4405,7 +4581,7 @@ type QueriedStripe struct {
|
|||
func (x *QueriedStripe) Reset() {
|
||||
*x = QueriedStripe{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[79]
|
||||
mi := &file_volume_server_proto_msgTypes[81]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4418,7 +4594,7 @@ func (x *QueriedStripe) String() string {
|
|||
func (*QueriedStripe) ProtoMessage() {}
|
||||
|
||||
func (x *QueriedStripe) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[79]
|
||||
mi := &file_volume_server_proto_msgTypes[81]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4431,7 +4607,7 @@ func (x *QueriedStripe) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use QueriedStripe.ProtoReflect.Descriptor instead.
|
||||
func (*QueriedStripe) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{79}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{81}
|
||||
}
|
||||
|
||||
func (x *QueriedStripe) GetRecords() []byte {
|
||||
|
@ -4453,7 +4629,7 @@ type VolumeNeedleStatusRequest struct {
|
|||
func (x *VolumeNeedleStatusRequest) Reset() {
|
||||
*x = VolumeNeedleStatusRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[80]
|
||||
mi := &file_volume_server_proto_msgTypes[82]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4466,7 +4642,7 @@ func (x *VolumeNeedleStatusRequest) String() string {
|
|||
func (*VolumeNeedleStatusRequest) ProtoMessage() {}
|
||||
|
||||
func (x *VolumeNeedleStatusRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[80]
|
||||
mi := &file_volume_server_proto_msgTypes[82]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4479,7 +4655,7 @@ func (x *VolumeNeedleStatusRequest) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use VolumeNeedleStatusRequest.ProtoReflect.Descriptor instead.
|
||||
func (*VolumeNeedleStatusRequest) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{82}
|
||||
}
|
||||
|
||||
func (x *VolumeNeedleStatusRequest) GetVolumeId() uint32 {
|
||||
|
@ -4512,7 +4688,7 @@ type VolumeNeedleStatusResponse struct {
|
|||
func (x *VolumeNeedleStatusResponse) Reset() {
|
||||
*x = VolumeNeedleStatusResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[81]
|
||||
mi := &file_volume_server_proto_msgTypes[83]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4525,7 +4701,7 @@ func (x *VolumeNeedleStatusResponse) String() string {
|
|||
func (*VolumeNeedleStatusResponse) ProtoMessage() {}
|
||||
|
||||
func (x *VolumeNeedleStatusResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[81]
|
||||
mi := &file_volume_server_proto_msgTypes[83]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4538,7 +4714,7 @@ func (x *VolumeNeedleStatusResponse) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use VolumeNeedleStatusResponse.ProtoReflect.Descriptor instead.
|
||||
func (*VolumeNeedleStatusResponse) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{81}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{83}
|
||||
}
|
||||
|
||||
func (x *VolumeNeedleStatusResponse) GetNeedleId() uint64 {
|
||||
|
@ -4596,7 +4772,7 @@ type QueryRequest_Filter struct {
|
|||
func (x *QueryRequest_Filter) Reset() {
|
||||
*x = QueryRequest_Filter{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[82]
|
||||
mi := &file_volume_server_proto_msgTypes[84]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4609,7 +4785,7 @@ func (x *QueryRequest_Filter) String() string {
|
|||
func (*QueryRequest_Filter) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_Filter) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[82]
|
||||
mi := &file_volume_server_proto_msgTypes[84]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4622,7 +4798,7 @@ func (x *QueryRequest_Filter) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use QueryRequest_Filter.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_Filter) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 0}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 0}
|
||||
}
|
||||
|
||||
func (x *QueryRequest_Filter) GetField() string {
|
||||
|
@ -4661,7 +4837,7 @@ type QueryRequest_InputSerialization struct {
|
|||
func (x *QueryRequest_InputSerialization) Reset() {
|
||||
*x = QueryRequest_InputSerialization{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[83]
|
||||
mi := &file_volume_server_proto_msgTypes[85]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4674,7 +4850,7 @@ func (x *QueryRequest_InputSerialization) String() string {
|
|||
func (*QueryRequest_InputSerialization) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_InputSerialization) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[83]
|
||||
mi := &file_volume_server_proto_msgTypes[85]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4687,7 +4863,7 @@ func (x *QueryRequest_InputSerialization) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use QueryRequest_InputSerialization.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_InputSerialization) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 1}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 1}
|
||||
}
|
||||
|
||||
func (x *QueryRequest_InputSerialization) GetCompressionType() string {
|
||||
|
@ -4730,7 +4906,7 @@ type QueryRequest_OutputSerialization struct {
|
|||
func (x *QueryRequest_OutputSerialization) Reset() {
|
||||
*x = QueryRequest_OutputSerialization{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[84]
|
||||
mi := &file_volume_server_proto_msgTypes[86]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4743,7 +4919,7 @@ func (x *QueryRequest_OutputSerialization) String() string {
|
|||
func (*QueryRequest_OutputSerialization) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_OutputSerialization) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[84]
|
||||
mi := &file_volume_server_proto_msgTypes[86]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4756,7 +4932,7 @@ func (x *QueryRequest_OutputSerialization) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use QueryRequest_OutputSerialization.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_OutputSerialization) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 2}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 2}
|
||||
}
|
||||
|
||||
func (x *QueryRequest_OutputSerialization) GetCsvOutput() *QueryRequest_OutputSerialization_CSVOutput {
|
||||
|
@ -4791,7 +4967,7 @@ type QueryRequest_InputSerialization_CSVInput struct {
|
|||
func (x *QueryRequest_InputSerialization_CSVInput) Reset() {
|
||||
*x = QueryRequest_InputSerialization_CSVInput{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[85]
|
||||
mi := &file_volume_server_proto_msgTypes[87]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4804,7 +4980,7 @@ func (x *QueryRequest_InputSerialization_CSVInput) String() string {
|
|||
func (*QueryRequest_InputSerialization_CSVInput) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_InputSerialization_CSVInput) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[85]
|
||||
mi := &file_volume_server_proto_msgTypes[87]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4817,7 +4993,7 @@ func (x *QueryRequest_InputSerialization_CSVInput) ProtoReflect() protoreflect.M
|
|||
|
||||
// Deprecated: Use QueryRequest_InputSerialization_CSVInput.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_InputSerialization_CSVInput) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 0}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 1, 0}
|
||||
}
|
||||
|
||||
func (x *QueryRequest_InputSerialization_CSVInput) GetFileHeaderInfo() string {
|
||||
|
@ -4880,7 +5056,7 @@ type QueryRequest_InputSerialization_JSONInput struct {
|
|||
func (x *QueryRequest_InputSerialization_JSONInput) Reset() {
|
||||
*x = QueryRequest_InputSerialization_JSONInput{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[86]
|
||||
mi := &file_volume_server_proto_msgTypes[88]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4893,7 +5069,7 @@ func (x *QueryRequest_InputSerialization_JSONInput) String() string {
|
|||
func (*QueryRequest_InputSerialization_JSONInput) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_InputSerialization_JSONInput) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[86]
|
||||
mi := &file_volume_server_proto_msgTypes[88]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4906,7 +5082,7 @@ func (x *QueryRequest_InputSerialization_JSONInput) ProtoReflect() protoreflect.
|
|||
|
||||
// Deprecated: Use QueryRequest_InputSerialization_JSONInput.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_InputSerialization_JSONInput) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 1}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 1, 1}
|
||||
}
|
||||
|
||||
func (x *QueryRequest_InputSerialization_JSONInput) GetType() string {
|
||||
|
@ -4925,7 +5101,7 @@ type QueryRequest_InputSerialization_ParquetInput struct {
|
|||
func (x *QueryRequest_InputSerialization_ParquetInput) Reset() {
|
||||
*x = QueryRequest_InputSerialization_ParquetInput{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[87]
|
||||
mi := &file_volume_server_proto_msgTypes[89]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4938,7 +5114,7 @@ func (x *QueryRequest_InputSerialization_ParquetInput) String() string {
|
|||
func (*QueryRequest_InputSerialization_ParquetInput) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_InputSerialization_ParquetInput) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[87]
|
||||
mi := &file_volume_server_proto_msgTypes[89]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4951,7 +5127,7 @@ func (x *QueryRequest_InputSerialization_ParquetInput) ProtoReflect() protorefle
|
|||
|
||||
// Deprecated: Use QueryRequest_InputSerialization_ParquetInput.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_InputSerialization_ParquetInput) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 1, 2}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 1, 2}
|
||||
}
|
||||
|
||||
type QueryRequest_OutputSerialization_CSVOutput struct {
|
||||
|
@ -4969,7 +5145,7 @@ type QueryRequest_OutputSerialization_CSVOutput struct {
|
|||
func (x *QueryRequest_OutputSerialization_CSVOutput) Reset() {
|
||||
*x = QueryRequest_OutputSerialization_CSVOutput{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[88]
|
||||
mi := &file_volume_server_proto_msgTypes[90]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -4982,7 +5158,7 @@ func (x *QueryRequest_OutputSerialization_CSVOutput) String() string {
|
|||
func (*QueryRequest_OutputSerialization_CSVOutput) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_OutputSerialization_CSVOutput) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[88]
|
||||
mi := &file_volume_server_proto_msgTypes[90]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -4995,7 +5171,7 @@ func (x *QueryRequest_OutputSerialization_CSVOutput) ProtoReflect() protoreflect
|
|||
|
||||
// Deprecated: Use QueryRequest_OutputSerialization_CSVOutput.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_OutputSerialization_CSVOutput) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 2, 0}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 2, 0}
|
||||
}
|
||||
|
||||
func (x *QueryRequest_OutputSerialization_CSVOutput) GetQuoteFields() string {
|
||||
|
@ -5044,7 +5220,7 @@ type QueryRequest_OutputSerialization_JSONOutput struct {
|
|||
func (x *QueryRequest_OutputSerialization_JSONOutput) Reset() {
|
||||
*x = QueryRequest_OutputSerialization_JSONOutput{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_volume_server_proto_msgTypes[89]
|
||||
mi := &file_volume_server_proto_msgTypes[91]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -5057,7 +5233,7 @@ func (x *QueryRequest_OutputSerialization_JSONOutput) String() string {
|
|||
func (*QueryRequest_OutputSerialization_JSONOutput) ProtoMessage() {}
|
||||
|
||||
func (x *QueryRequest_OutputSerialization_JSONOutput) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_volume_server_proto_msgTypes[89]
|
||||
mi := &file_volume_server_proto_msgTypes[91]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -5070,7 +5246,7 @@ func (x *QueryRequest_OutputSerialization_JSONOutput) ProtoReflect() protoreflec
|
|||
|
||||
// Deprecated: Use QueryRequest_OutputSerialization_JSONOutput.ProtoReflect.Descriptor instead.
|
||||
func (*QueryRequest_OutputSerialization_JSONOutput) Descriptor() ([]byte, []int) {
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{78, 2, 1}
|
||||
return file_volume_server_proto_rawDescGZIP(), []int{80, 2, 1}
|
||||
}
|
||||
|
||||
func (x *QueryRequest_OutputSerialization_JSONOutput) GetRecordDelimiter() string {
|
||||
|
@ -5539,6 +5715,33 @@ var file_volume_server_proto_rawDesc = []byte{
|
|||
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x76, 0x65,
|
||||
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x1b, 0x0a, 0x19, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
|
||||
0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70,
|
||||
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8e, 0x03, 0x0a, 0x1a, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e,
|
||||
0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75,
|
||||
0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64,
|
||||
0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x04, 0x52, 0x08, 0x6e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a,
|
||||
0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f,
|
||||
0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20,
|
||||
0x01, 0x28, 0x03, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6d,
|
||||
0x6f, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a,
|
||||
0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65,
|
||||
0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x0a, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x73,
|
||||
0x33, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x08, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x0b, 0x73, 0x33, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x12,
|
||||
0x22, 0x0a, 0x0d, 0x73, 0x33, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, 0x6b, 0x65, 0x79,
|
||||
0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x33, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74,
|
||||
0x4b, 0x65, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x33, 0x5f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e,
|
||||
0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x33, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e,
|
||||
0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x33, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18,
|
||||
0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x33, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e,
|
||||
0x74, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x62, 0x75, 0x63, 0x6b,
|
||||
0x65, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65,
|
||||
0x42, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65,
|
||||
0x5f, 0x6b, 0x65, 0x79, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x6d, 0x6f,
|
||||
0x74, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x1d, 0x0a, 0x1b, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e,
|
||||
0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70,
|
||||
0x6f, 0x6e, 0x73, 0x65, 0x22, 0xf8, 0x0c, 0x0a, 0x0c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63,
|
||||
|
@ -5662,7 +5865,7 @@ var file_volume_server_proto_rawDesc = []byte{
|
|||
0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x12, 0x10,
|
||||
0x0a, 0x03, 0x63, 0x72, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x63, 0x72, 0x63,
|
||||
0x12, 0x10, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74,
|
||||
0x74, 0x6c, 0x32, 0xa9, 0x21, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72,
|
||||
0x74, 0x6c, 0x32, 0x9f, 0x22, 0x0a, 0x0c, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72,
|
||||
0x76, 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x0b, 0x42, 0x61, 0x74, 0x63, 0x68, 0x44, 0x65, 0x6c, 0x65,
|
||||
0x74, 0x65, 0x12, 0x24, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76,
|
||||
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x44, 0x65, 0x6c, 0x65, 0x74,
|
||||
|
@ -5916,24 +6119,31 @@ var file_volume_server_proto_rawDesc = []byte{
|
|||
0x65, 0x72, 0x4c, 0x65, 0x61, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b,
|
||||
0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70,
|
||||
0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4c, 0x65,
|
||||
0x61, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a,
|
||||
0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x1e, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
|
||||
0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
|
||||
0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x69, 0x65,
|
||||
0x64, 0x53, 0x74, 0x72, 0x69, 0x70, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x71, 0x0a, 0x12, 0x56,
|
||||
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75,
|
||||
0x73, 0x12, 0x2b, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65,
|
||||
0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c,
|
||||
0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c,
|
||||
0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70,
|
||||
0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x53, 0x74,
|
||||
0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x39,
|
||||
0x5a, 0x37, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72,
|
||||
0x69, 0x73, 0x6c, 0x75, 0x73, 0x66, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73,
|
||||
0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f,
|
||||
0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x33,
|
||||
0x61, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x74, 0x0a,
|
||||
0x13, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, 0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65,
|
||||
0x65, 0x64, 0x6c, 0x65, 0x12, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65,
|
||||
0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, 0x64,
|
||||
0x57, 0x72, 0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76,
|
||||
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x6e, 0x64, 0x57, 0x72,
|
||||
0x69, 0x74, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
|
||||
0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x1e, 0x2e, 0x76,
|
||||
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
|
||||
0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76,
|
||||
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
|
||||
0x51, 0x75, 0x65, 0x72, 0x69, 0x65, 0x64, 0x53, 0x74, 0x72, 0x69, 0x70, 0x65, 0x22, 0x00, 0x30,
|
||||
0x01, 0x12, 0x71, 0x0a, 0x12, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c,
|
||||
0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2b, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
|
||||
0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d,
|
||||
0x65, 0x4e, 0x65, 0x65, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71,
|
||||
0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65,
|
||||
0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4e, 0x65,
|
||||
0x65, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x42, 0x39, 0x5a, 0x37, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
|
||||
0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x69, 0x73, 0x6c, 0x75, 0x73, 0x66, 0x2f, 0x73, 0x65, 0x61,
|
||||
0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x76,
|
||||
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62,
|
||||
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
@ -5948,7 +6158,7 @@ func file_volume_server_proto_rawDescGZIP() []byte {
|
|||
return file_volume_server_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_volume_server_proto_msgTypes = make([]protoimpl.MessageInfo, 90)
|
||||
var file_volume_server_proto_msgTypes = make([]protoimpl.MessageInfo, 92)
|
||||
var file_volume_server_proto_goTypes = []interface{}{
|
||||
(*BatchDeleteRequest)(nil), // 0: volume_server_pb.BatchDeleteRequest
|
||||
(*BatchDeleteResponse)(nil), // 1: volume_server_pb.BatchDeleteResponse
|
||||
|
@ -6028,32 +6238,34 @@ var file_volume_server_proto_goTypes = []interface{}{
|
|||
(*VolumeServerStatusResponse)(nil), // 75: volume_server_pb.VolumeServerStatusResponse
|
||||
(*VolumeServerLeaveRequest)(nil), // 76: volume_server_pb.VolumeServerLeaveRequest
|
||||
(*VolumeServerLeaveResponse)(nil), // 77: volume_server_pb.VolumeServerLeaveResponse
|
||||
(*QueryRequest)(nil), // 78: volume_server_pb.QueryRequest
|
||||
(*QueriedStripe)(nil), // 79: volume_server_pb.QueriedStripe
|
||||
(*VolumeNeedleStatusRequest)(nil), // 80: volume_server_pb.VolumeNeedleStatusRequest
|
||||
(*VolumeNeedleStatusResponse)(nil), // 81: volume_server_pb.VolumeNeedleStatusResponse
|
||||
(*QueryRequest_Filter)(nil), // 82: volume_server_pb.QueryRequest.Filter
|
||||
(*QueryRequest_InputSerialization)(nil), // 83: volume_server_pb.QueryRequest.InputSerialization
|
||||
(*QueryRequest_OutputSerialization)(nil), // 84: volume_server_pb.QueryRequest.OutputSerialization
|
||||
(*QueryRequest_InputSerialization_CSVInput)(nil), // 85: volume_server_pb.QueryRequest.InputSerialization.CSVInput
|
||||
(*QueryRequest_InputSerialization_JSONInput)(nil), // 86: volume_server_pb.QueryRequest.InputSerialization.JSONInput
|
||||
(*QueryRequest_InputSerialization_ParquetInput)(nil), // 87: volume_server_pb.QueryRequest.InputSerialization.ParquetInput
|
||||
(*QueryRequest_OutputSerialization_CSVOutput)(nil), // 88: volume_server_pb.QueryRequest.OutputSerialization.CSVOutput
|
||||
(*QueryRequest_OutputSerialization_JSONOutput)(nil), // 89: volume_server_pb.QueryRequest.OutputSerialization.JSONOutput
|
||||
(*FetchAndWriteNeedleRequest)(nil), // 78: volume_server_pb.FetchAndWriteNeedleRequest
|
||||
(*FetchAndWriteNeedleResponse)(nil), // 79: volume_server_pb.FetchAndWriteNeedleResponse
|
||||
(*QueryRequest)(nil), // 80: volume_server_pb.QueryRequest
|
||||
(*QueriedStripe)(nil), // 81: volume_server_pb.QueriedStripe
|
||||
(*VolumeNeedleStatusRequest)(nil), // 82: volume_server_pb.VolumeNeedleStatusRequest
|
||||
(*VolumeNeedleStatusResponse)(nil), // 83: volume_server_pb.VolumeNeedleStatusResponse
|
||||
(*QueryRequest_Filter)(nil), // 84: volume_server_pb.QueryRequest.Filter
|
||||
(*QueryRequest_InputSerialization)(nil), // 85: volume_server_pb.QueryRequest.InputSerialization
|
||||
(*QueryRequest_OutputSerialization)(nil), // 86: volume_server_pb.QueryRequest.OutputSerialization
|
||||
(*QueryRequest_InputSerialization_CSVInput)(nil), // 87: volume_server_pb.QueryRequest.InputSerialization.CSVInput
|
||||
(*QueryRequest_InputSerialization_JSONInput)(nil), // 88: volume_server_pb.QueryRequest.InputSerialization.JSONInput
|
||||
(*QueryRequest_InputSerialization_ParquetInput)(nil), // 89: volume_server_pb.QueryRequest.InputSerialization.ParquetInput
|
||||
(*QueryRequest_OutputSerialization_CSVOutput)(nil), // 90: volume_server_pb.QueryRequest.OutputSerialization.CSVOutput
|
||||
(*QueryRequest_OutputSerialization_JSONOutput)(nil), // 91: volume_server_pb.QueryRequest.OutputSerialization.JSONOutput
|
||||
}
|
||||
var file_volume_server_proto_depIdxs = []int32{
|
||||
2, // 0: volume_server_pb.BatchDeleteResponse.results:type_name -> volume_server_pb.DeleteResult
|
||||
68, // 1: volume_server_pb.VolumeInfo.files:type_name -> volume_server_pb.RemoteFile
|
||||
66, // 2: volume_server_pb.VolumeServerStatusResponse.disk_statuses:type_name -> volume_server_pb.DiskStatus
|
||||
67, // 3: volume_server_pb.VolumeServerStatusResponse.memory_status:type_name -> volume_server_pb.MemStatus
|
||||
82, // 4: volume_server_pb.QueryRequest.filter:type_name -> volume_server_pb.QueryRequest.Filter
|
||||
83, // 5: volume_server_pb.QueryRequest.input_serialization:type_name -> volume_server_pb.QueryRequest.InputSerialization
|
||||
84, // 6: volume_server_pb.QueryRequest.output_serialization:type_name -> volume_server_pb.QueryRequest.OutputSerialization
|
||||
85, // 7: volume_server_pb.QueryRequest.InputSerialization.csv_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.CSVInput
|
||||
86, // 8: volume_server_pb.QueryRequest.InputSerialization.json_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.JSONInput
|
||||
87, // 9: volume_server_pb.QueryRequest.InputSerialization.parquet_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.ParquetInput
|
||||
88, // 10: volume_server_pb.QueryRequest.OutputSerialization.csv_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.CSVOutput
|
||||
89, // 11: volume_server_pb.QueryRequest.OutputSerialization.json_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.JSONOutput
|
||||
84, // 4: volume_server_pb.QueryRequest.filter:type_name -> volume_server_pb.QueryRequest.Filter
|
||||
85, // 5: volume_server_pb.QueryRequest.input_serialization:type_name -> volume_server_pb.QueryRequest.InputSerialization
|
||||
86, // 6: volume_server_pb.QueryRequest.output_serialization:type_name -> volume_server_pb.QueryRequest.OutputSerialization
|
||||
87, // 7: volume_server_pb.QueryRequest.InputSerialization.csv_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.CSVInput
|
||||
88, // 8: volume_server_pb.QueryRequest.InputSerialization.json_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.JSONInput
|
||||
89, // 9: volume_server_pb.QueryRequest.InputSerialization.parquet_input:type_name -> volume_server_pb.QueryRequest.InputSerialization.ParquetInput
|
||||
90, // 10: volume_server_pb.QueryRequest.OutputSerialization.csv_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.CSVOutput
|
||||
91, // 11: volume_server_pb.QueryRequest.OutputSerialization.json_output:type_name -> volume_server_pb.QueryRequest.OutputSerialization.JSONOutput
|
||||
0, // 12: volume_server_pb.VolumeServer.BatchDelete:input_type -> volume_server_pb.BatchDeleteRequest
|
||||
4, // 13: volume_server_pb.VolumeServer.VacuumVolumeCheck:input_type -> volume_server_pb.VacuumVolumeCheckRequest
|
||||
6, // 14: volume_server_pb.VolumeServer.VacuumVolumeCompact:input_type -> volume_server_pb.VacuumVolumeCompactRequest
|
||||
|
@ -6090,48 +6302,50 @@ var file_volume_server_proto_depIdxs = []int32{
|
|||
72, // 45: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:input_type -> volume_server_pb.VolumeTierMoveDatFromRemoteRequest
|
||||
74, // 46: volume_server_pb.VolumeServer.VolumeServerStatus:input_type -> volume_server_pb.VolumeServerStatusRequest
|
||||
76, // 47: volume_server_pb.VolumeServer.VolumeServerLeave:input_type -> volume_server_pb.VolumeServerLeaveRequest
|
||||
78, // 48: volume_server_pb.VolumeServer.Query:input_type -> volume_server_pb.QueryRequest
|
||||
80, // 49: volume_server_pb.VolumeServer.VolumeNeedleStatus:input_type -> volume_server_pb.VolumeNeedleStatusRequest
|
||||
1, // 50: volume_server_pb.VolumeServer.BatchDelete:output_type -> volume_server_pb.BatchDeleteResponse
|
||||
5, // 51: volume_server_pb.VolumeServer.VacuumVolumeCheck:output_type -> volume_server_pb.VacuumVolumeCheckResponse
|
||||
7, // 52: volume_server_pb.VolumeServer.VacuumVolumeCompact:output_type -> volume_server_pb.VacuumVolumeCompactResponse
|
||||
9, // 53: volume_server_pb.VolumeServer.VacuumVolumeCommit:output_type -> volume_server_pb.VacuumVolumeCommitResponse
|
||||
11, // 54: volume_server_pb.VolumeServer.VacuumVolumeCleanup:output_type -> volume_server_pb.VacuumVolumeCleanupResponse
|
||||
13, // 55: volume_server_pb.VolumeServer.DeleteCollection:output_type -> volume_server_pb.DeleteCollectionResponse
|
||||
15, // 56: volume_server_pb.VolumeServer.AllocateVolume:output_type -> volume_server_pb.AllocateVolumeResponse
|
||||
17, // 57: volume_server_pb.VolumeServer.VolumeSyncStatus:output_type -> volume_server_pb.VolumeSyncStatusResponse
|
||||
19, // 58: volume_server_pb.VolumeServer.VolumeIncrementalCopy:output_type -> volume_server_pb.VolumeIncrementalCopyResponse
|
||||
21, // 59: volume_server_pb.VolumeServer.VolumeMount:output_type -> volume_server_pb.VolumeMountResponse
|
||||
23, // 60: volume_server_pb.VolumeServer.VolumeUnmount:output_type -> volume_server_pb.VolumeUnmountResponse
|
||||
25, // 61: volume_server_pb.VolumeServer.VolumeDelete:output_type -> volume_server_pb.VolumeDeleteResponse
|
||||
27, // 62: volume_server_pb.VolumeServer.VolumeMarkReadonly:output_type -> volume_server_pb.VolumeMarkReadonlyResponse
|
||||
29, // 63: volume_server_pb.VolumeServer.VolumeMarkWritable:output_type -> volume_server_pb.VolumeMarkWritableResponse
|
||||
31, // 64: volume_server_pb.VolumeServer.VolumeConfigure:output_type -> volume_server_pb.VolumeConfigureResponse
|
||||
33, // 65: volume_server_pb.VolumeServer.VolumeStatus:output_type -> volume_server_pb.VolumeStatusResponse
|
||||
35, // 66: volume_server_pb.VolumeServer.VolumeCopy:output_type -> volume_server_pb.VolumeCopyResponse
|
||||
65, // 67: volume_server_pb.VolumeServer.ReadVolumeFileStatus:output_type -> volume_server_pb.ReadVolumeFileStatusResponse
|
||||
37, // 68: volume_server_pb.VolumeServer.CopyFile:output_type -> volume_server_pb.CopyFileResponse
|
||||
39, // 69: volume_server_pb.VolumeServer.ReadNeedleBlob:output_type -> volume_server_pb.ReadNeedleBlobResponse
|
||||
41, // 70: volume_server_pb.VolumeServer.WriteNeedleBlob:output_type -> volume_server_pb.WriteNeedleBlobResponse
|
||||
43, // 71: volume_server_pb.VolumeServer.VolumeTailSender:output_type -> volume_server_pb.VolumeTailSenderResponse
|
||||
45, // 72: volume_server_pb.VolumeServer.VolumeTailReceiver:output_type -> volume_server_pb.VolumeTailReceiverResponse
|
||||
47, // 73: volume_server_pb.VolumeServer.VolumeEcShardsGenerate:output_type -> volume_server_pb.VolumeEcShardsGenerateResponse
|
||||
49, // 74: volume_server_pb.VolumeServer.VolumeEcShardsRebuild:output_type -> volume_server_pb.VolumeEcShardsRebuildResponse
|
||||
51, // 75: volume_server_pb.VolumeServer.VolumeEcShardsCopy:output_type -> volume_server_pb.VolumeEcShardsCopyResponse
|
||||
53, // 76: volume_server_pb.VolumeServer.VolumeEcShardsDelete:output_type -> volume_server_pb.VolumeEcShardsDeleteResponse
|
||||
55, // 77: volume_server_pb.VolumeServer.VolumeEcShardsMount:output_type -> volume_server_pb.VolumeEcShardsMountResponse
|
||||
57, // 78: volume_server_pb.VolumeServer.VolumeEcShardsUnmount:output_type -> volume_server_pb.VolumeEcShardsUnmountResponse
|
||||
59, // 79: volume_server_pb.VolumeServer.VolumeEcShardRead:output_type -> volume_server_pb.VolumeEcShardReadResponse
|
||||
61, // 80: volume_server_pb.VolumeServer.VolumeEcBlobDelete:output_type -> volume_server_pb.VolumeEcBlobDeleteResponse
|
||||
63, // 81: volume_server_pb.VolumeServer.VolumeEcShardsToVolume:output_type -> volume_server_pb.VolumeEcShardsToVolumeResponse
|
||||
71, // 82: volume_server_pb.VolumeServer.VolumeTierMoveDatToRemote:output_type -> volume_server_pb.VolumeTierMoveDatToRemoteResponse
|
||||
73, // 83: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:output_type -> volume_server_pb.VolumeTierMoveDatFromRemoteResponse
|
||||
75, // 84: volume_server_pb.VolumeServer.VolumeServerStatus:output_type -> volume_server_pb.VolumeServerStatusResponse
|
||||
77, // 85: volume_server_pb.VolumeServer.VolumeServerLeave:output_type -> volume_server_pb.VolumeServerLeaveResponse
|
||||
79, // 86: volume_server_pb.VolumeServer.Query:output_type -> volume_server_pb.QueriedStripe
|
||||
81, // 87: volume_server_pb.VolumeServer.VolumeNeedleStatus:output_type -> volume_server_pb.VolumeNeedleStatusResponse
|
||||
50, // [50:88] is the sub-list for method output_type
|
||||
12, // [12:50] is the sub-list for method input_type
|
||||
78, // 48: volume_server_pb.VolumeServer.FetchAndWriteNeedle:input_type -> volume_server_pb.FetchAndWriteNeedleRequest
|
||||
80, // 49: volume_server_pb.VolumeServer.Query:input_type -> volume_server_pb.QueryRequest
|
||||
82, // 50: volume_server_pb.VolumeServer.VolumeNeedleStatus:input_type -> volume_server_pb.VolumeNeedleStatusRequest
|
||||
1, // 51: volume_server_pb.VolumeServer.BatchDelete:output_type -> volume_server_pb.BatchDeleteResponse
|
||||
5, // 52: volume_server_pb.VolumeServer.VacuumVolumeCheck:output_type -> volume_server_pb.VacuumVolumeCheckResponse
|
||||
7, // 53: volume_server_pb.VolumeServer.VacuumVolumeCompact:output_type -> volume_server_pb.VacuumVolumeCompactResponse
|
||||
9, // 54: volume_server_pb.VolumeServer.VacuumVolumeCommit:output_type -> volume_server_pb.VacuumVolumeCommitResponse
|
||||
11, // 55: volume_server_pb.VolumeServer.VacuumVolumeCleanup:output_type -> volume_server_pb.VacuumVolumeCleanupResponse
|
||||
13, // 56: volume_server_pb.VolumeServer.DeleteCollection:output_type -> volume_server_pb.DeleteCollectionResponse
|
||||
15, // 57: volume_server_pb.VolumeServer.AllocateVolume:output_type -> volume_server_pb.AllocateVolumeResponse
|
||||
17, // 58: volume_server_pb.VolumeServer.VolumeSyncStatus:output_type -> volume_server_pb.VolumeSyncStatusResponse
|
||||
19, // 59: volume_server_pb.VolumeServer.VolumeIncrementalCopy:output_type -> volume_server_pb.VolumeIncrementalCopyResponse
|
||||
21, // 60: volume_server_pb.VolumeServer.VolumeMount:output_type -> volume_server_pb.VolumeMountResponse
|
||||
23, // 61: volume_server_pb.VolumeServer.VolumeUnmount:output_type -> volume_server_pb.VolumeUnmountResponse
|
||||
25, // 62: volume_server_pb.VolumeServer.VolumeDelete:output_type -> volume_server_pb.VolumeDeleteResponse
|
||||
27, // 63: volume_server_pb.VolumeServer.VolumeMarkReadonly:output_type -> volume_server_pb.VolumeMarkReadonlyResponse
|
||||
29, // 64: volume_server_pb.VolumeServer.VolumeMarkWritable:output_type -> volume_server_pb.VolumeMarkWritableResponse
|
||||
31, // 65: volume_server_pb.VolumeServer.VolumeConfigure:output_type -> volume_server_pb.VolumeConfigureResponse
|
||||
33, // 66: volume_server_pb.VolumeServer.VolumeStatus:output_type -> volume_server_pb.VolumeStatusResponse
|
||||
35, // 67: volume_server_pb.VolumeServer.VolumeCopy:output_type -> volume_server_pb.VolumeCopyResponse
|
||||
65, // 68: volume_server_pb.VolumeServer.ReadVolumeFileStatus:output_type -> volume_server_pb.ReadVolumeFileStatusResponse
|
||||
37, // 69: volume_server_pb.VolumeServer.CopyFile:output_type -> volume_server_pb.CopyFileResponse
|
||||
39, // 70: volume_server_pb.VolumeServer.ReadNeedleBlob:output_type -> volume_server_pb.ReadNeedleBlobResponse
|
||||
41, // 71: volume_server_pb.VolumeServer.WriteNeedleBlob:output_type -> volume_server_pb.WriteNeedleBlobResponse
|
||||
43, // 72: volume_server_pb.VolumeServer.VolumeTailSender:output_type -> volume_server_pb.VolumeTailSenderResponse
|
||||
45, // 73: volume_server_pb.VolumeServer.VolumeTailReceiver:output_type -> volume_server_pb.VolumeTailReceiverResponse
|
||||
47, // 74: volume_server_pb.VolumeServer.VolumeEcShardsGenerate:output_type -> volume_server_pb.VolumeEcShardsGenerateResponse
|
||||
49, // 75: volume_server_pb.VolumeServer.VolumeEcShardsRebuild:output_type -> volume_server_pb.VolumeEcShardsRebuildResponse
|
||||
51, // 76: volume_server_pb.VolumeServer.VolumeEcShardsCopy:output_type -> volume_server_pb.VolumeEcShardsCopyResponse
|
||||
53, // 77: volume_server_pb.VolumeServer.VolumeEcShardsDelete:output_type -> volume_server_pb.VolumeEcShardsDeleteResponse
|
||||
55, // 78: volume_server_pb.VolumeServer.VolumeEcShardsMount:output_type -> volume_server_pb.VolumeEcShardsMountResponse
|
||||
57, // 79: volume_server_pb.VolumeServer.VolumeEcShardsUnmount:output_type -> volume_server_pb.VolumeEcShardsUnmountResponse
|
||||
59, // 80: volume_server_pb.VolumeServer.VolumeEcShardRead:output_type -> volume_server_pb.VolumeEcShardReadResponse
|
||||
61, // 81: volume_server_pb.VolumeServer.VolumeEcBlobDelete:output_type -> volume_server_pb.VolumeEcBlobDeleteResponse
|
||||
63, // 82: volume_server_pb.VolumeServer.VolumeEcShardsToVolume:output_type -> volume_server_pb.VolumeEcShardsToVolumeResponse
|
||||
71, // 83: volume_server_pb.VolumeServer.VolumeTierMoveDatToRemote:output_type -> volume_server_pb.VolumeTierMoveDatToRemoteResponse
|
||||
73, // 84: volume_server_pb.VolumeServer.VolumeTierMoveDatFromRemote:output_type -> volume_server_pb.VolumeTierMoveDatFromRemoteResponse
|
||||
75, // 85: volume_server_pb.VolumeServer.VolumeServerStatus:output_type -> volume_server_pb.VolumeServerStatusResponse
|
||||
77, // 86: volume_server_pb.VolumeServer.VolumeServerLeave:output_type -> volume_server_pb.VolumeServerLeaveResponse
|
||||
79, // 87: volume_server_pb.VolumeServer.FetchAndWriteNeedle:output_type -> volume_server_pb.FetchAndWriteNeedleResponse
|
||||
81, // 88: volume_server_pb.VolumeServer.Query:output_type -> volume_server_pb.QueriedStripe
|
||||
83, // 89: volume_server_pb.VolumeServer.VolumeNeedleStatus:output_type -> volume_server_pb.VolumeNeedleStatusResponse
|
||||
51, // [51:90] is the sub-list for method output_type
|
||||
12, // [12:51] is the sub-list for method input_type
|
||||
12, // [12:12] is the sub-list for extension type_name
|
||||
12, // [12:12] is the sub-list for extension extendee
|
||||
0, // [0:12] is the sub-list for field type_name
|
||||
|
@ -7080,7 +7294,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest); i {
|
||||
switch v := v.(*FetchAndWriteNeedleRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7092,7 +7306,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueriedStripe); i {
|
||||
switch v := v.(*FetchAndWriteNeedleResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7104,7 +7318,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*VolumeNeedleStatusRequest); i {
|
||||
switch v := v.(*QueryRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7116,7 +7330,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*VolumeNeedleStatusResponse); i {
|
||||
switch v := v.(*QueriedStripe); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7128,7 +7342,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_Filter); i {
|
||||
switch v := v.(*VolumeNeedleStatusRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7140,7 +7354,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_InputSerialization); i {
|
||||
switch v := v.(*VolumeNeedleStatusResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7152,7 +7366,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_OutputSerialization); i {
|
||||
switch v := v.(*QueryRequest_Filter); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7164,7 +7378,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_InputSerialization_CSVInput); i {
|
||||
switch v := v.(*QueryRequest_InputSerialization); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7176,7 +7390,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_InputSerialization_JSONInput); i {
|
||||
switch v := v.(*QueryRequest_OutputSerialization); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7188,7 +7402,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_InputSerialization_ParquetInput); i {
|
||||
switch v := v.(*QueryRequest_InputSerialization_CSVInput); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7200,7 +7414,7 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_OutputSerialization_CSVOutput); i {
|
||||
switch v := v.(*QueryRequest_InputSerialization_JSONInput); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
@ -7212,6 +7426,30 @@ func file_volume_server_proto_init() {
|
|||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_InputSerialization_ParquetInput); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_OutputSerialization_CSVOutput); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_volume_server_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryRequest_OutputSerialization_JSONOutput); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
|
@ -7230,7 +7468,7 @@ func file_volume_server_proto_init() {
|
|||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_volume_server_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 90,
|
||||
NumMessages: 92,
|
||||
NumExtensions: 0,
|
||||
NumServices: 1,
|
||||
},
|
||||
|
@ -7296,6 +7534,8 @@ type VolumeServerClient interface {
|
|||
VolumeTierMoveDatFromRemote(ctx context.Context, in *VolumeTierMoveDatFromRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatFromRemoteClient, error)
|
||||
VolumeServerStatus(ctx context.Context, in *VolumeServerStatusRequest, opts ...grpc.CallOption) (*VolumeServerStatusResponse, error)
|
||||
VolumeServerLeave(ctx context.Context, in *VolumeServerLeaveRequest, opts ...grpc.CallOption) (*VolumeServerLeaveResponse, error)
|
||||
// remote storage
|
||||
FetchAndWriteNeedle(ctx context.Context, in *FetchAndWriteNeedleRequest, opts ...grpc.CallOption) (*FetchAndWriteNeedleResponse, error)
|
||||
// <experimental> query
|
||||
Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error)
|
||||
VolumeNeedleStatus(ctx context.Context, in *VolumeNeedleStatusRequest, opts ...grpc.CallOption) (*VolumeNeedleStatusResponse, error)
|
||||
|
@ -7771,6 +8011,15 @@ func (c *volumeServerClient) VolumeServerLeave(ctx context.Context, in *VolumeSe
|
|||
return out, nil
|
||||
}
|
||||
|
||||
func (c *volumeServerClient) FetchAndWriteNeedle(ctx context.Context, in *FetchAndWriteNeedleRequest, opts ...grpc.CallOption) (*FetchAndWriteNeedleResponse, error) {
|
||||
out := new(FetchAndWriteNeedleResponse)
|
||||
err := c.cc.Invoke(ctx, "/volume_server_pb.VolumeServer/FetchAndWriteNeedle", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *volumeServerClient) Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &_VolumeServer_serviceDesc.Streams[6], "/volume_server_pb.VolumeServer/Query", opts...)
|
||||
if err != nil {
|
||||
|
@ -7854,6 +8103,8 @@ type VolumeServerServer interface {
|
|||
VolumeTierMoveDatFromRemote(*VolumeTierMoveDatFromRemoteRequest, VolumeServer_VolumeTierMoveDatFromRemoteServer) error
|
||||
VolumeServerStatus(context.Context, *VolumeServerStatusRequest) (*VolumeServerStatusResponse, error)
|
||||
VolumeServerLeave(context.Context, *VolumeServerLeaveRequest) (*VolumeServerLeaveResponse, error)
|
||||
// remote storage
|
||||
FetchAndWriteNeedle(context.Context, *FetchAndWriteNeedleRequest) (*FetchAndWriteNeedleResponse, error)
|
||||
// <experimental> query
|
||||
Query(*QueryRequest, VolumeServer_QueryServer) error
|
||||
VolumeNeedleStatus(context.Context, *VolumeNeedleStatusRequest) (*VolumeNeedleStatusResponse, error)
|
||||
|
@ -7971,6 +8222,9 @@ func (*UnimplementedVolumeServerServer) VolumeServerStatus(context.Context, *Vol
|
|||
func (*UnimplementedVolumeServerServer) VolumeServerLeave(context.Context, *VolumeServerLeaveRequest) (*VolumeServerLeaveResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method VolumeServerLeave not implemented")
|
||||
}
|
||||
func (*UnimplementedVolumeServerServer) FetchAndWriteNeedle(context.Context, *FetchAndWriteNeedleRequest) (*FetchAndWriteNeedleResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method FetchAndWriteNeedle not implemented")
|
||||
}
|
||||
func (*UnimplementedVolumeServerServer) Query(*QueryRequest, VolumeServer_QueryServer) error {
|
||||
return status.Errorf(codes.Unimplemented, "method Query not implemented")
|
||||
}
|
||||
|
@ -8648,6 +8902,24 @@ func _VolumeServer_VolumeServerLeave_Handler(srv interface{}, ctx context.Contex
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _VolumeServer_FetchAndWriteNeedle_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(FetchAndWriteNeedleRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(VolumeServerServer).FetchAndWriteNeedle(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/volume_server_pb.VolumeServer/FetchAndWriteNeedle",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(VolumeServerServer).FetchAndWriteNeedle(ctx, req.(*FetchAndWriteNeedleRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _VolumeServer_Query_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
m := new(QueryRequest)
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
|
@ -8811,6 +9083,10 @@ var _VolumeServer_serviceDesc = grpc.ServiceDesc{
|
|||
MethodName: "VolumeServerLeave",
|
||||
Handler: _VolumeServer_VolumeServerLeave_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "FetchAndWriteNeedle",
|
||||
Handler: _VolumeServer_FetchAndWriteNeedle_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "VolumeNeedleStatus",
|
||||
Handler: _VolumeServer_VolumeNeedleStatus_Handler,
|
||||
|
|
75
weed/remote_storage/remote_storage.go
Normal file
75
weed/remote_storage/remote_storage.go
Normal file
|
@ -0,0 +1,75 @@
|
|||
package remote_storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"io"
|
||||
"strings"
|
||||
"sync"
|
||||
)
|
||||
|
||||
func ParseLocation(remote string) (loc *filer_pb.RemoteStorageLocation) {
|
||||
loc = &filer_pb.RemoteStorageLocation{}
|
||||
if strings.HasSuffix(string(remote), "/") {
|
||||
remote = remote[:len(remote)-1]
|
||||
}
|
||||
parts := strings.SplitN(string(remote), "/", 3)
|
||||
if len(parts) >= 1 {
|
||||
loc.Name = parts[0]
|
||||
}
|
||||
if len(parts) >= 2 {
|
||||
loc.Bucket = parts[1]
|
||||
}
|
||||
loc.Path = string(remote[len(loc.Name)+1+len(loc.Bucket):])
|
||||
if loc.Path == "" {
|
||||
loc.Path = "/"
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
type VisitFunc func(dir string, name string, isDirectory bool, remoteEntry *filer_pb.RemoteEntry) error
|
||||
|
||||
type RemoteStorageClient interface {
|
||||
Traverse(loc *filer_pb.RemoteStorageLocation, visitFn VisitFunc) error
|
||||
ReadFile(loc *filer_pb.RemoteStorageLocation, offset int64, size int64) (data []byte, err error)
|
||||
WriteFile(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry, reader io.Reader) (remoteEntry *filer_pb.RemoteEntry, err error)
|
||||
UpdateFileMetadata(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry) (err error)
|
||||
DeleteFile(loc *filer_pb.RemoteStorageLocation) (err error)
|
||||
}
|
||||
|
||||
type RemoteStorageClientMaker interface {
|
||||
Make(remoteConf *filer_pb.RemoteConf) (RemoteStorageClient, error)
|
||||
}
|
||||
|
||||
var (
|
||||
RemoteStorageClientMakers = make(map[string]RemoteStorageClientMaker)
|
||||
remoteStorageClients = make(map[string]RemoteStorageClient)
|
||||
remoteStorageClientsLock sync.Mutex
|
||||
)
|
||||
|
||||
func makeRemoteStorageClient(remoteConf *filer_pb.RemoteConf) (RemoteStorageClient, error) {
|
||||
maker, found := RemoteStorageClientMakers[remoteConf.Type]
|
||||
if !found {
|
||||
return nil, fmt.Errorf("remote storage type %s not found", remoteConf.Type)
|
||||
}
|
||||
return maker.Make(remoteConf)
|
||||
}
|
||||
|
||||
func GetRemoteStorage(remoteConf *filer_pb.RemoteConf) (RemoteStorageClient, error) {
|
||||
remoteStorageClientsLock.Lock()
|
||||
defer remoteStorageClientsLock.Unlock()
|
||||
|
||||
existingRemoteStorageClient, found := remoteStorageClients[remoteConf.Name]
|
||||
if found {
|
||||
return existingRemoteStorageClient, nil
|
||||
}
|
||||
|
||||
newRemoteStorageClient, err := makeRemoteStorageClient(remoteConf)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("make remote storage client %s: %v", remoteConf.Name, err)
|
||||
}
|
||||
|
||||
remoteStorageClients[remoteConf.Name] = newRemoteStorageClient
|
||||
|
||||
return newRemoteStorageClient, nil
|
||||
}
|
215
weed/remote_storage/s3/s3_storage_client.go
Normal file
215
weed/remote_storage/s3/s3_storage_client.go
Normal file
|
@ -0,0 +1,215 @@
|
|||
package s3
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/aws/aws-sdk-go/aws"
|
||||
"github.com/aws/aws-sdk-go/aws/credentials"
|
||||
"github.com/aws/aws-sdk-go/aws/session"
|
||||
"github.com/aws/aws-sdk-go/service/s3"
|
||||
"github.com/aws/aws-sdk-go/service/s3/s3iface"
|
||||
"github.com/aws/aws-sdk-go/service/s3/s3manager"
|
||||
"github.com/chrislusf/seaweedfs/weed/filer"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/remote_storage"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"io"
|
||||
)
|
||||
|
||||
func init() {
|
||||
remote_storage.RemoteStorageClientMakers["s3"] = new(s3RemoteStorageMaker)
|
||||
}
|
||||
|
||||
type s3RemoteStorageMaker struct{}
|
||||
|
||||
func (s s3RemoteStorageMaker) Make(conf *filer_pb.RemoteConf) (remote_storage.RemoteStorageClient, error) {
|
||||
client := &s3RemoteStorageClient{
|
||||
conf: conf,
|
||||
}
|
||||
config := &aws.Config{
|
||||
Region: aws.String(conf.S3Region),
|
||||
Endpoint: aws.String(conf.S3Endpoint),
|
||||
S3ForcePathStyle: aws.Bool(true),
|
||||
}
|
||||
if conf.S3AccessKey != "" && conf.S3SecretKey != "" {
|
||||
config.Credentials = credentials.NewStaticCredentials(conf.S3AccessKey, conf.S3SecretKey, "")
|
||||
}
|
||||
|
||||
sess, err := session.NewSession(config)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("create aws session: %v", err)
|
||||
}
|
||||
client.conn = s3.New(sess)
|
||||
return client, nil
|
||||
}
|
||||
|
||||
type s3RemoteStorageClient struct {
|
||||
conf *filer_pb.RemoteConf
|
||||
conn s3iface.S3API
|
||||
}
|
||||
|
||||
var _ = remote_storage.RemoteStorageClient(&s3RemoteStorageClient{})
|
||||
|
||||
func (s *s3RemoteStorageClient) Traverse(remote *filer_pb.RemoteStorageLocation, visitFn remote_storage.VisitFunc) (err error) {
|
||||
|
||||
pathKey := remote.Path[1:]
|
||||
|
||||
listInput := &s3.ListObjectsV2Input{
|
||||
Bucket: aws.String(remote.Bucket),
|
||||
ContinuationToken: nil,
|
||||
Delimiter: nil, // not aws.String("/"), iterate through all entries
|
||||
EncodingType: nil,
|
||||
ExpectedBucketOwner: nil,
|
||||
FetchOwner: nil,
|
||||
MaxKeys: nil, // aws.Int64(1000),
|
||||
Prefix: aws.String(pathKey),
|
||||
RequestPayer: nil,
|
||||
StartAfter: nil,
|
||||
}
|
||||
isLastPage := false
|
||||
for !isLastPage && err == nil {
|
||||
listErr := s.conn.ListObjectsV2Pages(listInput, func(page *s3.ListObjectsV2Output, lastPage bool) bool {
|
||||
for _, content := range page.Contents {
|
||||
key := *content.Key
|
||||
if len(pathKey) == 0 {
|
||||
key = "/" + key
|
||||
} else {
|
||||
key = key[len(pathKey):]
|
||||
}
|
||||
dir, name := util.FullPath(key).DirAndName()
|
||||
if err := visitFn(dir, name, false, &filer_pb.RemoteEntry{
|
||||
LastModifiedAt: (*content.LastModified).Unix(),
|
||||
Size: *content.Size,
|
||||
ETag: *content.ETag,
|
||||
StorageName: s.conf.Name,
|
||||
}); err != nil {
|
||||
return false
|
||||
}
|
||||
}
|
||||
listInput.ContinuationToken = page.NextContinuationToken
|
||||
isLastPage = lastPage
|
||||
return true
|
||||
})
|
||||
if listErr != nil {
|
||||
err = fmt.Errorf("list %v: %v", remote, listErr)
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
func (s *s3RemoteStorageClient) ReadFile(loc *filer_pb.RemoteStorageLocation, offset int64, size int64) (data []byte, err error) {
|
||||
downloader := s3manager.NewDownloaderWithClient(s.conn, func(u *s3manager.Downloader) {
|
||||
u.PartSize = int64(4 * 1024 * 1024)
|
||||
u.Concurrency = 1
|
||||
})
|
||||
|
||||
dataSlice := make([]byte, int(size))
|
||||
writerAt := aws.NewWriteAtBuffer(dataSlice)
|
||||
|
||||
_, err = downloader.Download(writerAt, &s3.GetObjectInput{
|
||||
Bucket: aws.String(loc.Bucket),
|
||||
Key: aws.String(loc.Path[1:]),
|
||||
Range: aws.String(fmt.Sprintf("bytes=%d-%d", offset, offset+size-1)),
|
||||
})
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to download file %s%s: %v", loc.Bucket, loc.Path, err)
|
||||
}
|
||||
|
||||
return writerAt.Bytes(), nil
|
||||
}
|
||||
|
||||
func (s *s3RemoteStorageClient) WriteFile(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry, reader io.Reader) (remoteEntry *filer_pb.RemoteEntry, err error) {
|
||||
|
||||
fileSize := int64(filer.FileSize(entry))
|
||||
|
||||
partSize := int64(8 * 1024 * 1024) // The minimum/default allowed part size is 5MB
|
||||
for partSize*1000 < fileSize {
|
||||
partSize *= 4
|
||||
}
|
||||
|
||||
// Create an uploader with the session and custom options
|
||||
uploader := s3manager.NewUploaderWithClient(s.conn, func(u *s3manager.Uploader) {
|
||||
u.PartSize = partSize
|
||||
u.Concurrency = 5
|
||||
})
|
||||
|
||||
// process tagging
|
||||
tags := ""
|
||||
for k, v := range entry.Extended {
|
||||
if len(tags) > 0 {
|
||||
tags = tags + "&"
|
||||
}
|
||||
tags = tags + k + "=" + string(v)
|
||||
}
|
||||
|
||||
// Upload the file to S3.
|
||||
_, err = uploader.Upload(&s3manager.UploadInput{
|
||||
Bucket: aws.String(loc.Bucket),
|
||||
Key: aws.String(loc.Path[1:]),
|
||||
Body: reader,
|
||||
ACL: aws.String("private"),
|
||||
ServerSideEncryption: aws.String("AES256"),
|
||||
StorageClass: aws.String("STANDARD_IA"),
|
||||
Tagging: aws.String(tags),
|
||||
})
|
||||
|
||||
//in case it fails to upload
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("upload to s3 %s/%s%s: %v", loc.Name, loc.Bucket, loc.Path, err)
|
||||
}
|
||||
|
||||
// read back the remote entry
|
||||
return s.readFileRemoteEntry(loc)
|
||||
|
||||
}
|
||||
|
||||
func toTagging(attributes map[string][]byte) *s3.Tagging {
|
||||
tagging := &s3.Tagging{}
|
||||
for k, v := range attributes {
|
||||
tagging.TagSet = append(tagging.TagSet, &s3.Tag{
|
||||
Key: aws.String(k),
|
||||
Value: aws.String(string(v)),
|
||||
})
|
||||
}
|
||||
return tagging
|
||||
}
|
||||
|
||||
func (s *s3RemoteStorageClient) readFileRemoteEntry(loc *filer_pb.RemoteStorageLocation) (*filer_pb.RemoteEntry, error) {
|
||||
resp, err := s.conn.HeadObject(&s3.HeadObjectInput{
|
||||
Bucket: aws.String(loc.Bucket),
|
||||
Key: aws.String(loc.Path[1:]),
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &filer_pb.RemoteEntry{
|
||||
LastModifiedAt: resp.LastModified.Unix(),
|
||||
Size: *resp.ContentLength,
|
||||
ETag: *resp.ETag,
|
||||
StorageName: s.conf.Name,
|
||||
}, nil
|
||||
|
||||
}
|
||||
|
||||
func (s *s3RemoteStorageClient) UpdateFileMetadata(loc *filer_pb.RemoteStorageLocation, entry *filer_pb.Entry) (err error) {
|
||||
tagging := toTagging(entry.Extended)
|
||||
if len(tagging.TagSet) > 0 {
|
||||
_, err = s.conn.PutObjectTagging(&s3.PutObjectTaggingInput{
|
||||
Bucket: aws.String(loc.Bucket),
|
||||
Key: aws.String(loc.Path[1:]),
|
||||
Tagging: toTagging(entry.Extended),
|
||||
})
|
||||
} else {
|
||||
_, err = s.conn.DeleteObjectTagging(&s3.DeleteObjectTaggingInput{
|
||||
Bucket: aws.String(loc.Bucket),
|
||||
Key: aws.String(loc.Path[1:]),
|
||||
})
|
||||
}
|
||||
return
|
||||
}
|
||||
func (s *s3RemoteStorageClient) DeleteFile(loc *filer_pb.RemoteStorageLocation) (err error) {
|
||||
_, err = s.conn.DeleteObject(&s3.DeleteObjectInput{
|
||||
Bucket: aws.String(loc.Bucket),
|
||||
Key: aws.String(loc.Path[1:]),
|
||||
})
|
||||
return
|
||||
}
|
|
@ -134,7 +134,7 @@ func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry, signatures [
|
|||
Attributes: entry.Attributes,
|
||||
Chunks: replicatedChunks,
|
||||
Content: entry.Content,
|
||||
Remote: entry.Remote,
|
||||
RemoteEntry: entry.RemoteEntry,
|
||||
},
|
||||
IsFromOtherCluster: true,
|
||||
Signatures: signatures,
|
||||
|
|
|
@ -1,13 +1,11 @@
|
|||
package s3api
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/filer"
|
||||
"github.com/chrislusf/seaweedfs/weed/glog"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"io"
|
||||
"time"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
)
|
||||
|
||||
func (s3a *S3ApiServer) subscribeMetaEvents(clientName string, prefix string, lastTsNs int64) error {
|
||||
|
@ -34,37 +32,8 @@ func (s3a *S3ApiServer) subscribeMetaEvents(clientName string, prefix string, la
|
|||
return nil
|
||||
}
|
||||
|
||||
for {
|
||||
err := s3a.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: clientName,
|
||||
PathPrefix: prefix,
|
||||
SinceNs: lastTsNs,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("subscribe: %v", err)
|
||||
}
|
||||
return util.Retry("followIamChanges", func() error {
|
||||
return pb.WithFilerClientFollowMetadata(s3a, clientName, prefix, lastTsNs, 0, processEventFn, true)
|
||||
})
|
||||
|
||||
for {
|
||||
resp, listenErr := stream.Recv()
|
||||
if listenErr == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if listenErr != nil {
|
||||
return listenErr
|
||||
}
|
||||
|
||||
if err := processEventFn(resp); err != nil {
|
||||
glog.Fatalf("process %v: %v", resp, err)
|
||||
}
|
||||
lastTsNs = resp.TsNs
|
||||
}
|
||||
})
|
||||
if err != nil {
|
||||
glog.Errorf("subscribing filer meta change: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -149,6 +149,8 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
|
|||
|
||||
fs.filer.LoadFilerConf()
|
||||
|
||||
fs.filer.LoadRemoteStorageConfAndMapping()
|
||||
|
||||
grace.OnInterrupt(func() {
|
||||
fs.filer.Shutdown()
|
||||
})
|
||||
|
|
|
@ -101,7 +101,7 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
|
|||
|
||||
//Seaweed custom header are not visible to Vue or javascript
|
||||
seaweedHeaders := []string{}
|
||||
for header, _ := range w.Header() {
|
||||
for header := range w.Header() {
|
||||
if strings.HasPrefix(header, "Seaweed-") {
|
||||
seaweedHeaders = append(seaweedHeaders, header)
|
||||
}
|
||||
|
@ -163,9 +163,18 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
|
|||
}
|
||||
return err
|
||||
}
|
||||
err = filer.StreamContent(fs.filer.MasterClient, writer, entry.Chunks, offset, size)
|
||||
if err != nil {
|
||||
glog.Errorf("failed to stream content %s: %v", r.URL, err)
|
||||
if entry.IsInRemoteOnly() {
|
||||
var data []byte
|
||||
data, err = fs.filer.ReadRemote(entry, offset, size)
|
||||
if err != nil {
|
||||
glog.Errorf("failed to read remote %s: %v", r.URL, err)
|
||||
}
|
||||
_, err = w.Write(data)
|
||||
} else {
|
||||
err = filer.StreamContent(fs.filer.MasterClient, writer, entry.Chunks, offset, size)
|
||||
if err != nil {
|
||||
glog.Errorf("failed to stream content %s: %v", r.URL, err)
|
||||
}
|
||||
}
|
||||
return err
|
||||
})
|
||||
|
|
|
@ -228,6 +228,7 @@ func (ms *MasterServer) startAdminScripts() {
|
|||
shellOptions.Masters = &masterAddress
|
||||
|
||||
shellOptions.FilerHost, shellOptions.FilerPort, err = util.ParseHostPort(filerHostPort)
|
||||
shellOptions.FilerAddress = filerHostPort
|
||||
shellOptions.Directory = "/"
|
||||
if err != nil {
|
||||
glog.V(0).Infof("failed to parse master.filer.default = %s : %v\n", filerHostPort, err)
|
||||
|
|
49
weed/server/volume_grpc_remote.go
Normal file
49
weed/server/volume_grpc_remote.go
Normal file
|
@ -0,0 +1,49 @@
|
|||
package weed_server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/remote_storage"
|
||||
"github.com/chrislusf/seaweedfs/weed/storage/needle"
|
||||
"github.com/chrislusf/seaweedfs/weed/storage/types"
|
||||
)
|
||||
|
||||
func (vs *VolumeServer) FetchAndWriteNeedle(ctx context.Context, req *volume_server_pb.FetchAndWriteNeedleRequest) (resp *volume_server_pb.FetchAndWriteNeedleResponse, err error) {
|
||||
resp = &volume_server_pb.FetchAndWriteNeedleResponse{}
|
||||
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
|
||||
if v == nil {
|
||||
return nil, fmt.Errorf("not found volume id %d", req.VolumeId)
|
||||
}
|
||||
|
||||
remoteConf := &filer_pb.RemoteConf{
|
||||
Type: req.RemoteType,
|
||||
Name: req.RemoteName,
|
||||
S3AccessKey: req.S3AccessKey,
|
||||
S3SecretKey: req.S3SecretKey,
|
||||
S3Region: req.S3Region,
|
||||
S3Endpoint: req.S3Endpoint,
|
||||
}
|
||||
|
||||
client, getClientErr := remote_storage.GetRemoteStorage(remoteConf)
|
||||
if getClientErr != nil {
|
||||
return nil, fmt.Errorf("get remote client: %v", getClientErr)
|
||||
}
|
||||
|
||||
remoteStorageLocation := &filer_pb.RemoteStorageLocation{
|
||||
Name: req.RemoteName,
|
||||
Bucket: req.RemoteBucket,
|
||||
Path: req.RemoteKey,
|
||||
}
|
||||
data, ReadRemoteErr := client.ReadFile(remoteStorageLocation, req.Offset, req.Size)
|
||||
if ReadRemoteErr != nil {
|
||||
return nil, fmt.Errorf("read from remote %+v: %v", remoteStorageLocation, ReadRemoteErr)
|
||||
}
|
||||
|
||||
if err = v.WriteNeedleBlob(types.NeedleId(req.NeedleId), data, types.Size(req.Size)); err != nil {
|
||||
return nil, fmt.Errorf("write blob needle %d size %d: %v", req.NeedleId, req.Size, err)
|
||||
}
|
||||
|
||||
return resp, nil
|
||||
}
|
|
@ -7,9 +7,11 @@ import (
|
|||
"github.com/chrislusf/seaweedfs/weed/filer"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"io"
|
||||
"regexp"
|
||||
"strings"
|
||||
)
|
||||
|
||||
func init() {
|
||||
|
@ -84,6 +86,9 @@ func (c *commandRemoteConfigure) listExistingRemoteStorages(commandEnv *CommandE
|
|||
fmt.Fprintf(writer, "skipping %s\n", entry.Name)
|
||||
return nil
|
||||
}
|
||||
if !strings.HasSuffix(entry.Name, filer.REMOTE_STORAGE_CONF_SUFFIX) {
|
||||
return nil
|
||||
}
|
||||
conf := &filer_pb.RemoteConf{}
|
||||
|
||||
if err := proto.Unmarshal(entry.Content, conf); err != nil {
|
||||
|
@ -92,9 +97,15 @@ func (c *commandRemoteConfigure) listExistingRemoteStorages(commandEnv *CommandE
|
|||
|
||||
conf.S3SecretKey = ""
|
||||
|
||||
fmt.Fprintf(writer, "%+v\n", conf)
|
||||
m := jsonpb.Marshaler{
|
||||
EmitDefaults: false,
|
||||
Indent: " ",
|
||||
}
|
||||
|
||||
return nil
|
||||
err := m.Marshal(writer, conf)
|
||||
fmt.Fprintln(writer)
|
||||
|
||||
return err
|
||||
})
|
||||
|
||||
}
|
||||
|
@ -105,7 +116,7 @@ func (c *commandRemoteConfigure) deleteRemoteStorage(commandEnv *CommandEnv, wri
|
|||
|
||||
request := &filer_pb.DeleteEntryRequest{
|
||||
Directory: filer.DirectoryEtcRemote,
|
||||
Name: storageName,
|
||||
Name: storageName + filer.REMOTE_STORAGE_CONF_SUFFIX,
|
||||
IgnoreRecursiveError: false,
|
||||
IsDeleteData: true,
|
||||
IsRecursive: true,
|
||||
|
@ -132,7 +143,7 @@ func (c *commandRemoteConfigure) saveRemoteStorage(commandEnv *CommandEnv, write
|
|||
}
|
||||
|
||||
if err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
return filer.SaveInsideFiler(client, filer.DirectoryEtcRemote, conf.Name, data)
|
||||
return filer.SaveInsideFiler(client, filer.DirectoryEtcRemote, conf.Name+filer.REMOTE_STORAGE_CONF_SUFFIX, data)
|
||||
}); err != nil && err != filer_pb.ErrNotFound {
|
||||
return err
|
||||
}
|
||||
|
|
232
weed/shell/command_remote_mount.go
Normal file
232
weed/shell/command_remote_mount.go
Normal file
|
@ -0,0 +1,232 @@
|
|||
package shell
|
||||
|
||||
import (
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/filer"
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/remote_storage"
|
||||
"github.com/chrislusf/seaweedfs/weed/util"
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
"io"
|
||||
)
|
||||
|
||||
func init() {
|
||||
Commands = append(Commands, &commandRemoteMount{})
|
||||
}
|
||||
|
||||
type commandRemoteMount struct {
|
||||
}
|
||||
|
||||
func (c *commandRemoteMount) Name() string {
|
||||
return "remote.mount"
|
||||
}
|
||||
|
||||
func (c *commandRemoteMount) Help() string {
|
||||
return `mount remote storage and pull its metadata
|
||||
|
||||
# assume a remote storage is configured to name "s3_1"
|
||||
remote.configure -name=s3_1 -type=s3 -access_key=xxx -secret_key=yyy
|
||||
|
||||
# mount and pull one bucket
|
||||
remote.mount -dir=xxx -remote=s3_1/bucket
|
||||
# mount and pull one directory in the bucket
|
||||
remote.mount -dir=xxx -remote=s3_1/bucket/dir1
|
||||
|
||||
`
|
||||
}
|
||||
|
||||
func (c *commandRemoteMount) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
|
||||
|
||||
remoteMountCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
|
||||
|
||||
dir := remoteMountCommand.String("dir", "", "a directory in filer")
|
||||
nonEmpty := remoteMountCommand.Bool("nonempty", false, "allows the mounting over a non-empty directory")
|
||||
remote := remoteMountCommand.String("remote", "", "a directory in remote storage, ex. <storageName>/<bucket>/path/to/dir")
|
||||
|
||||
if err = remoteMountCommand.Parse(args); err != nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if *dir == "" {
|
||||
return c.listExistingRemoteStorageMounts(commandEnv, writer)
|
||||
}
|
||||
|
||||
remoteStorageLocation := remote_storage.ParseLocation(*remote)
|
||||
|
||||
// find configuration for remote storage
|
||||
// remotePath is /<bucket>/path/to/dir
|
||||
remoteConf, err := c.findRemoteStorageConfiguration(commandEnv, writer, remoteStorageLocation)
|
||||
if err != nil {
|
||||
return fmt.Errorf("find configuration for %s: %v", *remote, err)
|
||||
}
|
||||
|
||||
// pull metadata from remote
|
||||
if err = c.pullMetadata(commandEnv, writer, *dir, *nonEmpty, remoteConf, remoteStorageLocation); err != nil {
|
||||
return fmt.Errorf("pull metadata: %v", err)
|
||||
}
|
||||
|
||||
// store a mount configuration in filer
|
||||
if err = c.saveMountMapping(commandEnv, writer, *dir, remoteStorageLocation); err != nil {
|
||||
return fmt.Errorf("save mount mapping: %v", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *commandRemoteMount) listExistingRemoteStorageMounts(commandEnv *CommandEnv, writer io.Writer) (err error) {
|
||||
|
||||
// read current mapping
|
||||
mappings, readErr := filer.ReadMountMappings(commandEnv.option.GrpcDialOption, commandEnv.option.FilerAddress)
|
||||
if readErr != nil {
|
||||
return readErr
|
||||
}
|
||||
|
||||
m := jsonpb.Marshaler{
|
||||
EmitDefaults: false,
|
||||
Indent: " ",
|
||||
}
|
||||
|
||||
err = m.Marshal(writer, mappings)
|
||||
fmt.Fprintln(writer)
|
||||
|
||||
return
|
||||
|
||||
}
|
||||
|
||||
func (c *commandRemoteMount) findRemoteStorageConfiguration(commandEnv *CommandEnv, writer io.Writer, remote *filer_pb.RemoteStorageLocation) (conf *filer_pb.RemoteConf, err error) {
|
||||
|
||||
return filer.ReadRemoteStorageConf(commandEnv.option.GrpcDialOption, commandEnv.option.FilerAddress, remote.Name)
|
||||
|
||||
}
|
||||
|
||||
func (c *commandRemoteMount) pullMetadata(commandEnv *CommandEnv, writer io.Writer, dir string, nonEmpty bool, remoteConf *filer_pb.RemoteConf, remote *filer_pb.RemoteStorageLocation) error {
|
||||
|
||||
// find existing directory, and ensure the directory is empty
|
||||
err := commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
parent, name := util.FullPath(dir).DirAndName()
|
||||
_, lookupErr := client.LookupDirectoryEntry(context.Background(), &filer_pb.LookupDirectoryEntryRequest{
|
||||
Directory: parent,
|
||||
Name: name,
|
||||
})
|
||||
if lookupErr != nil {
|
||||
return fmt.Errorf("lookup %s: %v", dir, lookupErr)
|
||||
}
|
||||
|
||||
mountToDirIsEmpty := true
|
||||
listErr := filer_pb.SeaweedList(client, dir, "", func(entry *filer_pb.Entry, isLast bool) error {
|
||||
mountToDirIsEmpty = false
|
||||
return nil
|
||||
}, "", false, 1)
|
||||
|
||||
if listErr != nil {
|
||||
return fmt.Errorf("list %s: %v", dir, listErr)
|
||||
}
|
||||
|
||||
if !mountToDirIsEmpty {
|
||||
if !nonEmpty {
|
||||
return fmt.Errorf("dir %s is not empty", dir)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// visit remote storage
|
||||
remoteStorage, err := remote_storage.GetRemoteStorage(remoteConf)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
ctx := context.Background()
|
||||
err = remoteStorage.Traverse(remote, func(remoteDir, name string, isDirectory bool, remoteEntry *filer_pb.RemoteEntry) error {
|
||||
localDir := dir + remoteDir
|
||||
println(util.NewFullPath(localDir, name))
|
||||
|
||||
lookupResponse, lookupErr := filer_pb.LookupEntry(client, &filer_pb.LookupDirectoryEntryRequest{
|
||||
Directory: localDir,
|
||||
Name: name,
|
||||
})
|
||||
var existingEntry *filer_pb.Entry
|
||||
if lookupErr != nil {
|
||||
if lookupErr != filer_pb.ErrNotFound {
|
||||
return lookupErr
|
||||
}
|
||||
} else {
|
||||
existingEntry = lookupResponse.Entry
|
||||
}
|
||||
|
||||
if existingEntry == nil {
|
||||
_, createErr := client.CreateEntry(ctx, &filer_pb.CreateEntryRequest{
|
||||
Directory: localDir,
|
||||
Entry: &filer_pb.Entry{
|
||||
Name: name,
|
||||
IsDirectory: isDirectory,
|
||||
Attributes: &filer_pb.FuseAttributes{
|
||||
FileSize: uint64(remoteEntry.Size),
|
||||
Mtime: remoteEntry.LastModifiedAt,
|
||||
FileMode: uint32(0644),
|
||||
},
|
||||
RemoteEntry: remoteEntry,
|
||||
},
|
||||
})
|
||||
return createErr
|
||||
} else {
|
||||
if existingEntry.RemoteEntry == nil || existingEntry.RemoteEntry.ETag != remoteEntry.ETag {
|
||||
existingEntry.RemoteEntry = remoteEntry
|
||||
existingEntry.Attributes.FileSize = uint64(remoteEntry.Size)
|
||||
existingEntry.Attributes.Mtime = remoteEntry.LastModifiedAt
|
||||
_, updateErr := client.UpdateEntry(ctx, &filer_pb.UpdateEntryRequest{
|
||||
Directory: localDir,
|
||||
Entry: existingEntry,
|
||||
})
|
||||
return updateErr
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
return err
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *commandRemoteMount) saveMountMapping(commandEnv *CommandEnv, writer io.Writer, dir string, remoteStorageLocation *filer_pb.RemoteStorageLocation) (err error) {
|
||||
|
||||
// read current mapping
|
||||
var oldContent, newContent []byte
|
||||
err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
oldContent, err = filer.ReadInsideFiler(client, filer.DirectoryEtcRemote, filer.REMOTE_STORAGE_MOUNT_FILE)
|
||||
return err
|
||||
})
|
||||
if err != nil {
|
||||
if err != filer_pb.ErrNotFound {
|
||||
return fmt.Errorf("read existing mapping: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// add new mapping
|
||||
newContent, err = filer.AddRemoteStorageMapping(oldContent, dir, remoteStorageLocation)
|
||||
if err != nil {
|
||||
return fmt.Errorf("add mapping %s~%s: %v", dir, remoteStorageLocation, err)
|
||||
}
|
||||
|
||||
// save back
|
||||
err = commandEnv.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
|
||||
return filer.SaveInsideFiler(client, filer.DirectoryEtcRemote, filer.REMOTE_STORAGE_MOUNT_FILE, newContent)
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("save mapping: %v", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -1,6 +1,8 @@
|
|||
package shell
|
||||
|
||||
import (
|
||||
"github.com/chrislusf/seaweedfs/weed/storage/types"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"testing"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
|
||||
|
@ -181,3 +183,14 @@ func TestBalance(t *testing.T) {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
func TestVolumeSelection(t *testing.T) {
|
||||
topologyInfo := parseOutput(topoData)
|
||||
|
||||
vids, err := collectVolumeIdsForTierChange(nil, topologyInfo, 1000, types.ToDiskType("hdd"), "", 20.0, 0);
|
||||
if err != nil {
|
||||
t.Errorf("collectVolumeIdsForTierChange: %v", err)
|
||||
}
|
||||
assert.Equal(t, 378, len(vids))
|
||||
|
||||
}
|
||||
|
|
|
@ -68,7 +68,7 @@ func parseOutput(output string) *master_pb.TopologyInfo {
|
|||
maxVolumeCount, _ := strconv.Atoi(maxVolumeCountStr)
|
||||
disk = &master_pb.DiskInfo{
|
||||
Type: diskType,
|
||||
MaxVolumeCount: uint64(maxVolumeCount),
|
||||
MaxVolumeCount: int64(maxVolumeCount),
|
||||
}
|
||||
dn.DiskInfos[types.ToDiskType(diskType).String()] = disk
|
||||
} else {
|
||||
|
|
|
@ -20,9 +20,10 @@ type ShellOptions struct {
|
|||
Masters *string
|
||||
GrpcDialOption grpc.DialOption
|
||||
// shell transient context
|
||||
FilerHost string
|
||||
FilerPort int64
|
||||
Directory string
|
||||
FilerHost string
|
||||
FilerPort int64
|
||||
FilerAddress string
|
||||
Directory string
|
||||
}
|
||||
|
||||
type CommandEnv struct {
|
||||
|
|
|
@ -50,7 +50,7 @@ func ReplicatedWrite(masterFn operation.GetMasterFn, s *storage.Store, volumeId
|
|||
}
|
||||
|
||||
if len(remoteLocations) > 0 { //send to other replica locations
|
||||
if err = distributedOperation(remoteLocations, s, func(location operation.Location) error {
|
||||
if err = DistributedOperation(remoteLocations, func(location operation.Location) error {
|
||||
u := url.URL{
|
||||
Scheme: "http",
|
||||
Host: location.Url,
|
||||
|
@ -115,7 +115,7 @@ func ReplicatedDelete(masterFn operation.GetMasterFn, store *storage.Store,
|
|||
}
|
||||
|
||||
if len(remoteLocations) > 0 { //send to other replica locations
|
||||
if err = distributedOperation(remoteLocations, store, func(location operation.Location) error {
|
||||
if err = DistributedOperation(remoteLocations, func(location operation.Location) error {
|
||||
return util.Delete("http://"+location.Url+r.URL.Path+"?type=replicate", string(jwt))
|
||||
}); err != nil {
|
||||
size = 0
|
||||
|
@ -144,7 +144,7 @@ type RemoteResult struct {
|
|||
Error error
|
||||
}
|
||||
|
||||
func distributedOperation(locations []operation.Location, store *storage.Store, op func(location operation.Location) error) error {
|
||||
func DistributedOperation(locations []operation.Location, op func(location operation.Location) error) error {
|
||||
length := len(locations)
|
||||
results := make(chan RemoteResult)
|
||||
for _, location := range locations {
|
||||
|
|
Loading…
Reference in a new issue